From 122a9e1e671ec7efe7c3dd1fee8bc8c25e53cf04 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Mon, 27 Sep 2021 20:16:22 +0530 Subject: [PATCH 01/26] Add Hudi plugin Add HudiPageSource implementation Add split manager and metadata implementation --- .../parquet/ParquetPageSourceFactory.java | 4 +- plugin/trino-hudi/pom.xml | 154 ++++++++++ .../java/io/trino/plugin/hudi/HudiConfig.java | 58 ++++ .../io/trino/plugin/hudi/HudiConnector.java | 192 ++++++++++++ .../plugin/hudi/HudiConnectorFactory.java | 89 ++++++ .../io/trino/plugin/hudi/HudiErrorCode.java | 53 ++++ .../io/trino/plugin/hudi/HudiFileFormat.java | 21 ++ .../trino/plugin/hudi/HudiHandleResolver.java | 67 +++++ .../io/trino/plugin/hudi/HudiMetadata.java | 66 ++++ .../plugin/hudi/HudiMetadataFactory.java | 40 +++ .../plugin/hudi/HudiMetastoreModule.java | 79 +++++ .../java/io/trino/plugin/hudi/HudiModule.java | 79 +++++ .../plugin/hudi/HudiPageSinkProvider.java | 51 ++++ .../plugin/hudi/HudiPageSourceProvider.java | 281 ++++++++++++++++++ .../java/io/trino/plugin/hudi/HudiPlugin.java | 29 ++ .../plugin/hudi/HudiSessionProperties.java | 30 ++ .../java/io/trino/plugin/hudi/HudiSplit.java | 123 ++++++++ .../trino/plugin/hudi/HudiSplitManager.java | 66 ++++ .../io/trino/plugin/hudi/HudiSplitSource.java | 74 +++++ .../io/trino/plugin/hudi/HudiTableHandle.java | 146 +++++++++ .../plugin/hudi/HudiTableProperties.java | 69 +++++ .../plugin/hudi/HudiTransactionManager.java | 49 +++ .../hudi/InternalHudiConnectorFactory.java | 124 ++++++++ pom.xml | 7 + 24 files changed, 1949 insertions(+), 2 deletions(-) create mode 100644 plugin/trino-hudi/pom.xml create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnector.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiErrorCode.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiFileFormat.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiHandleResolver.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetastoreModule.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiModule.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSinkProvider.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPlugin.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableProperties.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTransactionManager.java create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/InternalHudiConnectorFactory.java diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java index d72571d2736..033050ce73d 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java @@ -365,7 +365,7 @@ public static Optional getColumnType(HiveColumnH return Optional.of(new GroupType(baseType.getRepetition(), baseType.getName(), ImmutableList.of(type))); } - private static Optional getColumnIndexStore( + public static Optional getColumnIndexStore( ParquetDataSource dataSource, BlockMetaData blockMetadata, Map, RichColumnDescriptor> descriptorsByPath, @@ -440,7 +440,7 @@ public static TupleDomain getParquetTupleDomain( return TupleDomain.withColumnDomains(predicate.build()); } - private static org.apache.parquet.schema.Type getParquetType(HiveColumnHandle column, MessageType messageType, boolean useParquetColumnNames) + public static org.apache.parquet.schema.Type getParquetType(HiveColumnHandle column, MessageType messageType, boolean useParquetColumnNames) { if (useParquetColumnNames) { return getParquetTypeByName(column.getBaseColumnName(), messageType); diff --git a/plugin/trino-hudi/pom.xml b/plugin/trino-hudi/pom.xml new file mode 100644 index 00000000000..5edb7136f22 --- /dev/null +++ b/plugin/trino-hudi/pom.xml @@ -0,0 +1,154 @@ + + + 4.0.0 + + + trino-root + io.trino + 366-SNAPSHOT + ../../pom.xml + + + trino-hudi + Trino - Hudi Connector + trino-plugin + + + ${project.parent.basedir} + 0.9.0 + + + + + io.trino + trino-hive + + + io.trino + trino-memory-context + + + io.trino + trino-parquet + + + io.trino + trino-plugin-toolkit + + + io.trino.hadoop + hadoop-apache + + + io.trino.hive + hive-apache + + + io.airlift + bootstrap + + + io.airlift + configuration + + + io.airlift + event + + + io.airlift + json + + + io.airlift + log + + + com.google.guava + guava + + + com.google.inject + guice + + + javax.inject + javax.inject + + + javax.validation + validation-api + + + joda-time + joda-time + + + org.apache.hudi + hudi-common + ${dep.hudi.version} + + + org.apache.hbase + hbase-server + + + org.apache.orc + orc-core + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + fluent-hc + + + org.rocksdb + rocksdbjni + + + com.esotericsoftware + kryo-shaded + + + + + org.weakref + jmxutils + + + + io.trino + trino-spi + provided + + + io.airlift + slice + provided + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + org.openjdk.jol + jol-core + provided + + + + diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java new file mode 100644 index 00000000000..42c4a1f642b --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java @@ -0,0 +1,58 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; +import org.apache.hudi.common.model.HoodieFileFormat; + +import javax.validation.constraints.NotNull; + +import java.util.TimeZone; + +import static io.trino.plugin.hudi.HudiFileFormat.PARQUET; + +public class HudiConfig +{ + private HudiFileFormat fileFormat = PARQUET; + private String parquetTimeZone = TimeZone.getDefault().getID(); + + @NotNull + public HoodieFileFormat getFileFormat() + { + return HoodieFileFormat.valueOf(fileFormat.name()); + } + + @Config("hudi.file-format") + public HudiConfig setFileFormat(HudiFileFormat fileFormat) + { + this.fileFormat = fileFormat; + return this; + } + + @NotNull + public String getParquetTimeZone() + { + return parquetTimeZone; + } + + @Config("hudi.parquet.time-zone") + @ConfigDescription("Time zone for Parquet read and write") + public HudiConfig setParquetTimeZone(String parquetTimeZone) + { + this.parquetTimeZone = parquetTimeZone; + return this; + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnector.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnector.java new file mode 100644 index 00000000000..a899c72bbdd --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnector.java @@ -0,0 +1,192 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import io.airlift.bootstrap.LifeCycleManager; +import io.airlift.log.Logger; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata; +import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.plugin.hive.HiveTransactionHandle; +import io.trino.spi.classloader.ThreadContextClassLoader; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorAccessControl; +import io.trino.spi.connector.ConnectorMetadata; +import io.trino.spi.connector.ConnectorNodePartitioningProvider; +import io.trino.spi.connector.ConnectorPageSinkProvider; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorSplitManager; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.connector.SystemTable; +import io.trino.spi.procedure.Procedure; +import io.trino.spi.session.PropertyMetadata; +import io.trino.spi.transaction.IsolationLevel; + +import java.util.List; +import java.util.Optional; +import java.util.Set; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.spi.transaction.IsolationLevel.SERIALIZABLE; +import static io.trino.spi.transaction.IsolationLevel.checkConnectorSupports; +import static java.util.Objects.requireNonNull; + +public class HudiConnector + implements Connector +{ + private static final Logger log = Logger.get(HudiConnector.class); + + private final LifeCycleManager lifeCycleManager; + private final HudiTransactionManager transactionManager; + private final HudiMetadataFactory metadataFactory; + private final ConnectorSplitManager splitManager; + private final ConnectorPageSourceProvider pageSourceProvider; + private final ConnectorPageSinkProvider pageSinkProvider; + private final ConnectorNodePartitioningProvider nodePartitioningProvider; + private final Set systemTables; + private final List> sessionProperties; + private final List> tableProperties; + private final Optional accessControl; + private final Set procedures; + + public HudiConnector( + LifeCycleManager lifeCycleManager, + HudiTransactionManager transactionManager, + HudiMetadataFactory metadataFactory, + ConnectorSplitManager splitManager, + ConnectorPageSourceProvider pageSourceProvider, + ConnectorPageSinkProvider pageSinkProvider, + ConnectorNodePartitioningProvider nodePartitioningProvider, + Set systemTables, + Set sessionPropertiesProviders, + List> tableProperties, + Optional accessControl, + Set procedures) + { + this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); + this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); + this.metadataFactory = requireNonNull(metadataFactory, "metadataFactory is null"); + this.splitManager = requireNonNull(splitManager, "splitManager is null"); + this.pageSourceProvider = requireNonNull(pageSourceProvider, "pageSourceProvider is null"); + this.pageSinkProvider = requireNonNull(pageSinkProvider, "pageSinkProvider is null"); + this.nodePartitioningProvider = requireNonNull(nodePartitioningProvider, "nodePartitioningProvider is null"); + this.systemTables = ImmutableSet.copyOf(requireNonNull(systemTables, "systemTables is null")); + this.sessionProperties = requireNonNull(sessionPropertiesProviders, "sessionPropertiesProviders is null").stream() + .flatMap(sessionPropertiesProvider -> sessionPropertiesProvider.getSessionProperties().stream()) + .collect(toImmutableList()); + this.tableProperties = ImmutableList.copyOf(requireNonNull(tableProperties, "tableProperties is null")); + this.accessControl = requireNonNull(accessControl, "accessControl is null"); + this.procedures = requireNonNull(procedures, "procedures is null"); + } + + @Override + public ConnectorMetadata getMetadata(ConnectorTransactionHandle transactionHandle) + { + ConnectorMetadata metadata = transactionManager.get(transactionHandle); + return new ClassLoaderSafeConnectorMetadata(metadata, getClass().getClassLoader()); + } + + @Override + public ConnectorSplitManager getSplitManager() + { + return splitManager; + } + + @Override + public ConnectorPageSourceProvider getPageSourceProvider() + { + return pageSourceProvider; + } + + @Override + public ConnectorPageSinkProvider getPageSinkProvider() + { + return pageSinkProvider; + } + + @Override + public ConnectorNodePartitioningProvider getNodePartitioningProvider() + { + return nodePartitioningProvider; + } + + @Override + public Set getSystemTables() + { + return systemTables; + } + + @Override + public Set getProcedures() + { + return procedures; + } + + @Override + public List> getSessionProperties() + { + return sessionProperties; + } + + @Override + public List> getTableProperties() + { + return tableProperties; + } + + @Override + public ConnectorAccessControl getAccessControl() + { + return accessControl.orElseThrow(UnsupportedOperationException::new); + } + + @Override + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly, boolean autoCommit) + { + checkConnectorSupports(SERIALIZABLE, isolationLevel); + ConnectorTransactionHandle transaction = new HiveTransactionHandle(); + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(getClass().getClassLoader())) { + transactionManager.put(transaction, metadataFactory.create()); + } + return transaction; + } + + @Override + public void commit(ConnectorTransactionHandle transaction) + { + transactionManager.remove(transaction); + } + + @Override + public void rollback(ConnectorTransactionHandle transaction) + { + HudiMetadata metadata = transactionManager.remove(transaction); + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(getClass().getClassLoader())) { + metadata.rollback(); + } + } + + @Override + public final void shutdown() + { + try { + lifeCycleManager.stop(); + } + catch (Exception e) { + log.error(e, "Error shutting down connector"); + } + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java new file mode 100644 index 00000000000..1e016bd49de --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java @@ -0,0 +1,89 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.inject.Binder; +import com.google.inject.Module; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorContext; +import io.trino.spi.connector.ConnectorFactory; +import io.trino.spi.connector.ConnectorHandleResolver; + +import java.lang.reflect.InvocationTargetException; +import java.util.Map; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static com.google.common.base.Throwables.throwIfUnchecked; +import static java.util.Objects.requireNonNull; + +public class HudiConnectorFactory + implements ConnectorFactory +{ + private final String name; + private final Class module; + + public HudiConnectorFactory(String name) + { + this(name, EmptyModule.class); + } + + public HudiConnectorFactory(String name, Class module) + { + checkArgument(!isNullOrEmpty(name), "name is null or empty"); + this.name = name; + this.module = requireNonNull(module, "module is null"); + } + + @Override + public String getName() + { + return name; + } + + @Override + public ConnectorHandleResolver getHandleResolver() + { + return new HudiHandleResolver(); + } + + @Override + public Connector create(String catalogName, Map config, ConnectorContext context) + { + ClassLoader classLoader = context.duplicatePluginClassLoader(); + try { + Object moduleInstance = classLoader.loadClass(module.getName()).getConstructor().newInstance(); + Class moduleClass = classLoader.loadClass(Module.class.getName()); + return (Connector) classLoader.loadClass(InternalHudiConnectorFactory.class.getName()) + .getMethod("createConnector", String.class, Map.class, ConnectorContext.class, moduleClass) + .invoke(null, catalogName, config, context, moduleInstance); + } + catch (InvocationTargetException e) { + Throwable targetException = e.getTargetException(); + throwIfUnchecked(targetException); + throw new RuntimeException(targetException); + } + catch (ReflectiveOperationException e) { + throw new RuntimeException(e); + } + } + + public static class EmptyModule + implements Module + { + @Override + public void configure(Binder binder) {} + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiErrorCode.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiErrorCode.java new file mode 100644 index 00000000000..d38724451c2 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiErrorCode.java @@ -0,0 +1,53 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import io.trino.spi.ErrorCode; +import io.trino.spi.ErrorCodeSupplier; +import io.trino.spi.ErrorType; + +import static io.trino.spi.ErrorType.EXTERNAL; +import static io.trino.spi.ErrorType.INTERNAL_ERROR; +import static io.trino.spi.ErrorType.USER_ERROR; + +public enum HudiErrorCode + implements ErrorCodeSupplier +{ + HUDI_UNKNOWN_TABLE_TYPE(0, EXTERNAL), + HUDI_INVALID_METADATA(1, EXTERNAL), + HUDI_TOO_MANY_OPEN_PARTITIONS(2, USER_ERROR), + HUDI_INVALID_PARTITION_VALUE(3, EXTERNAL), + HUDI_BAD_DATA(4, EXTERNAL), + HUDI_MISSING_DATA(5, EXTERNAL), + HUDI_CANNOT_OPEN_SPLIT(6, EXTERNAL), + HUDI_WRITER_OPEN_ERROR(7, EXTERNAL), + HUDI_FILESYSTEM_ERROR(8, EXTERNAL), + HUDI_CURSOR_ERROR(9, EXTERNAL), + HUDI_WRITE_VALIDATION_FAILED(10, INTERNAL_ERROR), + HUDI_INVALID_SNAPSHOT_ID(11, USER_ERROR); + + private final ErrorCode errorCode; + + HudiErrorCode(int code, ErrorType type) + { + errorCode = new ErrorCode(code + 0x0100_0000, name(), type); + } + + @Override + public ErrorCode toErrorCode() + { + return errorCode; + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiFileFormat.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiFileFormat.java new file mode 100644 index 00000000000..afea9f3e1bd --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiFileFormat.java @@ -0,0 +1,21 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +public enum HudiFileFormat +{ + PARQUET, + ORC, +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiHandleResolver.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiHandleResolver.java new file mode 100644 index 00000000000..fd72934f825 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiHandleResolver.java @@ -0,0 +1,67 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HiveInsertTableHandle; +import io.trino.plugin.hive.HiveOutputTableHandle; +import io.trino.plugin.hive.HiveTransactionHandle; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ConnectorHandleResolver; +import io.trino.spi.connector.ConnectorInsertTableHandle; +import io.trino.spi.connector.ConnectorOutputTableHandle; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTransactionHandle; + +public class HudiHandleResolver + implements ConnectorHandleResolver +{ + @Override + public Class getTableHandleClass() + { + return HudiTableHandle.class; + } + + @Override + public Class getColumnHandleClass() + { + return HiveColumnHandle.class; + } + + @Override + public Class getSplitClass() + { + return HudiSplit.class; + } + + @Override + public Class getOutputTableHandleClass() + { + return HiveOutputTableHandle.class; + } + + @Override + public Class getInsertTableHandleClass() + { + return HiveInsertTableHandle.class; + } + + @Override + public Class getTransactionHandleClass() + { + return HiveTransactionHandle.class; + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java new file mode 100644 index 00000000000..7be903be135 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java @@ -0,0 +1,66 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.spi.connector.ConnectorMetadata; +import io.trino.spi.connector.ConnectorSession; +import org.apache.hudi.common.table.HoodieTableMetaClient; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class HudiMetadata + implements ConnectorMetadata +{ + private final HiveMetastore metastore; + private final HdfsEnvironment hdfsEnvironment; + private final HoodieTableMetaClient metaClient; + + public HudiMetadata(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment) + { + this(metastore, hdfsEnvironment, getTableMetaClient()); + } + + private static HoodieTableMetaClient getTableMetaClient() + { + return null; + } + + public HudiMetadata(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, HoodieTableMetaClient metaClient) + { + this.metastore = requireNonNull(metastore, "metastore is null"); + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + this.metaClient = requireNonNull(metaClient, "metaClient is null"); + } + + @Override + public List listSchemaNames(ConnectorSession session) + { + return null; + } + + public HiveMetastore getMetastore() + { + return metastore; + } + + public void rollback() + { + // TODO: cleanup open transaction + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java new file mode 100644 index 00000000000..d3dbb83dbb0 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java @@ -0,0 +1,40 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.metastore.HiveMetastore; + +import javax.inject.Inject; + +import static java.util.Objects.requireNonNull; + +public class HudiMetadataFactory +{ + private final HiveMetastore metastore; + private final HdfsEnvironment hdfsEnvironment; + + @Inject + public HudiMetadataFactory(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment) + { + this.metastore = requireNonNull(metastore, "metastore is null"); + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + } + + public HudiMetadata create() + { + return new HudiMetadata(metastore, hdfsEnvironment); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetastoreModule.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetastoreModule.java new file mode 100644 index 00000000000..e9c2a37cb36 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetastoreModule.java @@ -0,0 +1,79 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.inject.Binder; +import com.google.inject.Module; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.MetastoreTypeConfig; +import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; +import io.trino.plugin.hive.metastore.cache.CachingHiveMetastoreModule; +import io.trino.plugin.hive.metastore.cache.ForCachingHiveMetastore; +import io.trino.plugin.hive.metastore.file.FileMetastoreModule; +import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreModule; + +import javax.inject.Inject; + +import java.util.Optional; + +import static io.airlift.configuration.ConditionalModule.conditionalModule; +import static java.util.Objects.requireNonNull; + +public class HudiMetastoreModule + extends AbstractConfigurationAwareModule +{ + private final Optional metastore; + + public HudiMetastoreModule(Optional metastore) + { + this.metastore = requireNonNull(metastore, "metastore is null"); + } + + @Override + protected void setup(Binder binder) + { + if (metastore.isPresent()) { + binder.bind(HiveMetastore.class).annotatedWith(ForCachingHiveMetastore.class).toInstance(metastore.get()); + install(new CachingHiveMetastoreModule()); + } + else { + bindMetastoreModule("thrift", new ThriftMetastoreModule()); + bindMetastoreModule("file", new FileMetastoreModule()); + // TODO add support for Glue metastore + } + + binder.bind(MetastoreValidator.class).asEagerSingleton(); + } + + public static class MetastoreValidator + { + @Inject + public MetastoreValidator(HiveMetastore metastore) + { + if (metastore instanceof CachingHiveMetastore) { + throw new RuntimeException("Hive metastore caching must not be enabled for Hudi"); + } + } + } + + private void bindMetastoreModule(String name, Module module) + { + install(conditionalModule( + MetastoreTypeConfig.class, + metastore -> name.equalsIgnoreCase(metastore.getMetastoreType()), + module)); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiModule.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiModule.java new file mode 100644 index 00000000000..7997a787470 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiModule.java @@ -0,0 +1,79 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; +import io.trino.plugin.base.security.AllowAllAccessControl; +import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.plugin.hive.CachingDirectoryLister; +import io.trino.plugin.hive.DirectoryLister; +import io.trino.plugin.hive.FileFormatDataSourceStats; +import io.trino.plugin.hive.HdfsConfiguration; +import io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.HiveConfig; +import io.trino.plugin.hive.HiveHdfsConfiguration; +import io.trino.plugin.hive.HiveNodePartitioningProvider; +import io.trino.plugin.hive.HiveTransactionManager; +import io.trino.plugin.hive.metastore.MetastoreConfig; +import io.trino.plugin.hive.parquet.ParquetReaderConfig; +import io.trino.plugin.hive.parquet.ParquetWriterConfig; +import io.trino.spi.connector.ConnectorAccessControl; +import io.trino.spi.connector.ConnectorNodePartitioningProvider; +import io.trino.spi.connector.ConnectorPageSinkProvider; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorSplitManager; + +import static com.google.inject.multibindings.Multibinder.newSetBinder; +import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; +import static io.airlift.configuration.ConfigBinder.configBinder; +import static org.weakref.jmx.guice.ExportBinder.newExporter; + +public class HudiModule + implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(HdfsConfiguration.class).to(HiveHdfsConfiguration.class).in(Scopes.SINGLETON); + binder.bind(HdfsEnvironment.class).in(Scopes.SINGLETON); + binder.bind(DirectoryLister.class).to(CachingDirectoryLister.class).in(Scopes.SINGLETON); + binder.bind(HudiTransactionManager.class).in(Scopes.SINGLETON); + + configBinder(binder).bindConfig(HiveConfig.class); + configBinder(binder).bindConfig(HudiConfig.class); + configBinder(binder).bindConfig(MetastoreConfig.class); + + newSetBinder(binder, SessionPropertiesProvider.class).addBinding().to(HudiSessionProperties.class).in(Scopes.SINGLETON); + binder.bind(HudiTableProperties.class).in(Scopes.SINGLETON); + + binder.bind(ConnectorSplitManager.class).to(HudiSplitManager.class).in(Scopes.SINGLETON); + binder.bind(ConnectorPageSourceProvider.class).to(HudiPageSourceProvider.class).in(Scopes.SINGLETON); + binder.bind(ConnectorPageSinkProvider.class).to(HudiPageSinkProvider.class).in(Scopes.SINGLETON); + binder.bind(ConnectorNodePartitioningProvider.class).to(HiveNodePartitioningProvider.class).in(Scopes.SINGLETON); + + configBinder(binder).bindConfig(ParquetReaderConfig.class); + configBinder(binder).bindConfig(ParquetWriterConfig.class); + + binder.bind(HudiMetadataFactory.class).in(Scopes.SINGLETON); + binder.bind(HiveTransactionManager.class).in(Scopes.SINGLETON); + + binder.bind(FileFormatDataSourceStats.class).in(Scopes.SINGLETON); + newExporter(binder).export(FileFormatDataSourceStats.class).withGeneratedName(); + + newOptionalBinder(binder, ConnectorAccessControl.class).setDefault().to(AllowAllAccessControl.class).in(Scopes.SINGLETON); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSinkProvider.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSinkProvider.java new file mode 100644 index 00000000000..1c54e4deed8 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSinkProvider.java @@ -0,0 +1,51 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableMap; +import io.trino.plugin.hive.HiveInsertTableHandle; +import io.trino.plugin.hive.HiveOutputTableHandle; +import io.trino.plugin.hive.HiveWritableTableHandle; +import io.trino.spi.connector.ConnectorInsertTableHandle; +import io.trino.spi.connector.ConnectorOutputTableHandle; +import io.trino.spi.connector.ConnectorPageSink; +import io.trino.spi.connector.ConnectorPageSinkProvider; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTransactionHandle; + +import java.util.Map; + +public class HudiPageSinkProvider + implements ConnectorPageSinkProvider +{ + @Override + public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorOutputTableHandle outputTableHandle) + { + HiveOutputTableHandle handle = (HiveOutputTableHandle) outputTableHandle; + return createPageSink(handle, true, session, handle.getAdditionalTableParameters()); + } + + @Override + public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorInsertTableHandle insertTableHandle) + { + HiveWritableTableHandle handle = (HiveInsertTableHandle) insertTableHandle; + return createPageSink(handle, false, session, ImmutableMap.of() /* for insert properties are taken from metastore */); + } + + private ConnectorPageSink createPageSink(HiveWritableTableHandle handle, boolean isCreateTable, ConnectorSession session, Map additionalTableParameters) + { + return null; + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java new file mode 100644 index 00000000000..60910624e77 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java @@ -0,0 +1,281 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableList; +import io.trino.parquet.Field; +import io.trino.parquet.ParquetCorruptionException; +import io.trino.parquet.ParquetDataSource; +import io.trino.parquet.ParquetDataSourceId; +import io.trino.parquet.ParquetReaderOptions; +import io.trino.parquet.RichColumnDescriptor; +import io.trino.parquet.predicate.Predicate; +import io.trino.parquet.reader.MetadataReader; +import io.trino.parquet.reader.ParquetReader; +import io.trino.plugin.hive.FileFormatDataSourceStats; +import io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.HdfsEnvironment.HdfsContext; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.ReaderColumns; +import io.trino.plugin.hive.parquet.HdfsParquetDataSource; +import io.trino.plugin.hive.parquet.ParquetPageSource; +import io.trino.plugin.hive.parquet.ParquetReaderConfig; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.connector.DynamicFilter; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.security.ConnectorIdentity; +import io.trino.spi.type.Type; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.BlockMissingException; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.FileMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.internal.filter2.columnindex.ColumnIndexStore; +import org.apache.parquet.io.MessageColumnIO; +import org.apache.parquet.schema.MessageType; +import org.joda.time.DateTimeZone; + +import javax.inject.Inject; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; +import static io.trino.parquet.ParquetTypeUtils.getColumnIO; +import static io.trino.parquet.ParquetTypeUtils.getDescriptors; +import static io.trino.parquet.ParquetTypeUtils.lookupColumnByName; +import static io.trino.parquet.predicate.PredicateUtils.buildPredicate; +import static io.trino.parquet.predicate.PredicateUtils.predicateMatches; +import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.REGULAR; +import static io.trino.plugin.hive.HivePageSourceProvider.projectBaseColumns; +import static io.trino.plugin.hive.HivePageSourceProvider.projectSufficientColumns; +import static io.trino.plugin.hive.parquet.HiveParquetColumnIOConverter.constructField; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.PARQUET_ROW_INDEX_COLUMN; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.getColumnIndexStore; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.getColumnType; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.getParquetTupleDomain; +import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.getParquetType; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_BAD_DATA; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_CANNOT_OPEN_SPLIT; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_MISSING_DATA; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toList; +import static java.util.stream.Collectors.toUnmodifiableList; + +public class HudiPageSourceProvider + implements ConnectorPageSourceProvider +{ + private final HdfsEnvironment hdfsEnvironment; + private final FileFormatDataSourceStats fileFormatDataSourceStats; + private final ParquetReaderOptions parquetReaderOptions; + private final DateTimeZone timeZone; + + @Inject + public HudiPageSourceProvider( + HdfsEnvironment hdfsEnvironment, + FileFormatDataSourceStats fileFormatDataSourceStats, + ParquetReaderConfig parquetReaderConfig, + HudiConfig hudiConfig) + { + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + this.fileFormatDataSourceStats = requireNonNull(fileFormatDataSourceStats, "fileFormatDataSourceStats is null"); + this.parquetReaderOptions = requireNonNull(parquetReaderConfig, "parquetReaderConfig is null").toParquetReaderOptions(); + requireNonNull(hudiConfig, "hudiConfig is null"); + this.timeZone = DateTimeZone.forID(hudiConfig.getParquetTimeZone()); + } + + @Override + public ConnectorPageSource createPageSource( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorSplit connectorSplit, + ConnectorTableHandle connectorTable, + List columns, + DynamicFilter dynamicFilter) + { + HudiSplit split = (HudiSplit) connectorSplit; + HudiTableHandle table = (HudiTableHandle) connectorTable; + Path path = new Path(split.getPath()); + long start = split.getStart(); + long length = split.getLength(); + long estimatedFileSize = split.getFileSize(); + List hiveColumns = columns.stream() + .map(HiveColumnHandle.class::cast) + .collect(toList()); + Configuration configuration = hdfsEnvironment.getConfiguration(new HdfsContext(session), path); + return createParquetPageSource( + hdfsEnvironment, + configuration, + path, + start, + length, + estimatedFileSize, + hiveColumns, + true, + split.getPredicate(), + fileFormatDataSourceStats, + timeZone, + parquetReaderOptions, + session.getIdentity()); + } + + private static ConnectorPageSource createParquetPageSource( + HdfsEnvironment hdfsEnvironment, + Configuration configuration, + Path path, + long start, + long length, + long estimatedFileSize, + List columns, + boolean useParquetColumnNames, + TupleDomain effectivePredicate, + FileFormatDataSourceStats stats, + DateTimeZone timeZone, + ParquetReaderOptions options, + ConnectorIdentity identity) + { + ParquetDataSource dataSource = null; + // TODO: Reuse some elements of ParquetPageSourceFactory and extract the try block to a new HudiParquetReader class. + try { + FileSystem fileSystem = hdfsEnvironment.getFileSystem(identity, path, configuration); + FSDataInputStream inputStream = hdfsEnvironment.doAs(identity, () -> fileSystem.open(path)); + dataSource = new HdfsParquetDataSource(new ParquetDataSourceId(path.toString()), estimatedFileSize, inputStream, stats, options); + ParquetDataSource theDataSource = dataSource; // extra variable required for lambda below + ParquetMetadata parquetMetadata = hdfsEnvironment.doAs(identity, () -> MetadataReader.readFooter(theDataSource)); + FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); + MessageType fileSchema = fileMetaData.getSchema(); + + Optional message = projectSufficientColumns(columns) + .map(projection -> projection.get().stream() + .map(HiveColumnHandle.class::cast) + .collect(toUnmodifiableList())) + .orElse(columns).stream() + .filter(column -> column.getColumnType() == REGULAR) + .map(column -> getColumnType(column, fileSchema, useParquetColumnNames)) + .filter(Optional::isPresent) + .map(Optional::get) + .map(type -> new MessageType(fileSchema.getName(), type)) + .reduce(MessageType::union); + + MessageType requestedSchema = message.orElse(new MessageType(fileSchema.getName(), ImmutableList.of())); + MessageColumnIO messageColumn = getColumnIO(fileSchema, requestedSchema); + + Map, RichColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, requestedSchema); + TupleDomain parquetTupleDomain = options.isIgnoreStatistics() + ? TupleDomain.all() + : getParquetTupleDomain(descriptorsByPath, effectivePredicate, fileSchema, useParquetColumnNames); + + Predicate parquetPredicate = buildPredicate(requestedSchema, parquetTupleDomain, descriptorsByPath, timeZone); + + long nextStart = 0; + ImmutableList.Builder blocks = ImmutableList.builder(); + ImmutableList.Builder blockStarts = ImmutableList.builder(); + ImmutableList.Builder> columnIndexes = ImmutableList.builder(); + for (BlockMetaData block : parquetMetadata.getBlocks()) { + long firstDataPage = block.getColumns().get(0).getFirstDataPageOffset(); + Optional columnIndex = getColumnIndexStore(dataSource, block, descriptorsByPath, parquetTupleDomain, options); + if (start <= firstDataPage && firstDataPage < start + length + && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain, columnIndex)) { + blocks.add(block); + blockStarts.add(nextStart); + columnIndexes.add(columnIndex); + } + nextStart += block.getRowCount(); + } + ParquetReader parquetReader = new ParquetReader( + Optional.ofNullable(fileMetaData.getCreatedBy()), + messageColumn, + blocks.build(), + Optional.of(blockStarts.build()), + dataSource, + timeZone, + newSimpleAggregatedMemoryContext(), + options, + parquetPredicate, + columnIndexes.build()); + Optional readerProjections = projectBaseColumns(columns); + List baseColumns = readerProjections.map(projection -> + projection.get().stream() + .map(HiveColumnHandle.class::cast) + .collect(toUnmodifiableList())) + .orElse(columns); + + for (HiveColumnHandle column : baseColumns) { + checkArgument(column == PARQUET_ROW_INDEX_COLUMN || column.getColumnType() == REGULAR, "column type must be REGULAR: %s", column); + } + + ImmutableList.Builder trinoTypes = ImmutableList.builder(); + ImmutableList.Builder> internalFields = ImmutableList.builder(); + ImmutableList.Builder rowIndexColumns = ImmutableList.builder(); + for (HiveColumnHandle column : baseColumns) { + trinoTypes.add(column.getBaseType()); + rowIndexColumns.add(column == PARQUET_ROW_INDEX_COLUMN); + if (column == PARQUET_ROW_INDEX_COLUMN) { + internalFields.add(Optional.empty()); + } + else { + internalFields.add(Optional.ofNullable(getParquetType(column, fileSchema, useParquetColumnNames)) + .flatMap(field -> { + String columnName = useParquetColumnNames ? column.getBaseColumnName() : fileSchema.getFields().get(column.getBaseHiveColumnIndex()).getName(); + return constructField(column.getBaseType(), lookupColumnByName(messageColumn, columnName)); + })); + } + } + + return new ParquetPageSource( + parquetReader, + trinoTypes.build(), + rowIndexColumns.build(), + internalFields.build()); + } + catch (IOException | RuntimeException e) { + try { + if (dataSource != null) { + dataSource.close(); + } + } + catch (IOException ignored) { + } + if (e instanceof TrinoException) { + throw (TrinoException) e; + } + String message = format("Error opening Hudi split %s (offset=%s, length=%s): %s", path, start, length, e.getMessage()); + + if (e instanceof ParquetCorruptionException) { + throw new TrinoException(HUDI_BAD_DATA, message, e); + } + + if (e instanceof BlockMissingException) { + throw new TrinoException(HUDI_MISSING_DATA, message, e); + } + throw new TrinoException(HUDI_CANNOT_OPEN_SPLIT, message, e); + } + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPlugin.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPlugin.java new file mode 100644 index 00000000000..565976f6389 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPlugin.java @@ -0,0 +1,29 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableList; +import io.trino.spi.Plugin; +import io.trino.spi.connector.ConnectorFactory; + +public class HudiPlugin + implements Plugin +{ + @Override + public Iterable getConnectorFactories() + { + return ImmutableList.of(new HudiConnectorFactory("hudi")); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java new file mode 100644 index 00000000000..f53ba9bf8ff --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java @@ -0,0 +1,30 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.spi.session.PropertyMetadata; + +import java.util.List; + +public class HudiSessionProperties + implements SessionPropertiesProvider +{ + @Override + public List> getSessionProperties() + { + return null; + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java new file mode 100644 index 00000000000..d59beecc3e6 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java @@ -0,0 +1,123 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HivePartitionKey; +import io.trino.spi.HostAddress; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.predicate.TupleDomain; + +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class HudiSplit + implements ConnectorSplit +{ + private final String path; + private final long start; + private final long length; + private final long fileSize; + private final List addresses; + private final TupleDomain predicate; + private final List partitionKeys; + + @JsonCreator + public HudiSplit( + @JsonProperty("path") String path, + @JsonProperty("start") long start, + @JsonProperty("length") long length, + @JsonProperty("fileSize") long fileSize, + @JsonProperty("addresses") List addresses, + @JsonProperty("predicate") TupleDomain predicate, + @JsonProperty("partitionKeys") List partitionKeys) + { + checkArgument(start >= 0, "start must be positive"); + checkArgument(length >= 0, "length must be positive"); + checkArgument(fileSize >= 0, "fileSize must be positive"); + + this.path = requireNonNull(path, "path is null"); + this.start = start; + this.length = length; + this.fileSize = fileSize; + this.addresses = ImmutableList.copyOf(requireNonNull(addresses, "addresses is null")); + this.predicate = requireNonNull(predicate, "predicate is null"); + this.partitionKeys = ImmutableList.copyOf(requireNonNull(partitionKeys, "partitionKeys is null")); + } + + @Override + public boolean isRemotelyAccessible() + { + return false; + } + + @Override + public List getAddresses() + { + return addresses; + } + + @Override + public Object getInfo() + { + return ImmutableMap.builder() + .put("path", path) + .put("start", start) + .put("length", length) + .build(); + } + + @JsonProperty + public String getPath() + { + return path; + } + + @JsonProperty + public long getStart() + { + return start; + } + + @JsonProperty + public long getLength() + { + return length; + } + + @JsonProperty + public long getFileSize() + { + return fileSize; + } + + @JsonProperty + public TupleDomain getPredicate() + { + return predicate; + } + + @JsonProperty + public List getPartitionKeys() + { + return partitionKeys; + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java new file mode 100644 index 00000000000..cfd6aabe10d --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -0,0 +1,66 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitSource; +import io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.authentication.HiveIdentity; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.Table; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplitManager; +import io.trino.spi.connector.ConnectorSplitSource; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.connector.Constraint; +import io.trino.spi.connector.DynamicFilter; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.TableNotFoundException; + +import javax.inject.Inject; + +import static java.util.Objects.requireNonNull; + +public class HudiSplitManager + implements ConnectorSplitManager +{ + private final HudiTransactionManager transactionManager; + private final HdfsEnvironment hdfsEnvironment; + + @Inject + public HudiSplitManager(HudiTransactionManager transactionManager, HdfsEnvironment hdfsEnvironment) + { + this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + } + + @Override + public ConnectorSplitSource getSplits( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorTableHandle tableHandle, + SplitSchedulingStrategy splitSchedulingStrategy, + DynamicFilter dynamicFilter, + Constraint constraint) + { + HudiTableHandle hudiTable = (HudiTableHandle) tableHandle; + SchemaTableName tableName = hudiTable.getSchemaTableName(); + HiveMetastore metastore = transactionManager.get(transaction).getMetastore(); + Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) + .orElseThrow(() -> new TableNotFoundException(tableName)); + HudiSplitSource splitSource = new HudiSplitSource(hudiTable); + return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java new file mode 100644 index 00000000000..0f574593389 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -0,0 +1,74 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableList; +import io.trino.plugin.hive.HivePartitionKey; +import io.trino.spi.connector.ConnectorPartitionHandle; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorSplitSource; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.CompletableFuture.completedFuture; + +public class HudiSplitSource + implements ConnectorSplitSource +{ + private final HudiTableHandle tableHandle; + + public HudiSplitSource(HudiTableHandle tableHandle) + { + this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); + } + + @Override + public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) + { + List splits = new ArrayList<>(); + HudiSplit hudiSplit = new HudiSplit( + "", + 0L, + 100L, + 100L, + ImmutableList.of(), + tableHandle.getPredicate(), + getPartitionKeys(tableHandle)); + splits.add(hudiSplit); + return completedFuture(new ConnectorSplitBatch(splits, isFinished())); + } + + @Override + public void close() + { + // TODO: close file iterable + } + + @Override + public boolean isFinished() + { + // TODO: finish based on file iterator + return false; + } + + private static List getPartitionKeys(HudiTableHandle hudiTableHandle) + { + List partitionKeys = new ArrayList<>(); + return partitionKeys; + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java new file mode 100644 index 00000000000..1d89395e199 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java @@ -0,0 +1,146 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.predicate.TupleDomain; +import org.apache.hudi.common.model.HoodieTableType; + +import java.util.Locale; +import java.util.Optional; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; +import static java.lang.Long.parseLong; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class HudiTableHandle + implements ConnectorTableHandle +{ + private static final Pattern TABLE_PATTERN = Pattern.compile("" + + "(?[^$@]+)" + + "(?:@(?[0-9]+))?" + + "(?:\\$(?[^@]+)(?:@(?[0-9]+))?)?"); + + private final String schemaName; + private final String tableName; + private final HoodieTableType tableType; + private final Optional snapshotId; + private final TupleDomain predicate; + + @JsonCreator + public HudiTableHandle( + @JsonProperty("schemaName") String schemaName, + @JsonProperty("tableName") String tableName, + @JsonProperty("tableType") HoodieTableType tableType, + @JsonProperty("snapshotId") Optional snapshotId, + @JsonProperty("predicate") TupleDomain predicate) + { + this.schemaName = requireNonNull(schemaName, "schemaName is null"); + this.tableName = requireNonNull(tableName, "tableName is null"); + this.tableType = requireNonNull(tableType, "tableType is null"); + this.snapshotId = requireNonNull(snapshotId, "snapshotId is null"); + this.predicate = requireNonNull(predicate, "predicate is null"); + } + + @JsonProperty + public String getSchemaName() + { + return schemaName; + } + + @JsonProperty + public String getTableName() + { + return tableName; + } + + @JsonProperty + public HoodieTableType getTableType() + { + return tableType; + } + + @JsonProperty + public Optional getSnapshotId() + { + return snapshotId; + } + + @JsonProperty + public TupleDomain getPredicate() + { + return predicate; + } + + public SchemaTableName getSchemaTableName() + { + return new SchemaTableName(schemaName, tableName); + } + + @Override + public String toString() + { + return getSchemaTableName().toString(); + } + + public static HudiTableHandle from(SchemaTableName name) + { + Matcher match = TABLE_PATTERN.matcher(name.getTableName()); + if (!match.matches()) { + throw new TrinoException(NOT_SUPPORTED, "Invalid Hudi table name: " + name); + } + + String table = match.group("table"); + String typeString = match.group("type"); + String ver1 = match.group("ver1"); + String ver2 = match.group("ver2"); + + HoodieTableType type = HoodieTableType.COPY_ON_WRITE; + if (typeString != null) { + try { + type = HoodieTableType.valueOf(typeString.toUpperCase(Locale.ROOT)); + } + catch (IllegalArgumentException e) { + throw new TrinoException(NOT_SUPPORTED, format("Invalid Hudi table name (unknown type '%s'): %s", typeString, name)); + } + } + + Optional version = Optional.empty(); + if (type == HoodieTableType.COPY_ON_WRITE || type == HoodieTableType.MERGE_ON_READ) { + if (ver1 != null && ver2 != null) { + throw new TrinoException(NOT_SUPPORTED, "Invalid Hudi table name (cannot specify two @ versions): " + name); + } + if (ver1 != null) { + version = Optional.of(parseLong(ver1)); + } + else if (ver2 != null) { + version = Optional.of(parseLong(ver2)); + } + } + else if (ver1 != null || ver2 != null) { + throw new TrinoException(NOT_SUPPORTED, format("Invalid Hudi table name (cannot use @ version with table type '%s'): %s", type, name)); + } + + return new HudiTableHandle(name.getSchemaName(), table, type, version, TupleDomain.all()); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableProperties.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableProperties.java new file mode 100644 index 00000000000..7e9960f53bb --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableProperties.java @@ -0,0 +1,69 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableList; +import io.trino.spi.session.PropertyMetadata; +import org.apache.hudi.common.model.HoodieFileFormat; + +import javax.inject.Inject; + +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static io.trino.spi.session.PropertyMetadata.enumProperty; +import static io.trino.spi.session.PropertyMetadata.stringProperty; + +public class HudiTableProperties +{ + public static final String FILE_FORMAT_PROPERTY = "format"; + public static final String LOCATION_PROPERTY = "location"; + + private final List> tableProperties; + + @Inject + public HudiTableProperties(HudiConfig hudiConfig) + { + tableProperties = ImmutableList.>builder() + .add(enumProperty( + FILE_FORMAT_PROPERTY, + "File format for the table", + HoodieFileFormat.class, + hudiConfig.getFileFormat(), + false)) + .add(stringProperty( + LOCATION_PROPERTY, + "File system location URI for the table", + null, + false)) + .build(); + } + + public List> getTableProperties() + { + return tableProperties; + } + + public static HoodieFileFormat getFileFormat(Map tableProperties) + { + return (HoodieFileFormat) tableProperties.get(FILE_FORMAT_PROPERTY); + } + + public static Optional getTableLocation(Map tableProperties) + { + return Optional.ofNullable((String) tableProperties.get(LOCATION_PROPERTY)); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTransactionManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTransactionManager.java new file mode 100644 index 00000000000..7fad31361b2 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTransactionManager.java @@ -0,0 +1,49 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import io.trino.spi.connector.ConnectorMetadata; +import io.trino.spi.connector.ConnectorTransactionHandle; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; + +public class HudiTransactionManager +{ + private final Map transactions = new ConcurrentHashMap<>(); + + public HudiMetadata get(ConnectorTransactionHandle transaction) + { + HudiMetadata metadata = transactions.get(transaction); + checkArgument(metadata != null, "no such transaction: %s", transaction); + return metadata; + } + + public HudiMetadata remove(ConnectorTransactionHandle transaction) + { + HudiMetadata metadata = transactions.remove(transaction); + checkArgument(metadata != null, "no such transaction: %s", transaction); + return metadata; + } + + public void put(ConnectorTransactionHandle transaction, HudiMetadata metadata) + { + ConnectorMetadata existing = transactions.putIfAbsent(transaction, metadata); + checkState(existing == null, "transaction already exists: %s", existing); + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/InternalHudiConnectorFactory.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/InternalHudiConnectorFactory.java new file mode 100644 index 00000000000..867f73d1532 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/InternalHudiConnectorFactory.java @@ -0,0 +1,124 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableSet; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.TypeLiteral; +import io.airlift.bootstrap.Bootstrap; +import io.airlift.bootstrap.LifeCycleManager; +import io.airlift.event.client.EventModule; +import io.airlift.json.JsonModule; +import io.trino.plugin.base.CatalogName; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorPageSinkProvider; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProvider; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitManager; +import io.trino.plugin.base.classloader.ClassLoaderSafeNodePartitioningProvider; +import io.trino.plugin.base.jmx.MBeanServerModule; +import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.plugin.hive.HiveHdfsModule; +import io.trino.plugin.hive.NodeVersion; +import io.trino.plugin.hive.authentication.HdfsAuthenticationModule; +import io.trino.plugin.hive.azure.HiveAzureModule; +import io.trino.plugin.hive.gcs.HiveGcsModule; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreModule; +import io.trino.plugin.hive.s3.HiveS3Module; +import io.trino.spi.NodeManager; +import io.trino.spi.PageIndexerFactory; +import io.trino.spi.classloader.ThreadContextClassLoader; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorAccessControl; +import io.trino.spi.connector.ConnectorContext; +import io.trino.spi.connector.ConnectorNodePartitioningProvider; +import io.trino.spi.connector.ConnectorPageSinkProvider; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorSplitManager; +import io.trino.spi.procedure.Procedure; +import io.trino.spi.type.TypeManager; +import org.weakref.jmx.guice.MBeanModule; + +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static java.util.Objects.requireNonNull; + +public final class InternalHudiConnectorFactory +{ + private InternalHudiConnectorFactory() {} + + public static Connector createConnector(String catalogName, Map config, ConnectorContext context, Module module, Optional metastore) + { + requireNonNull(config, "config is null"); + ClassLoader classLoader = InternalHudiConnectorFactory.class.getClassLoader(); + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + Bootstrap app = new Bootstrap( + new EventModule(), + new MBeanModule(), + new JsonModule(), + new HudiModule(), + new HudiMetastoreModule(metastore), + new HiveMetastoreModule(Optional.empty()), + new HiveHdfsModule(), + new HiveS3Module(), + new HiveGcsModule(), + new HiveAzureModule(), + new HdfsAuthenticationModule(), + new MBeanServerModule(), + binder -> { + binder.bind(NodeVersion.class).toInstance(new NodeVersion(context.getNodeManager().getCurrentNode().getVersion())); + binder.bind(NodeManager.class).toInstance(context.getNodeManager()); + binder.bind(TypeManager.class).toInstance(context.getTypeManager()); + binder.bind(PageIndexerFactory.class).toInstance(context.getPageIndexerFactory()); + binder.bind(CatalogName.class).toInstance(new CatalogName(catalogName)); + }, + module); + + Injector injector = app + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + + LifeCycleManager lifeCycleManager = injector.getInstance(LifeCycleManager.class); + HudiTransactionManager transactionManager = injector.getInstance(HudiTransactionManager.class); + HudiMetadataFactory metadataFactory = injector.getInstance(HudiMetadataFactory.class); + ConnectorSplitManager splitManager = injector.getInstance(ConnectorSplitManager.class); + ConnectorPageSourceProvider connectorPageSource = injector.getInstance(ConnectorPageSourceProvider.class); + ConnectorPageSinkProvider pageSinkProvider = injector.getInstance(ConnectorPageSinkProvider.class); + ConnectorNodePartitioningProvider connectorDistributionProvider = injector.getInstance(ConnectorNodePartitioningProvider.class); + Set sessionPropertiesProviders = injector.getInstance(Key.get(new TypeLiteral>() {})); + HudiTableProperties hudiTableProperties = injector.getInstance(HudiTableProperties.class); + Set procedures = injector.getInstance(Key.get(new TypeLiteral>() {})); + Optional accessControl = injector.getInstance(Key.get(new TypeLiteral>() {})); + + return new HudiConnector( + lifeCycleManager, + transactionManager, + metadataFactory, + new ClassLoaderSafeConnectorSplitManager(splitManager, classLoader), + new ClassLoaderSafeConnectorPageSourceProvider(connectorPageSource, classLoader), + new ClassLoaderSafeConnectorPageSinkProvider(pageSinkProvider, classLoader), + new ClassLoaderSafeNodePartitioningProvider(connectorDistributionProvider, classLoader), + ImmutableSet.of(), + sessionPropertiesProviders, + hudiTableProperties.getTableProperties(), + accessControl, + procedures); + } + } +} diff --git a/pom.xml b/pom.xml index 887dcedd6d9..16917a281a6 100644 --- a/pom.xml +++ b/pom.xml @@ -120,6 +120,7 @@ plugin/trino-hive plugin/trino-hive-hadoop2 plugin/trino-http-event-listener + plugin/trino-hudi plugin/trino-iceberg plugin/trino-jmx plugin/trino-kafka @@ -271,6 +272,12 @@ ${project.version} + + io.trino + trino-hudi + ${project.version} + + io.trino trino-iceberg From 39f855b255608fc97a09e94f81870c19c762f508 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Thu, 9 Dec 2021 19:18:51 +0530 Subject: [PATCH 02/26] Add concrete implementation of connector metadata and split --- .../io/trino/plugin/hudi/HudiInputInfo.java | 49 ++++++++ .../io/trino/plugin/hudi/HudiMetadata.java | 119 ++++++++++++++++-- .../plugin/hudi/HudiMetadataFactory.java | 7 +- .../plugin/hudi/HudiPageSourceProvider.java | 7 +- .../java/io/trino/plugin/hudi/HudiSplit.java | 15 ++- .../trino/plugin/hudi/HudiSplitManager.java | 5 +- .../io/trino/plugin/hudi/HudiSplitSource.java | 52 ++++++-- .../io/trino/plugin/hudi/HudiTableHandle.java | 96 +++++--------- 8 files changed, 258 insertions(+), 92 deletions(-) create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiInputInfo.java diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiInputInfo.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiInputInfo.java new file mode 100644 index 00000000000..36843d75540 --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiInputInfo.java @@ -0,0 +1,49 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +public class HudiInputInfo +{ + private final List partitionIds; + // Code that serialize HudiInputInfo into log would often need the ability to limit the length of log entries. + // This boolean field allows such code to mark the log entry as length limited. + private final boolean truncated; + + @JsonCreator + public HudiInputInfo( + @JsonProperty("partitionIds") List partitionIds, + @JsonProperty("truncated") boolean truncated) + { + this.partitionIds = partitionIds; + this.truncated = truncated; + } + + @JsonProperty + public List getPartitionIds() + { + return partitionIds; + } + + @JsonProperty + public boolean isTruncated() + { + return truncated; + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java index 7be903be135..cc039262594 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java @@ -15,43 +15,111 @@ package io.trino.plugin.hudi; import io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HivePartition; +import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.Table; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorMetadata; import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTableProperties; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.type.TypeManager; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import java.util.List; +import java.util.Map; +import java.util.Optional; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static io.trino.plugin.hive.HiveSessionProperties.getTimestampPrecision; +import static io.trino.plugin.hive.util.HiveUtil.hiveColumnHandles; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_UNKNOWN_TABLE_TYPE; +import static java.lang.String.format; import static java.util.Objects.requireNonNull; +import static java.util.function.Function.identity; +import static org.apache.hudi.common.fs.FSUtils.getFs; +import static org.apache.hudi.common.table.HoodieTableMetaClient.METAFOLDER_NAME; +import static org.apache.hudi.exception.TableNotFoundException.checkTableValidity; public class HudiMetadata implements ConnectorMetadata { private final HiveMetastore metastore; private final HdfsEnvironment hdfsEnvironment; - private final HoodieTableMetaClient metaClient; + private final TypeManager typeManager; - public HudiMetadata(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment) + public HudiMetadata(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, TypeManager typeManager) { - this(metastore, hdfsEnvironment, getTableMetaClient()); + this.metastore = requireNonNull(metastore, "metastore is null"); + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); } - private static HoodieTableMetaClient getTableMetaClient() + @Override + public List listSchemaNames(ConnectorSession session) { - return null; + return metastore.getAllDatabases(); } - public HudiMetadata(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, HoodieTableMetaClient metaClient) + @Override + public HudiTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) { - this.metastore = requireNonNull(metastore, "metastore is null"); - this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); - this.metaClient = requireNonNull(metaClient, "metaClient is null"); + //HudiTableHandle handle = HudiTableHandle.from(tableName); + Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) + .orElseThrow(() -> new TableNotFoundException(tableName)); + if (!isHudiTable(session, table)) { + throw new TrinoException(HUDI_UNKNOWN_TABLE_TYPE, format("Not a Hudi table: %s", tableName)); + } + return new HudiTableHandle( + tableName.getSchemaName(), + tableName.getTableName(), + HoodieTableType.COPY_ON_WRITE, + TupleDomain.all(), + Optional.empty(), + getTableMetaClient(session, table)); } @Override - public List listSchemaNames(ConnectorSession session) + public ConnectorTableProperties getTableProperties(ConnectorSession session, ConnectorTableHandle tableHandle) { - return null; + return new ConnectorTableProperties(); + } + + @Override + public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) + { + SchemaTableName tableName = ((HudiTableHandle) tableHandle).getSchemaTableName(); + Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) + .orElseThrow(() -> new TableNotFoundException(tableName)); + return hiveColumnHandles(table, typeManager, getTimestampPrecision(session)).stream() + .collect(toImmutableMap(HiveColumnHandle::getName, identity())); + } + + @Override + public Optional getInfo(ConnectorTableHandle table) + { + return ((HudiTableHandle) table).getPartitions() + .map(partitions -> new HudiInputInfo( + partitions.stream() + .map(HivePartition::getPartitionId) + .collect(toImmutableList()), + false)); + } + + @Override + public boolean usesLegacyTableLayouts() + { + return false; } public HiveMetastore getMetastore() @@ -59,8 +127,37 @@ public HiveMetastore getMetastore() return metastore; } + public HoodieTableMetaClient getTableMetaClient(ConnectorSession session, SchemaTableName tableName) + { + Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) + .orElseThrow(() -> new TableNotFoundException(tableName)); + String basePath = table.getStorage().getLocation(); + Configuration conf = hdfsEnvironment.getConfiguration(new HdfsEnvironment.HdfsContext(session), new Path(basePath)); + return HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build(); + } + + public HoodieTableMetaClient getTableMetaClient(ConnectorSession session, Table table) + { + String basePath = table.getStorage().getLocation(); + Configuration conf = hdfsEnvironment.getConfiguration(new HdfsEnvironment.HdfsContext(session), new Path(basePath)); + return HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build(); + } + public void rollback() { // TODO: cleanup open transaction } + + private boolean isHudiTable(ConnectorSession session, Table table) + { + String basePath = table.getStorage().getLocation(); + Configuration conf = hdfsEnvironment.getConfiguration(new HdfsEnvironment.HdfsContext(session), new Path(basePath)); + try { + checkTableValidity(getFs(basePath, conf), new Path(basePath), new Path(basePath, METAFOLDER_NAME)); + } + catch (Exception e) { + return false; + } + return true; + } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java index d3dbb83dbb0..c698ca7f26f 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java @@ -16,6 +16,7 @@ import io.trino.plugin.hive.HdfsEnvironment; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.spi.type.TypeManager; import javax.inject.Inject; @@ -25,16 +26,18 @@ public class HudiMetadataFactory { private final HiveMetastore metastore; private final HdfsEnvironment hdfsEnvironment; + private final TypeManager typeManager; @Inject - public HudiMetadataFactory(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment) + public HudiMetadataFactory(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, TypeManager typeManager) { this.metastore = requireNonNull(metastore, "metastore is null"); this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); } public HudiMetadata create() { - return new HudiMetadata(metastore, hdfsEnvironment); + return new HudiMetadata(metastore, hdfsEnvironment, typeManager); } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java index 60910624e77..762aec86453 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java @@ -28,6 +28,7 @@ import io.trino.plugin.hive.HdfsEnvironment; import io.trino.plugin.hive.HdfsEnvironment.HdfsContext; import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HivePartitionKey; import io.trino.plugin.hive.ReaderColumns; import io.trino.plugin.hive.parquet.HdfsParquetDataSource; import io.trino.plugin.hive.parquet.ParquetPageSource; @@ -143,7 +144,8 @@ public ConnectorPageSource createPageSource( fileFormatDataSourceStats, timeZone, parquetReaderOptions, - session.getIdentity()); + session.getIdentity(), + split.getPartitionKeys()); } private static ConnectorPageSource createParquetPageSource( @@ -159,7 +161,8 @@ private static ConnectorPageSource createParquetPageSource( FileFormatDataSourceStats stats, DateTimeZone timeZone, ParquetReaderOptions options, - ConnectorIdentity identity) + ConnectorIdentity identity, + List partitionKeys) { ParquetDataSource dataSource = null; // TODO: Reuse some elements of ParquetPageSourceFactory and extract the try block to a new HudiParquetReader class. diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java index d59beecc3e6..2a63ee7b75b 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java @@ -26,6 +26,7 @@ import java.util.List; +import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; @@ -66,7 +67,7 @@ public HudiSplit( @Override public boolean isRemotelyAccessible() { - return false; + return true; } @Override @@ -82,6 +83,7 @@ public Object getInfo() .put("path", path) .put("start", start) .put("length", length) + .put("fileSize", fileSize) .build(); } @@ -120,4 +122,15 @@ public List getPartitionKeys() { return partitionKeys; } + + @Override + public String toString() + { + return toStringHelper(this) + .addValue(path) + .addValue(start) + .addValue(length) + .addValue(fileSize) + .toString(); + } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index cfd6aabe10d..e132f7003a2 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -28,6 +28,8 @@ import io.trino.spi.connector.DynamicFilter; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.connector.TableNotFoundException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import javax.inject.Inject; @@ -60,7 +62,8 @@ public ConnectorSplitSource getSplits( HiveMetastore metastore = transactionManager.get(transaction).getMetastore(); Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(tableName)); - HudiSplitSource splitSource = new HudiSplitSource(hudiTable); + Configuration conf = hdfsEnvironment.getConfiguration(new HdfsEnvironment.HdfsContext(session), new Path(table.getStorage().getLocation())); + HudiSplitSource splitSource = new HudiSplitSource(hudiTable, conf); return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index 0f574593389..bc7b8b6f05c 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -19,11 +19,22 @@ import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitSource; +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.TimelineUtils; +import org.apache.hudi.common.table.view.FileSystemViewManager; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.concurrent.CompletableFuture; +import static com.google.common.collect.Iterators.limit; import static java.util.Objects.requireNonNull; import static java.util.concurrent.CompletableFuture.completedFuture; @@ -31,25 +42,39 @@ public class HudiSplitSource implements ConnectorSplitSource { private final HudiTableHandle tableHandle; + private final HoodieTableMetaClient metaClient; + private final HoodieTableFileSystemView fileSystemView; + private final Iterator hoodieBaseFileIterator; - public HudiSplitSource(HudiTableHandle tableHandle) + public HudiSplitSource(HudiTableHandle tableHandle, Configuration conf) { this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); + this.metaClient = tableHandle.getMetaClient(); + HoodieEngineContext engineContext = new HoodieLocalEngineContext(conf); + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() + .enable(false) + .build(); + this.fileSystemView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, metaClient, metadataConfig); + this.hoodieBaseFileIterator = fileSystemView.getLatestBaseFiles().iterator(); } @Override public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) { List splits = new ArrayList<>(); - HudiSplit hudiSplit = new HudiSplit( - "", - 0L, - 100L, - 100L, - ImmutableList.of(), - tableHandle.getPredicate(), - getPartitionKeys(tableHandle)); - splits.add(hudiSplit); + Iterator baseFileIterator = limit(hoodieBaseFileIterator, maxSize); + while (baseFileIterator.hasNext()) { + HoodieBaseFile baseFile = baseFileIterator.next(); + splits.add(new HudiSplit( + baseFile.getPath(), + 0L, + baseFile.getFileLen(), + baseFile.getFileSize(), + ImmutableList.of(), + tableHandle.getPredicate(), + getPartitionKeys())); + } + return completedFuture(new ConnectorSplitBatch(splits, isFinished())); } @@ -62,13 +87,14 @@ public void close() @Override public boolean isFinished() { - // TODO: finish based on file iterator - return false; + return !hoodieBaseFileIterator.hasNext(); } - private static List getPartitionKeys(HudiTableHandle hudiTableHandle) + private List getPartitionKeys() { List partitionKeys = new ArrayList<>(); + List partitions = TimelineUtils.getPartitionsWritten(metaClient.getActiveTimeline()); + return partitionKeys; } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java index 1d89395e199..a7953d20714 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java @@ -15,51 +15,50 @@ package io.trino.plugin.hudi; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import io.trino.plugin.hive.HiveColumnHandle; -import io.trino.spi.TrinoException; +import io.trino.plugin.hive.HivePartition; import io.trino.spi.connector.ConnectorTableHandle; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.predicate.TupleDomain; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.TimelineUtils; -import java.util.Locale; +import java.util.List; import java.util.Optional; -import java.util.regex.Matcher; -import java.util.regex.Pattern; +import java.util.stream.Collectors; -import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; -import static java.lang.Long.parseLong; -import static java.lang.String.format; import static java.util.Objects.requireNonNull; public class HudiTableHandle implements ConnectorTableHandle { - private static final Pattern TABLE_PATTERN = Pattern.compile("" + - "(?
[^$@]+)" + - "(?:@(?[0-9]+))?" + - "(?:\\$(?[^@]+)(?:@(?[0-9]+))?)?"); - private final String schemaName; private final String tableName; private final HoodieTableType tableType; - private final Optional snapshotId; private final TupleDomain predicate; + private final Optional> partitions; + private final HoodieTableMetaClient metaClient; @JsonCreator public HudiTableHandle( @JsonProperty("schemaName") String schemaName, @JsonProperty("tableName") String tableName, @JsonProperty("tableType") HoodieTableType tableType, - @JsonProperty("snapshotId") Optional snapshotId, - @JsonProperty("predicate") TupleDomain predicate) + @JsonProperty("predicate") TupleDomain predicate, + Optional> partitions, + HoodieTableMetaClient metaClient) { this.schemaName = requireNonNull(schemaName, "schemaName is null"); this.tableName = requireNonNull(tableName, "tableName is null"); this.tableType = requireNonNull(tableType, "tableType is null"); - this.snapshotId = requireNonNull(snapshotId, "snapshotId is null"); this.predicate = requireNonNull(predicate, "predicate is null"); + this.partitions = requireNonNull(partitions, "partitions is null").map(ImmutableList::copyOf); + this.metaClient = requireNonNull(metaClient, "metaClient is null"); } @JsonProperty @@ -81,15 +80,29 @@ public HoodieTableType getTableType() } @JsonProperty - public Optional getSnapshotId() + public TupleDomain getPredicate() { - return snapshotId; + return predicate; } - @JsonProperty - public TupleDomain getPredicate() + @JsonIgnore + public Optional> getPartitions() { - return predicate; + if (partitions.isEmpty()) { + List partitionIds = TimelineUtils.getPartitionsWritten(metaClient.getActiveTimeline()); + List hivePartitions = partitionIds.stream() + .map(p -> new HivePartition(getSchemaTableName(), p, ImmutableMap.of())) + .collect(Collectors.toList()); + return Optional.of(hivePartitions); + } + + return partitions; + } + + @JsonIgnore + public HoodieTableMetaClient getMetaClient() + { + return metaClient; } public SchemaTableName getSchemaTableName() @@ -102,45 +115,4 @@ public String toString() { return getSchemaTableName().toString(); } - - public static HudiTableHandle from(SchemaTableName name) - { - Matcher match = TABLE_PATTERN.matcher(name.getTableName()); - if (!match.matches()) { - throw new TrinoException(NOT_SUPPORTED, "Invalid Hudi table name: " + name); - } - - String table = match.group("table"); - String typeString = match.group("type"); - String ver1 = match.group("ver1"); - String ver2 = match.group("ver2"); - - HoodieTableType type = HoodieTableType.COPY_ON_WRITE; - if (typeString != null) { - try { - type = HoodieTableType.valueOf(typeString.toUpperCase(Locale.ROOT)); - } - catch (IllegalArgumentException e) { - throw new TrinoException(NOT_SUPPORTED, format("Invalid Hudi table name (unknown type '%s'): %s", typeString, name)); - } - } - - Optional version = Optional.empty(); - if (type == HoodieTableType.COPY_ON_WRITE || type == HoodieTableType.MERGE_ON_READ) { - if (ver1 != null && ver2 != null) { - throw new TrinoException(NOT_SUPPORTED, "Invalid Hudi table name (cannot specify two @ versions): " + name); - } - if (ver1 != null) { - version = Optional.of(parseLong(ver1)); - } - else if (ver2 != null) { - version = Optional.of(parseLong(ver2)); - } - } - else if (ver1 != null || ver2 != null) { - throw new TrinoException(NOT_SUPPORTED, format("Invalid Hudi table name (cannot use @ version with table type '%s'): %s", type, name)); - } - - return new HudiTableHandle(name.getSchemaName(), table, type, version, TupleDomain.all()); - } } From 48711578cf7303bf81356583d759e850e4ae776c Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Fri, 10 Dec 2021 09:18:11 +0530 Subject: [PATCH 03/26] Add Hudi plugin to trino-server-dev --- core/trino-server/src/main/provisio/presto.xml | 6 ++++++ testing/trino-server-dev/etc/config.properties | 1 + 2 files changed, 7 insertions(+) diff --git a/core/trino-server/src/main/provisio/presto.xml b/core/trino-server/src/main/provisio/presto.xml index 48190e7b21a..02575a105ae 100644 --- a/core/trino-server/src/main/provisio/presto.xml +++ b/core/trino-server/src/main/provisio/presto.xml @@ -74,6 +74,12 @@ + + + + + + diff --git a/testing/trino-server-dev/etc/config.properties b/testing/trino-server-dev/etc/config.properties index e1e4530f742..fd2beb31dba 100644 --- a/testing/trino-server-dev/etc/config.properties +++ b/testing/trino-server-dev/etc/config.properties @@ -36,6 +36,7 @@ plugin.bundles=\ ../../plugin/trino-jmx/pom.xml,\ ../../plugin/trino-raptor-legacy/pom.xml,\ ../../plugin/trino-hive-hadoop2/pom.xml,\ + ../../plugin/trino-hudi/pom.xml,\ ../../plugin/trino-example-http/pom.xml,\ ../../plugin/trino-kafka/pom.xml, \ ../../plugin/trino-tpch/pom.xml, \ From c609b2ed2f3efe65a1837e0f7dcf6c928c39bd85 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Mon, 13 Dec 2021 21:05:42 +0530 Subject: [PATCH 04/26] Add table and column metadata --- .../java/io/trino/plugin/hudi/HudiConfig.java | 26 ++-- .../io/trino/plugin/hudi/HudiConnector.java | 19 ++- .../plugin/hudi/HudiConnectorFactory.java | 7 - .../io/trino/plugin/hudi/HudiMetadata.java | 132 ++++++++++++++++-- .../plugin/hudi/HudiPageSourceProvider.java | 3 +- .../plugin/hudi/HudiSessionProperties.java | 42 +++++- .../io/trino/plugin/hudi/HudiSplitSource.java | 3 +- .../io/trino/plugin/hudi/HudiTableHandle.java | 40 +++++- .../{HudiFileFormat.java => HudiUtil.java} | 13 +- .../hudi/InternalHudiConnectorFactory.java | 16 ++- .../etc/catalog/hudi.properties | 17 +++ testing/trino-server-dev/etc/log.properties | 3 + 12 files changed, 256 insertions(+), 65 deletions(-) rename plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/{HudiFileFormat.java => HudiUtil.java} (63%) create mode 100644 testing/trino-server-dev/etc/catalog/hudi.properties diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java index 42c4a1f642b..68420d45924 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java @@ -20,14 +20,12 @@ import javax.validation.constraints.NotNull; -import java.util.TimeZone; - -import static io.trino.plugin.hudi.HudiFileFormat.PARQUET; +import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET; public class HudiConfig { - private HudiFileFormat fileFormat = PARQUET; - private String parquetTimeZone = TimeZone.getDefault().getID(); + private HoodieFileFormat fileFormat = PARQUET; + private boolean metadataEnabled; @NotNull public HoodieFileFormat getFileFormat() @@ -36,23 +34,23 @@ public HoodieFileFormat getFileFormat() } @Config("hudi.file-format") - public HudiConfig setFileFormat(HudiFileFormat fileFormat) + public HudiConfig setFileFormat(HoodieFileFormat fileFormat) { this.fileFormat = fileFormat; return this; } - @NotNull - public String getParquetTimeZone() + @Config("hudi.metadata-enabled") + @ConfigDescription("Fetch the list of file names and sizes from metadata rather than storage") + public HudiConfig setMetadataEnabled(boolean metadataEnabled) { - return parquetTimeZone; + this.metadataEnabled = metadataEnabled; + return this; } - @Config("hudi.parquet.time-zone") - @ConfigDescription("Time zone for Parquet read and write") - public HudiConfig setParquetTimeZone(String parquetTimeZone) + @NotNull + public boolean isMetadataEnabled() { - this.parquetTimeZone = parquetTimeZone; - return this; + return this.metadataEnabled; } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnector.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnector.java index a899c72bbdd..401bdf44619 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnector.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnector.java @@ -24,6 +24,7 @@ import io.trino.spi.classloader.ThreadContextClassLoader; import io.trino.spi.connector.Connector; import io.trino.spi.connector.ConnectorAccessControl; +import io.trino.spi.connector.ConnectorHandleResolver; import io.trino.spi.connector.ConnectorMetadata; import io.trino.spi.connector.ConnectorNodePartitioningProvider; import io.trino.spi.connector.ConnectorPageSinkProvider; @@ -31,7 +32,6 @@ import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.ConnectorTransactionHandle; import io.trino.spi.connector.SystemTable; -import io.trino.spi.procedure.Procedure; import io.trino.spi.session.PropertyMetadata; import io.trino.spi.transaction.IsolationLevel; @@ -60,7 +60,6 @@ public class HudiConnector private final List> sessionProperties; private final List> tableProperties; private final Optional accessControl; - private final Set procedures; public HudiConnector( LifeCycleManager lifeCycleManager, @@ -73,8 +72,7 @@ public HudiConnector( Set systemTables, Set sessionPropertiesProviders, List> tableProperties, - Optional accessControl, - Set procedures) + Optional accessControl) { this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); @@ -89,7 +87,12 @@ public HudiConnector( .collect(toImmutableList()); this.tableProperties = ImmutableList.copyOf(requireNonNull(tableProperties, "tableProperties is null")); this.accessControl = requireNonNull(accessControl, "accessControl is null"); - this.procedures = requireNonNull(procedures, "procedures is null"); + } + + @Override + public Optional getHandleResolver() + { + return Optional.of(new HudiHandleResolver()); } @Override @@ -129,12 +132,6 @@ public Set getSystemTables() return systemTables; } - @Override - public Set getProcedures() - { - return procedures; - } - @Override public List> getSessionProperties() { diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java index 1e016bd49de..34d077f756a 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java @@ -19,7 +19,6 @@ import io.trino.spi.connector.Connector; import io.trino.spi.connector.ConnectorContext; import io.trino.spi.connector.ConnectorFactory; -import io.trino.spi.connector.ConnectorHandleResolver; import java.lang.reflect.InvocationTargetException; import java.util.Map; @@ -53,12 +52,6 @@ public String getName() return name; } - @Override - public ConnectorHandleResolver getHandleResolver() - { - return new HudiHandleResolver(); - } - @Override public Connector create(String catalogName, Map config, ConnectorContext context) { diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java index cc039262594..52194feb815 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java @@ -14,17 +14,24 @@ package io.trino.plugin.hudi; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Sets; import io.trino.plugin.hive.HdfsEnvironment; import io.trino.plugin.hive.HiveColumnHandle; import io.trino.plugin.hive.HivePartition; +import io.trino.plugin.hive.acid.AcidSchema; import io.trino.plugin.hive.authentication.HiveIdentity; +import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.Table; import io.trino.spi.TrinoException; import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.connector.ConnectorMetadata; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTableMetadata; import io.trino.spi.connector.ConnectorTableProperties; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.connector.TableNotFoundException; @@ -38,15 +45,29 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.function.Function; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; -import static io.trino.plugin.hive.HiveSessionProperties.getTimestampPrecision; +import static com.google.common.collect.Iterables.concat; +import static io.trino.plugin.hive.HiveColumnHandle.BUCKET_COLUMN_NAME; +import static io.trino.plugin.hive.HiveColumnHandle.FILE_MODIFIED_TIME_COLUMN_NAME; +import static io.trino.plugin.hive.HiveColumnHandle.FILE_SIZE_COLUMN_NAME; +import static io.trino.plugin.hive.HiveColumnHandle.PARTITION_COLUMN_NAME; +import static io.trino.plugin.hive.HiveColumnHandle.PATH_COLUMN_NAME; +import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_METADATA; +import static io.trino.plugin.hive.HiveMetadata.TABLE_COMMENT; +import static io.trino.plugin.hive.HiveTableProperties.EXTERNAL_LOCATION_PROPERTY; +import static io.trino.plugin.hive.HiveTableProperties.PARTITIONED_BY_PROPERTY; +import static io.trino.plugin.hive.HiveTimestampPrecision.NANOSECONDS; +import static io.trino.plugin.hive.util.HiveUtil.columnExtraInfo; import static io.trino.plugin.hive.util.HiveUtil.hiveColumnHandles; import static io.trino.plugin.hudi.HudiErrorCode.HUDI_UNKNOWN_TABLE_TYPE; import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static java.util.function.Function.identity; +import static org.apache.hadoop.hive.metastore.TableType.EXTERNAL_TABLE; +import static org.apache.hadoop.hive.ql.io.AcidUtils.isFullAcidTable; import static org.apache.hudi.common.fs.FSUtils.getFs; import static org.apache.hudi.common.table.HoodieTableMetaClient.METAFOLDER_NAME; import static org.apache.hudi.exception.TableNotFoundException.checkTableValidity; @@ -74,7 +95,6 @@ public List listSchemaNames(ConnectorSession session) @Override public HudiTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) { - //HudiTableHandle handle = HudiTableHandle.from(tableName); Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(tableName)); if (!isHudiTable(session, table)) { @@ -83,10 +103,45 @@ public HudiTableHandle getTableHandle(ConnectorSession session, SchemaTableName return new HudiTableHandle( tableName.getSchemaName(), tableName.getTableName(), + table.getStorage().getLocation(), HoodieTableType.COPY_ON_WRITE, TupleDomain.all(), - Optional.empty(), - getTableMetaClient(session, table)); + Optional.of(getTableMetaClient(session, table))); + } + + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) + { + HudiTableHandle hudiTableHandle = (HudiTableHandle) table; + return getTableMetadata(session, hudiTableHandle.getSchemaTableName()); + } + + private ConnectorTableMetadata getTableMetadata(ConnectorSession session, SchemaTableName tableName) + { + Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) + .orElseThrow(() -> new TableNotFoundException(tableName)); + Function metadataGetter = columnMetadataGetter(table); + ImmutableList.Builder columns = ImmutableList.builder(); + for (HiveColumnHandle columnHandle : hiveColumnHandles(table, typeManager, NANOSECONDS)) { + columns.add(metadataGetter.apply(columnHandle)); + } + + // External location property + ImmutableMap.Builder properties = ImmutableMap.builder(); + if (table.getTableType().equals(EXTERNAL_TABLE.name())) { + properties.put(EXTERNAL_LOCATION_PROPERTY, table.getStorage().getLocation()); + } + + // Partitioning property + List partitionedBy = table.getPartitionColumns().stream() + .map(Column::getName) + .collect(toImmutableList()); + if (!partitionedBy.isEmpty()) { + properties.put(PARTITIONED_BY_PROPERTY, partitionedBy); + } + + Optional comment = Optional.ofNullable(table.getParameters().get(TABLE_COMMENT)); + return new ConnectorTableMetadata(tableName, columns.build(), properties.build(), comment); } @Override @@ -101,10 +156,16 @@ public Map getColumnHandles(ConnectorSession session, Conn SchemaTableName tableName = ((HudiTableHandle) tableHandle).getSchemaTableName(); Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(tableName)); - return hiveColumnHandles(table, typeManager, getTimestampPrecision(session)).stream() + return hiveColumnHandles(table, typeManager, NANOSECONDS).stream() .collect(toImmutableMap(HiveColumnHandle::getName, identity())); } + @Override + public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle) + { + return ((HiveColumnHandle) columnHandle).getColumnMetadata(); + } + @Override public Optional getInfo(ConnectorTableHandle table) { @@ -127,20 +188,11 @@ public HiveMetastore getMetastore() return metastore; } - public HoodieTableMetaClient getTableMetaClient(ConnectorSession session, SchemaTableName tableName) - { - Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new TableNotFoundException(tableName)); - String basePath = table.getStorage().getLocation(); - Configuration conf = hdfsEnvironment.getConfiguration(new HdfsEnvironment.HdfsContext(session), new Path(basePath)); - return HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build(); - } - public HoodieTableMetaClient getTableMetaClient(ConnectorSession session, Table table) { String basePath = table.getStorage().getLocation(); Configuration conf = hdfsEnvironment.getConfiguration(new HdfsEnvironment.HdfsContext(session), new Path(basePath)); - return HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build(); + return HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath).build(); } public void rollback() @@ -160,4 +212,54 @@ private boolean isHudiTable(ConnectorSession session, Table table) } return true; } + + private static Function columnMetadataGetter(Table table) + { + ImmutableList.Builder columnNames = ImmutableList.builder(); + table.getPartitionColumns().stream().map(Column::getName).forEach(columnNames::add); + table.getDataColumns().stream().map(Column::getName).forEach(columnNames::add); + List allColumnNames = columnNames.build(); + if (allColumnNames.size() > Sets.newHashSet(allColumnNames).size()) { + throw new TrinoException(HIVE_INVALID_METADATA, + format("Hive metadata for table %s is invalid: Table descriptor contains duplicate columns", table.getTableName())); + } + + List tableColumns = table.getDataColumns(); + ImmutableMap.Builder> builder = ImmutableMap.builder(); + for (Column field : concat(tableColumns, table.getPartitionColumns())) { + if (field.getComment().isPresent() && !field.getComment().get().equals("from deserializer")) { + builder.put(field.getName(), field.getComment()); + } + else { + builder.put(field.getName(), Optional.empty()); + } + } + + // add hidden columns + builder.put(PATH_COLUMN_NAME, Optional.empty()); + if (table.getStorage().getBucketProperty().isPresent()) { + builder.put(BUCKET_COLUMN_NAME, Optional.empty()); + } + builder.put(FILE_SIZE_COLUMN_NAME, Optional.empty()); + builder.put(FILE_MODIFIED_TIME_COLUMN_NAME, Optional.empty()); + if (!table.getPartitionColumns().isEmpty()) { + builder.put(PARTITION_COLUMN_NAME, Optional.empty()); + } + + if (isFullAcidTable(table.getParameters())) { + for (String name : AcidSchema.ACID_COLUMN_NAMES) { + builder.put(name, Optional.empty()); + } + } + + Map> columnComment = builder.build(); + + return handle -> ColumnMetadata.builder() + .setName(handle.getName()) + .setType(handle.getType()) + .setComment(columnComment.get(handle.getName())) + .setExtraInfo(Optional.ofNullable(columnExtraInfo(handle.isPartitionKey()))) + .setHidden(handle.isHidden()) + .build(); + } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java index 762aec86453..647e4c6cf7d 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java @@ -65,6 +65,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.TimeZone; import static com.google.common.base.Preconditions.checkArgument; import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; @@ -109,7 +110,7 @@ public HudiPageSourceProvider( this.fileFormatDataSourceStats = requireNonNull(fileFormatDataSourceStats, "fileFormatDataSourceStats is null"); this.parquetReaderOptions = requireNonNull(parquetReaderConfig, "parquetReaderConfig is null").toParquetReaderOptions(); requireNonNull(hudiConfig, "hudiConfig is null"); - this.timeZone = DateTimeZone.forID(hudiConfig.getParquetTimeZone()); + this.timeZone = DateTimeZone.forID(TimeZone.getDefault().getID()); } @Override diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java index f53ba9bf8ff..9932c203764 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java @@ -14,17 +14,57 @@ package io.trino.plugin.hudi; +import com.google.common.collect.ImmutableList; import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.spi.connector.ConnectorSession; import io.trino.spi.session.PropertyMetadata; +import org.apache.hudi.common.model.HoodieFileFormat; + +import javax.inject.Inject; import java.util.List; +import static io.trino.spi.session.PropertyMetadata.booleanProperty; +import static io.trino.spi.session.PropertyMetadata.enumProperty; + public class HudiSessionProperties implements SessionPropertiesProvider { + private static final String FILE_FORMAT = "file_format"; + private static final String METADATA_ENABLED = "metadata_enabled"; + + private final List> sessionProperties; + + @Inject + public HudiSessionProperties(HudiConfig hudiConfig) + { + sessionProperties = ImmutableList.of( + enumProperty( + FILE_FORMAT, + "Currently, only Parquet is supported", + HoodieFileFormat.class, + hudiConfig.getFileFormat(), + false), + booleanProperty( + METADATA_ENABLED, + "For Hudi tables prefer to fetch the list of files from its metadata", + hudiConfig.isMetadataEnabled(), + false)); + } + @Override public List> getSessionProperties() { - return null; + return sessionProperties; + } + + public static HoodieFileFormat getFileFormat(ConnectorSession session) + { + return session.getProperty(FILE_FORMAT, HoodieFileFormat.class); + } + + public static boolean isHudiMetadataEnabled(ConnectorSession session) + { + return session.getProperty(METADATA_ENABLED, Boolean.class); } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index bc7b8b6f05c..3f9845e4a79 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -35,6 +35,7 @@ import java.util.concurrent.CompletableFuture; import static com.google.common.collect.Iterators.limit; +import static io.trino.plugin.hudi.HudiUtil.getMetaClient; import static java.util.Objects.requireNonNull; import static java.util.concurrent.CompletableFuture.completedFuture; @@ -49,7 +50,7 @@ public class HudiSplitSource public HudiSplitSource(HudiTableHandle tableHandle, Configuration conf) { this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); - this.metaClient = tableHandle.getMetaClient(); + this.metaClient = tableHandle.getMetaClient().orElseGet(() -> getMetaClient(conf, tableHandle.getBasePath())); HoodieEngineContext engineContext = new HoodieLocalEngineContext(conf); HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() .enable(false) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java index a7953d20714..0ce83b2e585 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java @@ -39,22 +39,46 @@ public class HudiTableHandle { private final String schemaName; private final String tableName; + private final String basePath; private final HoodieTableType tableType; private final TupleDomain predicate; private final Optional> partitions; - private final HoodieTableMetaClient metaClient; + private final Optional metaClient; @JsonCreator public HudiTableHandle( @JsonProperty("schemaName") String schemaName, @JsonProperty("tableName") String tableName, + @JsonProperty("basePath") String basePath, @JsonProperty("tableType") HoodieTableType tableType, - @JsonProperty("predicate") TupleDomain predicate, + @JsonProperty("predicate") TupleDomain predicate) + { + this(schemaName, tableName, basePath, tableType, predicate, Optional.empty(), Optional.empty()); + } + + public HudiTableHandle( + String schemaName, + String tableName, + String basePath, + HoodieTableType tableType, + TupleDomain predicate, + Optional metaClient) + { + this(schemaName, tableName, basePath, tableType, predicate, Optional.empty(), metaClient); + } + + public HudiTableHandle( + String schemaName, + String tableName, + String basePath, + HoodieTableType tableType, + TupleDomain predicate, Optional> partitions, - HoodieTableMetaClient metaClient) + Optional metaClient) { this.schemaName = requireNonNull(schemaName, "schemaName is null"); this.tableName = requireNonNull(tableName, "tableName is null"); + this.basePath = requireNonNull(basePath, "basePath is null"); this.tableType = requireNonNull(tableType, "tableType is null"); this.predicate = requireNonNull(predicate, "predicate is null"); this.partitions = requireNonNull(partitions, "partitions is null").map(ImmutableList::copyOf); @@ -73,6 +97,12 @@ public String getTableName() return tableName; } + @JsonProperty + public String getBasePath() + { + return basePath; + } + @JsonProperty public HoodieTableType getTableType() { @@ -89,7 +119,7 @@ public TupleDomain getPredicate() public Optional> getPartitions() { if (partitions.isEmpty()) { - List partitionIds = TimelineUtils.getPartitionsWritten(metaClient.getActiveTimeline()); + List partitionIds = TimelineUtils.getPartitionsWritten(metaClient.get().getActiveTimeline()); List hivePartitions = partitionIds.stream() .map(p -> new HivePartition(getSchemaTableName(), p, ImmutableMap.of())) .collect(Collectors.toList()); @@ -100,7 +130,7 @@ public Optional> getPartitions() } @JsonIgnore - public HoodieTableMetaClient getMetaClient() + public Optional getMetaClient() { return metaClient; } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiFileFormat.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java similarity index 63% rename from plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiFileFormat.java rename to plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java index afea9f3e1bd..5a1ab973980 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiFileFormat.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java @@ -14,8 +14,15 @@ package io.trino.plugin.hudi; -public enum HudiFileFormat +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.common.table.HoodieTableMetaClient; + +public class HudiUtil { - PARQUET, - ORC, + private HudiUtil() {} + + public static HoodieTableMetaClient getMetaClient(Configuration conf, String basePath) + { + return HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath).build(); + } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/InternalHudiConnectorFactory.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/InternalHudiConnectorFactory.java index 867f73d1532..7105509cfef 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/InternalHudiConnectorFactory.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/InternalHudiConnectorFactory.java @@ -29,6 +29,7 @@ import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitManager; import io.trino.plugin.base.classloader.ClassLoaderSafeNodePartitioningProvider; import io.trino.plugin.base.jmx.MBeanServerModule; +import io.trino.plugin.base.security.AllowAllAccessControl; import io.trino.plugin.base.session.SessionPropertiesProvider; import io.trino.plugin.hive.HiveHdfsModule; import io.trino.plugin.hive.NodeVersion; @@ -48,7 +49,6 @@ import io.trino.spi.connector.ConnectorPageSinkProvider; import io.trino.spi.connector.ConnectorPageSourceProvider; import io.trino.spi.connector.ConnectorSplitManager; -import io.trino.spi.procedure.Procedure; import io.trino.spi.type.TypeManager; import org.weakref.jmx.guice.MBeanModule; @@ -62,6 +62,11 @@ public final class InternalHudiConnectorFactory { private InternalHudiConnectorFactory() {} + public static Connector createConnector(String catalogName, Map config, ConnectorContext context, Module module) + { + return createConnector(catalogName, config, context, module, Optional.empty()); + } + public static Connector createConnector(String catalogName, Map config, ConnectorContext context, Module module, Optional metastore) { requireNonNull(config, "config is null"); @@ -72,8 +77,7 @@ public static Connector createConnector(String catalogName, Map new MBeanModule(), new JsonModule(), new HudiModule(), - new HudiMetastoreModule(metastore), - new HiveMetastoreModule(Optional.empty()), + new HiveMetastoreModule(metastore), new HiveHdfsModule(), new HiveS3Module(), new HiveGcsModule(), @@ -103,8 +107,7 @@ public static Connector createConnector(String catalogName, Map ConnectorNodePartitioningProvider connectorDistributionProvider = injector.getInstance(ConnectorNodePartitioningProvider.class); Set sessionPropertiesProviders = injector.getInstance(Key.get(new TypeLiteral>() {})); HudiTableProperties hudiTableProperties = injector.getInstance(HudiTableProperties.class); - Set procedures = injector.getInstance(Key.get(new TypeLiteral>() {})); - Optional accessControl = injector.getInstance(Key.get(new TypeLiteral>() {})); + Optional accessControl = Optional.of(new AllowAllAccessControl()); return new HudiConnector( lifeCycleManager, @@ -117,8 +120,7 @@ public static Connector createConnector(String catalogName, Map ImmutableSet.of(), sessionPropertiesProviders, hudiTableProperties.getTableProperties(), - accessControl, - procedures); + accessControl); } } } diff --git a/testing/trino-server-dev/etc/catalog/hudi.properties b/testing/trino-server-dev/etc/catalog/hudi.properties new file mode 100644 index 00000000000..33ba981c9bc --- /dev/null +++ b/testing/trino-server-dev/etc/catalog/hudi.properties @@ -0,0 +1,17 @@ +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +connector.name=hudi +hive.metastore.uri=thrift://localhost:9083 +#hive.config.resources=/Users/sagars/core-site.xml,/Users/sagars/hdfs-site.xml diff --git a/testing/trino-server-dev/etc/log.properties b/testing/trino-server-dev/etc/log.properties index b615d661c74..18b148d8f5f 100644 --- a/testing/trino-server-dev/etc/log.properties +++ b/testing/trino-server-dev/etc/log.properties @@ -12,3 +12,6 @@ io.trino.server.PluginManager=DEBUG # Maven plugin loading code com.ning.http.client=WARN +io.trino.plugin.hudi=DEBUG +io.trino.plugin.hive=DEBUG +org.apache.hudi=DEBUG From 26a901e7a0b833d0980e2fa30916bc89937ed78c Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Tue, 14 Dec 2021 13:58:28 +0530 Subject: [PATCH 05/26] Fix show tables --- .../io/trino/plugin/hudi/HudiMetadata.java | 42 ++++++++++++++++--- 1 file changed, 37 insertions(+), 5 deletions(-) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java index 52194feb815..705fec13987 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java @@ -62,6 +62,7 @@ import static io.trino.plugin.hive.HiveTimestampPrecision.NANOSECONDS; import static io.trino.plugin.hive.util.HiveUtil.columnExtraInfo; import static io.trino.plugin.hive.util.HiveUtil.hiveColumnHandles; +import static io.trino.plugin.hive.util.HiveUtil.isHiveSystemSchema; import static io.trino.plugin.hudi.HudiErrorCode.HUDI_UNKNOWN_TABLE_TYPE; import static java.lang.String.format; import static java.util.Objects.requireNonNull; @@ -95,18 +96,24 @@ public List listSchemaNames(ConnectorSession session) @Override public HudiTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) { - Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new TableNotFoundException(tableName)); - if (!isHudiTable(session, table)) { + requireNonNull(tableName, "tableName is null"); + if (isHiveSystemSchema(tableName.getSchemaName())) { + return null; + } + Optional
table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()); + if (table.isEmpty()) { + return null; + } + if (!isHudiTable(session, table.get())) { throw new TrinoException(HUDI_UNKNOWN_TABLE_TYPE, format("Not a Hudi table: %s", tableName)); } return new HudiTableHandle( tableName.getSchemaName(), tableName.getTableName(), - table.getStorage().getLocation(), + table.get().getStorage().getLocation(), HoodieTableType.COPY_ON_WRITE, TupleDomain.all(), - Optional.of(getTableMetaClient(session, table))); + Optional.of(getTableMetaClient(session, table.get()))); } @Override @@ -177,6 +184,31 @@ public Optional getInfo(ConnectorTableHandle table) false)); } + @Override + public List listTables(ConnectorSession session, Optional optionalSchemaName) + { + ImmutableList.Builder tableNames = ImmutableList.builder(); + for (String schemaName : listSchemas(session, optionalSchemaName)) { + for (String tableName : metastore.getAllTables(schemaName)) { + tableNames.add(new SchemaTableName(schemaName, tableName)); + } + } + + tableNames.addAll(listMaterializedViews(session, optionalSchemaName)); + return tableNames.build(); + } + + private List listSchemas(ConnectorSession session, Optional schemaName) + { + if (schemaName.isPresent()) { + if (isHiveSystemSchema(schemaName.get())) { + return ImmutableList.of(); + } + return ImmutableList.of(schemaName.get()); + } + return listSchemaNames(session); + } + @Override public boolean usesLegacyTableLayouts() { From 4964b6e24fac0f0e1dd6477ffe0e2893c6a121fd Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Thu, 16 Dec 2021 20:49:04 +0530 Subject: [PATCH 06/26] Handle partition and add support for metadata listing --- .../hive/BackgroundHiveSplitLoader.java | 27 +---- .../io/trino/plugin/hive/util/HiveUtil.java | 23 +++++ plugin/trino-hudi/pom.xml | 41 +++++++- .../plugin/hudi/HudiPageSourceProvider.java | 11 ++- .../java/io/trino/plugin/hudi/HudiSplit.java | 1 + .../trino/plugin/hudi/HudiSplitManager.java | 99 ++++++++++++++++++- .../io/trino/plugin/hudi/HudiSplitSource.java | 59 ++++++++--- .../java/io/trino/plugin/hudi/HudiUtil.java | 22 +++++ 8 files changed, 237 insertions(+), 46 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java index 5546808b2bd..37f33bc6cfa 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java @@ -28,7 +28,6 @@ import io.trino.plugin.hive.HiveSplit.BucketConversion; import io.trino.plugin.hive.HiveSplit.BucketValidation; import io.trino.plugin.hive.acid.AcidTransaction; -import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.Partition; import io.trino.plugin.hive.metastore.Table; import io.trino.plugin.hive.util.HiveBucketing.BucketingVersion; @@ -100,8 +99,6 @@ import static io.trino.plugin.hive.HiveErrorCode.HIVE_BAD_DATA; import static io.trino.plugin.hive.HiveErrorCode.HIVE_FILESYSTEM_ERROR; import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_BUCKET_FILES; -import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_METADATA; -import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_PARTITION_VALUE; import static io.trino.plugin.hive.HiveErrorCode.HIVE_UNKNOWN_ERROR; import static io.trino.plugin.hive.HivePartitionManager.partitionMatches; import static io.trino.plugin.hive.HiveSessionProperties.getMaxInitialSplitSize; @@ -114,11 +111,11 @@ import static io.trino.plugin.hive.util.HiveFileIterator.NestedDirectoryPolicy.FAIL; import static io.trino.plugin.hive.util.HiveFileIterator.NestedDirectoryPolicy.IGNORED; import static io.trino.plugin.hive.util.HiveFileIterator.NestedDirectoryPolicy.RECURSE; -import static io.trino.plugin.hive.util.HiveUtil.checkCondition; import static io.trino.plugin.hive.util.HiveUtil.getFooterCount; import static io.trino.plugin.hive.util.HiveUtil.getHeaderCount; import static io.trino.plugin.hive.util.HiveUtil.getInputFormat; import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeyColumnHandles; +import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeys; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static java.lang.Integer.parseInt; import static java.lang.Math.max; @@ -942,28 +939,6 @@ private static List getTargetPathsFromSymlink(FileSystem fileSystem, Path } } - private static List getPartitionKeys(Table table, Optional partition) - { - if (partition.isEmpty()) { - return ImmutableList.of(); - } - ImmutableList.Builder partitionKeys = ImmutableList.builder(); - List keys = table.getPartitionColumns(); - List values = partition.get().getValues(); - checkCondition(keys.size() == values.size(), HIVE_INVALID_METADATA, "Expected %s partition key values, but got %s", keys.size(), values.size()); - for (int i = 0; i < keys.size(); i++) { - String name = keys.get(i).getName(); - HiveType hiveType = keys.get(i).getType(); - if (!hiveType.isSupportedType(table.getStorage().getStorageFormat())) { - throw new TrinoException(NOT_SUPPORTED, format("Unsupported Hive type %s found in partition keys of table %s.%s", hiveType, table.getDatabaseName(), table.getTableName())); - } - String value = values.get(i); - checkCondition(value != null, HIVE_INVALID_PARTITION_VALUE, "partition key value cannot be null for field: %s", name); - partitionKeys.add(new HivePartitionKey(name, value)); - } - return partitionKeys.build(); - } - private static Properties getPartitionSchema(Table table, Optional partition) { if (partition.isEmpty()) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java index c6d398402f0..100809bdef9 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java @@ -32,6 +32,7 @@ import io.trino.plugin.hive.HiveType; import io.trino.plugin.hive.avro.TrinoAvroSerDe; import io.trino.plugin.hive.metastore.Column; +import io.trino.plugin.hive.metastore.Partition; import io.trino.plugin.hive.metastore.SortingColumn; import io.trino.plugin.hive.metastore.Table; import io.trino.spi.ErrorCodeSupplier; @@ -1143,4 +1144,26 @@ public static boolean isIcebergTable(Table table) { return ICEBERG_TABLE_TYPE_VALUE.equalsIgnoreCase(table.getParameters().get(ICEBERG_TABLE_TYPE_NAME)); } + + public static List getPartitionKeys(Table table, Optional partition) + { + if (partition.isEmpty()) { + return ImmutableList.of(); + } + ImmutableList.Builder partitionKeys = ImmutableList.builder(); + List keys = table.getPartitionColumns(); + List values = partition.get().getValues(); + checkCondition(keys.size() == values.size(), HIVE_INVALID_METADATA, "Expected %s partition key values, but got %s", keys.size(), values.size()); + for (int i = 0; i < keys.size(); i++) { + String name = keys.get(i).getName(); + HiveType hiveType = keys.get(i).getType(); + if (!hiveType.isSupportedType(table.getStorage().getStorageFormat())) { + throw new TrinoException(NOT_SUPPORTED, format("Unsupported Hive type %s found in partition keys of table %s.%s", hiveType, table.getDatabaseName(), table.getTableName())); + } + String value = values.get(i); + checkCondition(value != null, HIVE_INVALID_PARTITION_VALUE, "partition key value cannot be null for field: %s", name); + partitionKeys.add(new HivePartitionKey(name, value)); + } + return partitionKeys.build(); + } } diff --git a/plugin/trino-hudi/pom.xml b/plugin/trino-hudi/pom.xml index 5edb7136f22..5142e356668 100644 --- a/plugin/trino-hudi/pom.xml +++ b/plugin/trino-hudi/pom.xml @@ -17,7 +17,7 @@ ${project.parent.basedir} - 0.9.0 + 0.10.0 @@ -124,6 +124,45 @@ + + org.apache.hudi + hudi-hadoop-mr + ${dep.hudi.version} + + + org.apache.hbase + hbase-server + + + org.apache.orc + orc-core + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + fluent-hc + + + org.rocksdb + rocksdbjni + + + com.esotericsoftware + kryo-shaded + + + org.weakref jmxutils diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java index 647e4c6cf7d..db0368cb2e2 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java @@ -15,6 +15,7 @@ package io.trino.plugin.hudi; import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; import io.trino.parquet.Field; import io.trino.parquet.ParquetCorruptionException; import io.trino.parquet.ParquetDataSource; @@ -67,7 +68,6 @@ import java.util.Optional; import java.util.TimeZone; -import static com.google.common.base.Preconditions.checkArgument; import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; import static io.trino.parquet.ParquetTypeUtils.getColumnIO; import static io.trino.parquet.ParquetTypeUtils.getDescriptors; @@ -94,6 +94,8 @@ public class HudiPageSourceProvider implements ConnectorPageSourceProvider { + private static final Logger log = Logger.get(HudiPageSourceProvider.class); + private final HdfsEnvironment hdfsEnvironment; private final FileFormatDataSourceStats fileFormatDataSourceStats; private final ParquetReaderOptions parquetReaderOptions; @@ -165,6 +167,8 @@ private static ConnectorPageSource createParquetPageSource( ConnectorIdentity identity, List partitionKeys) { + log.debug(">>> Creating Parquet Page Source with partition keys: " + partitionKeys); + ParquetDataSource dataSource = null; // TODO: Reuse some elements of ParquetPageSourceFactory and extract the try block to a new HudiParquetReader class. try { @@ -231,9 +235,10 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq .collect(toUnmodifiableList())) .orElse(columns); - for (HiveColumnHandle column : baseColumns) { + // TODO: add a check for patition column type + /*for (HiveColumnHandle column : baseColumns) { checkArgument(column == PARQUET_ROW_INDEX_COLUMN || column.getColumnType() == REGULAR, "column type must be REGULAR: %s", column); - } + }*/ ImmutableList.Builder trinoTypes = ImmutableList.builder(); ImmutableList.Builder> internalFields = ImmutableList.builder(); diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java index 2a63ee7b75b..3a978f2fcd3 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplit.java @@ -70,6 +70,7 @@ public boolean isRemotelyAccessible() return true; } + @JsonProperty @Override public List getAddresses() { diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index e132f7003a2..5102357ce58 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -14,11 +14,17 @@ package io.trino.plugin.hudi; +import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitSource; import io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.HivePartitionKey; import io.trino.plugin.hive.authentication.HiveIdentity; +import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.Partition; import io.trino.plugin.hive.metastore.Table; +import io.trino.plugin.hive.util.HiveUtil; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.ConnectorSplitSource; @@ -28,16 +34,38 @@ import io.trino.spi.connector.DynamicFilter; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.predicate.TupleDomain; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.hadoop.HoodieParquetInputFormat; import javax.inject.Inject; +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.Properties; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.hive.util.ConfigurationUtils.toJobConf; +import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeys; +import static io.trino.plugin.hudi.HudiSessionProperties.isHudiMetadataEnabled; +import static io.trino.plugin.hudi.HudiUtil.getMetaClient; +import static io.trino.plugin.hudi.HudiUtil.getPartitionSchema; +import static io.trino.plugin.hudi.HudiUtil.isHudiParquetInputFormat; import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_LOCATION; +import static org.apache.hudi.common.table.timeline.TimelineUtils.getPartitionsWritten; public class HudiSplitManager implements ConnectorSplitManager { + private static final Logger log = Logger.get(HudiSplitManager.class); + private final HudiTransactionManager transactionManager; private final HdfsEnvironment hdfsEnvironment; @@ -57,13 +85,78 @@ public ConnectorSplitSource getSplits( DynamicFilter dynamicFilter, Constraint constraint) { + log.debug(" >>>> Getting Splits <<<< "); + HiveIdentity identity = new HiveIdentity(session); HudiTableHandle hudiTable = (HudiTableHandle) tableHandle; SchemaTableName tableName = hudiTable.getSchemaTableName(); HiveMetastore metastore = transactionManager.get(transaction).getMetastore(); - Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) + Table table = metastore.getTable(identity, tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(tableName)); - Configuration conf = hdfsEnvironment.getConfiguration(new HdfsEnvironment.HdfsContext(session), new Path(table.getStorage().getLocation())); - HudiSplitSource splitSource = new HudiSplitSource(hudiTable, conf); + HdfsEnvironment.HdfsContext context = new HdfsEnvironment.HdfsContext(session); + FileSystem fs = null; + try { + fs = hdfsEnvironment.getFileSystem(context, new Path(table.getStorage().getLocation())); + } + catch (IOException e) { + e.printStackTrace(); + } + Configuration conf = hdfsEnvironment.getConfiguration(context, new Path(table.getStorage().getLocation())); + HoodieTableMetaClient metaClient = hudiTable.getMetaClient().orElseGet(() -> getMetaClient(conf, hudiTable.getBasePath())); + List partitionValues = getPartitionsWritten(metaClient.getActiveTimeline()); + log.debug("Fetched partitions from Hudi: " + partitionValues); + hudiTable.getPartitions().ifPresent(p -> p.forEach(p1 -> log.debug("Partitions from TableHandle: " + p1))); + + List columnNames = table.getPartitionColumns().stream() + .map(Column::getName) + .collect(toImmutableList()); + log.debug("Column Names: " + columnNames); + HudiSplitSource splitSource; + String tablePath = table.getStorage().getLocation(); + Optional fileStatuses = Optional.empty(); + if (!columnNames.isEmpty()) { + List> partitionNames = metastore.getPartitionNamesByFilter(identity, tableName.getSchemaName(), tableName.getTableName(), columnNames, TupleDomain.all()) + .orElseThrow(() -> new TableNotFoundException(hudiTable.getSchemaTableName())) + .stream() + .map(HiveUtil::toPartitionValues) + .collect(toImmutableList()); + log.debug("Partition Names: " + partitionNames); + + Optional partition = metastore.getPartition(identity, table, partitionNames.get(0)); + + log.debug("Fetched partitions from Metastore: " + partition.get()); + Properties schema = getPartitionSchema(table, partition); + String dataDir = schema.getProperty(META_TABLE_LOCATION); + log.debug("Partition schema: " + schema); + + List partitionKeys = getPartitionKeys(table, partition); + partitionKeys.forEach(p -> log.warn("Fetched partitions from HiveUtil: " + p)); + + InputFormat inputFormat = HiveUtil.getInputFormat(conf, schema, false); + log.debug(">>> Check for inputFormat: " + isHudiParquetInputFormat(inputFormat)); + + try { + if (isHudiParquetInputFormat(inputFormat)) { + fileStatuses = Optional.of(((HoodieParquetInputFormat) inputFormat).listStatus(toJobConf(conf))); + } + if (fileStatuses.isPresent()) { + log.debug(">>> Total Files: " + fileStatuses.get().length); + if (fileStatuses.get().length == 0 && fs != null) { + fileStatuses = Optional.of(fs.listStatus(new Path(dataDir))); + log.debug(">>> Total Files: " + fileStatuses.get().length); + } + } + log.debug(">>> Total Splits: " + inputFormat.getSplits(toJobConf(conf), 0).length); + } + catch (IOException e) { + e.printStackTrace(); + } + splitSource = new HudiSplitSource(hudiTable, conf, partitionKeys, isHudiMetadataEnabled(session), fileStatuses, tablePath, dataDir); + } + else { + // no partitions, so data dir is same as table path + splitSource = new HudiSplitSource(hudiTable, conf, ImmutableList.of(), isHudiMetadataEnabled(session), fileStatuses, tablePath, tablePath); + } + return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index 3f9845e4a79..9e34fbfae31 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -15,23 +15,27 @@ package io.trino.plugin.hudi; import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; import io.trino.plugin.hive.HivePartitionKey; import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitSource; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import static com.google.common.collect.Iterators.limit; @@ -42,30 +46,67 @@ public class HudiSplitSource implements ConnectorSplitSource { + private static final Logger log = Logger.get(HudiSplitSource.class); + private final HudiTableHandle tableHandle; private final HoodieTableMetaClient metaClient; private final HoodieTableFileSystemView fileSystemView; private final Iterator hoodieBaseFileIterator; + private final List partitionKeys; + private final boolean metadataEnabled; + private final Optional fileStatuses; + private final String tablePath; + private final String dataDir; - public HudiSplitSource(HudiTableHandle tableHandle, Configuration conf) + public HudiSplitSource( + HudiTableHandle tableHandle, + Configuration conf, + List partitionKeys, + boolean metadataEnabled, + Optional fileStatuses, + String tablePath, + String dataDir) { this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); + this.partitionKeys = requireNonNull(partitionKeys, "partitionKeys is null"); + this.metadataEnabled = metadataEnabled; + this.fileStatuses = fileStatuses; + this.tablePath = tablePath; + this.dataDir = dataDir; this.metaClient = tableHandle.getMetaClient().orElseGet(() -> getMetaClient(conf, tableHandle.getBasePath())); HoodieEngineContext engineContext = new HoodieLocalEngineContext(conf); HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() - .enable(false) + .enable(metadataEnabled) .build(); this.fileSystemView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, metaClient, metadataConfig); - this.hoodieBaseFileIterator = fileSystemView.getLatestBaseFiles().iterator(); + + log.debug("Table path: %s \nDirectory: %s", tablePath, dataDir); + String partition = FSUtils.getRelativePartitionPath(new Path(tablePath), new Path(dataDir)); + log.debug("Partition: %s", partition); + if (fileStatuses.isPresent()) { + log.warn(">>> FileStatus present adding to view: %s", fileStatuses.get().length); + fileSystemView.addFilesToView(fileStatuses.get()); + this.hoodieBaseFileIterator = fileSystemView.fetchLatestBaseFiles(partition).iterator(); + } + else { + this.hoodieBaseFileIterator = fileSystemView.getLatestBaseFiles(partition).iterator(); + } } @Override public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) { + log.debug("Getting next batch with partitionKeys: " + partitionKeys); List splits = new ArrayList<>(); Iterator baseFileIterator = limit(hoodieBaseFileIterator, maxSize); while (baseFileIterator.hasNext()) { HoodieBaseFile baseFile = baseFileIterator.next(); + log.warn(">>>> Base File: " + baseFile); + FileStatus fileStatus = baseFile.getFileStatus(); + log.warn(">>>> FileStatus: " + fileStatus.toString()); + /*String[] name = new String[] {"localhost:" + DFS_DATANODE_DEFAULT_PORT}; + String[] host = new String[] {"localhost"}; + BlockLocation[] blockLocations = new BlockLocation[] {new BlockLocation(name, host, 0L, fileStatus.getLen())};*/ splits.add(new HudiSplit( baseFile.getPath(), 0L, @@ -73,7 +114,7 @@ public CompletableFuture getNextBatch(ConnectorPartitionHan baseFile.getFileSize(), ImmutableList.of(), tableHandle.getPredicate(), - getPartitionKeys())); + partitionKeys)); } return completedFuture(new ConnectorSplitBatch(splits, isFinished())); @@ -90,12 +131,4 @@ public boolean isFinished() { return !hoodieBaseFileIterator.hasNext(); } - - private List getPartitionKeys() - { - List partitionKeys = new ArrayList<>(); - List partitions = TimelineUtils.getPartitionsWritten(metaClient.getActiveTimeline()); - - return partitionKeys; - } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java index 5a1ab973980..fcd18de72b0 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java @@ -14,8 +14,17 @@ package io.trino.plugin.hudi; +import io.trino.plugin.hive.metastore.Partition; +import io.trino.plugin.hive.metastore.Table; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapred.InputFormat; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.hadoop.HoodieParquetInputFormat; + +import java.util.Optional; +import java.util.Properties; + +import static io.trino.plugin.hive.metastore.MetastoreUtil.getHiveSchema; public class HudiUtil { @@ -25,4 +34,17 @@ public static HoodieTableMetaClient getMetaClient(Configuration conf, String bas { return HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath).build(); } + + public static boolean isHudiParquetInputFormat(InputFormat inputFormat) + { + return inputFormat instanceof HoodieParquetInputFormat; + } + + public static Properties getPartitionSchema(Table table, Optional partition) + { + if (partition.isEmpty()) { + return getHiveSchema(table); + } + return getHiveSchema(partition.get(), table); + } } From 955a6f5b181f344f6cf56978037f5f39fb7d737b Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Fri, 17 Dec 2021 20:11:44 +0530 Subject: [PATCH 07/26] Add tests for the connector --- plugin/trino-hudi/pom.xml | 75 ++++++++ .../trino/plugin/hudi/HudiSplitManager.java | 45 +++-- .../hudi/AbstractHudiTestQueryFramework.java | 178 ++++++++++++++++++ .../plugin/hudi/TestHudiConnectorFactory.java | 68 +++++++ .../io/trino/plugin/hudi/TestHudiSanity.java | 39 ++++ .../io/trino/plugin/hudi/TestHudiSplit.java | 51 +++++ .../plugin/hudi/TestHudiTableHandle.java | 49 +++++ .../.hoodie/20211217110514527.commit | 50 +++++ .../20211217110514527.commit.requested | 0 .../.hoodie/20211217110514527.inflight | 48 +++++ .../.hoodie/hoodie.properties | 14 ++ .../.hoodie_partition_metadata | 4 + ...66c47a0-0_0-6-11_20211217110514527.parquet | Bin 0 -> 436273 bytes .../.hoodie/20211216071453747.commit | 51 +++++ .../20211216071453747.commit.requested | 0 .../.hoodie/20211216071453747.inflight | 48 +++++ .../stock_ticks_cow/.hoodie/hoodie.properties | 13 ++ .../2018/08/31/.hoodie_partition_metadata | 4 + ...317216-0_0-28-26_20211216071453747.parquet | Bin 0 -> 440747 bytes 19 files changed, 718 insertions(+), 19 deletions(-) create mode 100644 plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/AbstractHudiTestQueryFramework.java create mode 100644 plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConnectorFactory.java create mode 100644 plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSanity.java create mode 100644 plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSplit.java create mode 100644 plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiTableHandle.java create mode 100644 plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.commit create mode 100644 plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.commit.requested create mode 100644 plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.inflight create mode 100644 plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/hoodie.properties create mode 100644 plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie_partition_metadata create mode 100644 plugin/trino-hudi/src/test/resources/hudi_non_part_cow/d0875d00-483d-4e8b-bbbe-c520366c47a0-0_0-6-11_20211217110514527.parquet create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.commit create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.commit.requested create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.inflight create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/hoodie.properties create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_cow/2018/08/31/.hoodie_partition_metadata create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_cow/2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet diff --git a/plugin/trino-hudi/pom.xml b/plugin/trino-hudi/pom.xml index 5142e356668..0d66076edc4 100644 --- a/plugin/trino-hudi/pom.xml +++ b/plugin/trino-hudi/pom.xml @@ -188,6 +188,81 @@ jol-core provided + + + + io.trino + trino-hive + test-jar + test + + + io.trino + trino-hive-hadoop2 + test + + + io.trino + trino-main + test + + + io.trino + trino-main + test-jar + test + + + io.trino + trino-parser + test + + + io.trino + trino-spi + test-jar + test + + + io.trino + trino-testing + test + + + io.trino + trino-testing-services + test + + + io.trino + trino-tpch + test + + + io.trino.tpch + tpch + test + + + io.airlift + testing + test + + + org.assertj + assertj-core + test + + + org.jetbrains + annotations + test + + + org.testng + testng + test + diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index 5102357ce58..e1dda79fd4a 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -131,32 +131,39 @@ public ConnectorSplitSource getSplits( List partitionKeys = getPartitionKeys(table, partition); partitionKeys.forEach(p -> log.warn("Fetched partitions from HiveUtil: " + p)); - InputFormat inputFormat = HiveUtil.getInputFormat(conf, schema, false); - log.debug(">>> Check for inputFormat: " + isHudiParquetInputFormat(inputFormat)); - - try { - if (isHudiParquetInputFormat(inputFormat)) { - fileStatuses = Optional.of(((HoodieParquetInputFormat) inputFormat).listStatus(toJobConf(conf))); - } - if (fileStatuses.isPresent()) { - log.debug(">>> Total Files: " + fileStatuses.get().length); - if (fileStatuses.get().length == 0 && fs != null) { - fileStatuses = Optional.of(fs.listStatus(new Path(dataDir))); - log.debug(">>> Total Files: " + fileStatuses.get().length); - } - } - log.debug(">>> Total Splits: " + inputFormat.getSplits(toJobConf(conf), 0).length); - } - catch (IOException e) { - e.printStackTrace(); - } + fileStatuses = getFileStatuses(fs, conf, dataDir, fileStatuses, schema); splitSource = new HudiSplitSource(hudiTable, conf, partitionKeys, isHudiMetadataEnabled(session), fileStatuses, tablePath, dataDir); } else { // no partitions, so data dir is same as table path + Properties schema = getPartitionSchema(table, Optional.empty()); + fileStatuses = getFileStatuses(fs, conf, tablePath, fileStatuses, schema); splitSource = new HudiSplitSource(hudiTable, conf, ImmutableList.of(), isHudiMetadataEnabled(session), fileStatuses, tablePath, tablePath); } return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); } + + private static Optional getFileStatuses(FileSystem fs, Configuration conf, String tablePath, Optional fileStatuses, Properties schema) + { + InputFormat inputFormat = HiveUtil.getInputFormat(conf, schema, false); + log.debug(">>> Check for inputFormat: " + isHudiParquetInputFormat(inputFormat)); + + try { + if (isHudiParquetInputFormat(inputFormat)) { + fileStatuses = Optional.of(((HoodieParquetInputFormat) inputFormat).listStatus(toJobConf(conf))); + } + if (fileStatuses.isPresent()) { + log.debug(">>> Total Files: " + fileStatuses.get().length); + if (fileStatuses.get().length == 0 && fs != null) { + fileStatuses = Optional.of(fs.listStatus(new Path(tablePath))); + log.debug(">>> Total Files: " + fileStatuses.get().length); + } + } + } + catch (IOException e) { + e.printStackTrace(); + } + return fileStatuses; + } } diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/AbstractHudiTestQueryFramework.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/AbstractHudiTestQueryFramework.java new file mode 100644 index 00000000000..1e68a9b1ac7 --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/AbstractHudiTestQueryFramework.java @@ -0,0 +1,178 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableMap; +import io.trino.Session; +import io.trino.plugin.hive.HivePlugin; +import io.trino.plugin.tpch.TpchPlugin; +import io.trino.spi.type.TimeZoneKey; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.QueryRunner; + +import java.nio.file.Path; +import java.util.Locale; +import java.util.Map; + +import static io.trino.testing.TestingSession.testSessionBuilder; +import static java.lang.String.format; + +public class AbstractHudiTestQueryFramework + extends AbstractTestQueryFramework +{ + public static final String HUDI_CATALOG = "hudi"; + public static final String HIVE_CATALOG = "hive"; + public static final String HUDI_SCHEMA = "default"; + + static final String NON_PARTITIONED_TABLE_NAME = "hudi_non_part_cow"; + static final String PARTITIONED_TABLE_NAME = "stock_ticks_cow"; + + private static final String CREATE_NON_PARTITIONED_TABLE_STATEMENT = "CREATE TABLE %s.\"%s\".\"%s\" (\n" + + " _hoodie_commit_time varchar,\n" + + " _hoodie_commit_seqno varchar,\n" + + " _hoodie_record_key varchar,\n" + + " _hoodie_partition_path varchar,\n" + + " _hoodie_file_name varchar,\n" + + " rowid varchar,\n" + + " partitionid varchar,\n" + + " precomb bigint,\n" + + " name varchar,\n" + + " versionid varchar,\n" + + " tobedeletedstr varchar,\n" + + " inttolong integer,\n" + + " longtoint bigint\n" + + " )\n" + + " WITH (\n" + + " external_location = '%s',\n" + + " format = 'PARQUET'\n" + + " )"; + + private static final String CREATE_PARTITIONED_TABLE_STATEMENT = "CREATE TABLE %s.\"%s\".\"%s\" (\n" + + " _hoodie_commit_time varchar,\n" + + " _hoodie_commit_seqno varchar,\n" + + " _hoodie_record_key varchar,\n" + + " _hoodie_partition_path varchar,\n" + + " _hoodie_file_name varchar,\n" + + " volume bigint,\n" + + " ts varchar,\n" + + " symbol varchar,\n" + + " year integer,\n" + + " month varchar,\n" + + " high double,\n" + + " low double,\n" + + " key varchar,\n" + + " date varchar,\n" + + " close double,\n" + + " open double,\n" + + " day varchar,\n" + + " dt varchar\n" + + " )\n" + + " WITH (\n" + + " external_location = '%s',\n" + + " format = 'PARQUET',\n" + + " partitioned_by = ARRAY['dt']\n" + + " )"; + + private static final Map TABLE_NAME_TO_CREATE_STATEMENT = new ImmutableMap.Builder() + .put(NON_PARTITIONED_TABLE_NAME, CREATE_NON_PARTITIONED_TABLE_STATEMENT) + .put(PARTITIONED_TABLE_NAME, CREATE_PARTITIONED_TABLE_STATEMENT) + .build(); + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return createHudiQueryRunner(ImmutableMap.of()); + } + + protected void assertHudiQuery(String table, String testQuery, String expResults, boolean fail) + { + try { + syncHudiTableInMetastore(table); + if (!fail) { + assertQuery(testQuery, expResults); + } + else { + assertQueryFails(testQuery, expResults); + } + } + finally { + dropHudiTableFromMetastore(table); + } + } + + protected static String getTableBasePath(String tableName) + { + return AbstractHudiTestQueryFramework.class.getClassLoader().getResource(tableName).toString(); + } + + private static DistributedQueryRunner createHudiQueryRunner(Map extraProperties) + throws Exception + { + Session session = testSessionBuilder() + .setCatalog(HUDI_CATALOG) + .setSchema(HUDI_SCHEMA.toLowerCase(Locale.ROOT)) + .setTimeZoneKey(TimeZoneKey.UTC_KEY) + .build(); + + DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(session) + .setExtraProperties(extraProperties) + .build(); + + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + + Path dataDir = queryRunner.getCoordinator().getBaseDataDir().resolve("hudi_metadata"); + Path catalogDir = dataDir.getParent().resolve("catalog"); + + // Install Hudi connector + queryRunner.installPlugin(new HudiPlugin()); + Map hudiProperties = ImmutableMap.builder() + .put("hive.metastore", "file") + .put("hive.metastore.catalog.dir", catalogDir.toFile().toURI().toString()) + .build(); + queryRunner.createCatalog(HUDI_CATALOG, "hudi", hudiProperties); + + // Install Hive connector + queryRunner.installPlugin(new HivePlugin()); + Map hiveProperties = ImmutableMap.builder() + .put("hive.metastore", "file") + .put("hive.metastore.catalog.dir", catalogDir.toFile().toURI().toString()) + .put("hive.allow-drop-table", "true") + .put("hive.security", "legacy") + .build(); + queryRunner.createCatalog(HIVE_CATALOG, "hive", hiveProperties); + queryRunner.execute(format("CREATE SCHEMA %s.%s", HIVE_CATALOG, HUDI_SCHEMA)); + + return queryRunner; + } + + protected void syncHudiTableInMetastore(String tableName) + { + getQueryRunner().execute(format( + TABLE_NAME_TO_CREATE_STATEMENT.get(tableName), + HIVE_CATALOG, + HUDI_SCHEMA, + tableName, + getTableBasePath(tableName))); + } + + protected void dropHudiTableFromMetastore(String tableName) + { + getQueryRunner().execute( + format("DROP TABLE IF EXISTS %s.\"%s\".\"%s\"", HIVE_CATALOG, HUDI_SCHEMA, tableName)); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConnectorFactory.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConnectorFactory.java new file mode 100644 index 00000000000..1fd113dbd7e --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConnectorFactory.java @@ -0,0 +1,68 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableMap; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata; +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitManager; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.testing.TestingConnectorContext; +import org.testng.annotations.Test; + +import java.util.Map; + +import static io.airlift.testing.Assertions.assertInstanceOf; +import static io.trino.plugin.hudi.AbstractHudiTestQueryFramework.HUDI_CATALOG; +import static io.trino.spi.transaction.IsolationLevel.READ_UNCOMMITTED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class TestHudiConnectorFactory +{ + @Test + public void testCreateConnector() + { + assertCreateConnector("thrift://localhost:1234"); + assertCreateConnector("thrift://localhost:1234,thrift://192.0.2.3:5678"); + + assertCreateConnectorFails("abc", "metastoreUri scheme is missing: abc"); + assertCreateConnectorFails("thrift://:8090", "metastoreUri host is missing: thrift://:8090"); + assertCreateConnectorFails("thrift://localhost", "metastoreUri port is missing: thrift://localhost"); + assertCreateConnectorFails("abc::", "metastoreUri scheme must be thrift: abc::"); + assertCreateConnectorFails("", "metastoreUris must specify at least one URI"); + assertCreateConnectorFails("thrift://localhost:1234,thrift://test-1", "metastoreUri port is missing: thrift://test-1"); + } + + private static void assertCreateConnector(String metastoreUri) + { + Map config = ImmutableMap.builder() + .put("hive.metastore.uri", metastoreUri) + .build(); + + Connector connector = new HudiConnectorFactory(HUDI_CATALOG).create("test", config, new TestingConnectorContext()); + ConnectorTransactionHandle transaction = connector.beginTransaction(READ_UNCOMMITTED, true, true); + assertInstanceOf(connector.getMetadata(transaction), ClassLoaderSafeConnectorMetadata.class); + assertInstanceOf(connector.getSplitManager(), ClassLoaderSafeConnectorSplitManager.class); + assertInstanceOf(connector.getPageSourceProvider(), ConnectorPageSourceProvider.class); + connector.commit(transaction); + } + + private static void assertCreateConnectorFails(String metastoreUri, String exceptionString) + { + assertThatThrownBy(() -> assertCreateConnector(metastoreUri)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining(exceptionString); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSanity.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSanity.java new file mode 100644 index 00000000000..8aecfb78fd1 --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSanity.java @@ -0,0 +1,39 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import org.testng.annotations.Test; + +import static java.lang.String.format; + +public class TestHudiSanity + extends AbstractHudiTestQueryFramework +{ + @Test + public void readNonPartitionedTable() + { + String testQuery = format("SELECT rowid, name FROM \"%s\"", NON_PARTITIONED_TABLE_NAME); + String expResults = "SELECT * FROM VALUES('row_1', 'bob'),('row_2', 'john'),('row_3', 'tom')"; + assertHudiQuery(NON_PARTITIONED_TABLE_NAME, testQuery, expResults, false); + } + + @Test + public void readPartitionedTable() + { + String testQuery = format("SELECT symbol, max(ts) FROM \"%s\" group by symbol HAVING symbol = 'GOOG'", PARTITIONED_TABLE_NAME); + String expResults = "SELECT * FROM VALUES('GOOG', '2018-08-31 10:59:00')"; + assertHudiQuery(PARTITIONED_TABLE_NAME, testQuery, expResults, false); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSplit.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSplit.java new file mode 100644 index 00000000000..ecc6ae014eb --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSplit.java @@ -0,0 +1,51 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableList; +import io.airlift.json.JsonCodec; +import io.trino.spi.predicate.TupleDomain; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; + +public class TestHudiSplit +{ + private final JsonCodec codec = JsonCodec.jsonCodec(HudiSplit.class); + + @Test + public void testJsonRoundTrip() + { + HudiSplit expectedSplit = new HudiSplit( + "/user/hive/warehouse/stock_ticks_cow/2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet", + 0L, + 440747L, + 440747L, + ImmutableList.of(), + TupleDomain.all(), + ImmutableList.of()); + + String json = codec.toJson(expectedSplit); + HudiSplit actualSplit = codec.fromJson(json); + + assertEquals(actualSplit.getPath(), expectedSplit.getPath()); + assertEquals(actualSplit.getAddresses(), expectedSplit.getAddresses()); + assertEquals(actualSplit.getPartitionKeys(), expectedSplit.getPartitionKeys()); + assertEquals(actualSplit.getStart(), expectedSplit.getStart()); + assertEquals(actualSplit.getLength(), expectedSplit.getLength()); + assertEquals(actualSplit.getFileSize(), expectedSplit.getFileSize()); + assertEquals(actualSplit.getAddresses(), expectedSplit.getAddresses()); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiTableHandle.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiTableHandle.java new file mode 100644 index 00000000000..3541b081436 --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiTableHandle.java @@ -0,0 +1,49 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import io.airlift.json.JsonCodec; +import io.trino.spi.predicate.TupleDomain; +import org.apache.hudi.common.model.HoodieTableType; +import org.testng.annotations.Test; + +import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE; +import static org.testng.Assert.assertEquals; + +public class TestHudiTableHandle +{ + private final JsonCodec codec = JsonCodec.jsonCodec(HudiTableHandle.class); + + @Test + public void testJsonRoundTrip() + { + HudiTableHandle expectedHandle = new HudiTableHandle( + "schema", + "table", + "/tmp/hudi_trips", + HoodieTableType.valueOf(COPY_ON_WRITE.name()), + TupleDomain.all()); + + String json = codec.toJson(expectedHandle); + HudiTableHandle actualHandle = codec.fromJson(json); + + assertEquals(actualHandle.getSchemaName(), expectedHandle.getSchemaName()); + assertEquals(actualHandle.getTableName(), expectedHandle.getTableName()); + assertEquals(actualHandle.getPredicate(), expectedHandle.getPredicate()); + assertEquals(actualHandle.getTableType(), expectedHandle.getTableType()); + assertEquals(actualHandle.getBasePath(), expectedHandle.getBasePath()); + assertEquals(actualHandle.getTableType(), expectedHandle.getTableType()); + } +} diff --git a/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.commit b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.commit new file mode 100644 index 00000000000..f77eeb137f0 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.commit @@ -0,0 +1,50 @@ +{ + "partitionToWriteStats" : { + "" : [ { + "fileId" : "d0875d00-483d-4e8b-bbbe-c520366c47a0-0", + "path" : "d0875d00-483d-4e8b-bbbe-c520366c47a0-0_0-6-11_20211217110514527.parquet", + "prevCommit" : "null", + "numWrites" : 3, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 436273, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 436273, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"hudi_non_part_cow_record\",\"namespace\":\"hoodie.hudi_non_part_cow\",\"fields\":[{\"name\":\"rowId\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"partitionId\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"preComb\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"name\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"versionId\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"toBeDeletedStr\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"intToLong\",\"type\":[\"null\",\"int\"],\"default\":null},{\"name\":\"longToInt\",\"type\":[\"null\",\"long\"],\"default\":null}]}" + }, + "operationType" : "INSERT", + "writePartitionPaths" : [ "" ], + "fileIdAndRelativePaths" : { + "d0875d00-483d-4e8b-bbbe-c520366c47a0-0" : "d0875d00-483d-4e8b-bbbe-c520366c47a0-0_0-6-11_20211217110514527.parquet" + }, + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 1743, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + } +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.commit.requested b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.commit.requested new file mode 100644 index 00000000000..e69de29bb2d diff --git a/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.inflight b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.inflight new file mode 100644 index 00000000000..6605bcaf9b3 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/20211217110514527.inflight @@ -0,0 +1,48 @@ +{ + "partitionToWriteStats" : { + "" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "INSERT", + "writePartitionPaths" : [ "" ], + "fileIdAndRelativePaths" : { + "" : null + }, + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + } +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/hoodie.properties b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/hoodie.properties new file mode 100644 index 00000000000..3d03fa7915c --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie/hoodie.properties @@ -0,0 +1,14 @@ +#Properties saved on Fri Dec 17 11:05:14 UTC 2021 +#Fri Dec 17 11:05:14 UTC 2021 +hoodie.table.precombine.field=preComb +hoodie.table.partition.fields= +hoodie.table.type=COPY_ON_WRITE +hoodie.archivelog.folder=archived +hoodie.populate.meta.fields=true +hoodie.timeline.layout.version=1 +hoodie.table.version=3 +hoodie.table.recordkey.fields=rowId +hoodie.table.base.file.format=PARQUET +hoodie.table.keygenerator.class=org.apache.hudi.keygen.NonpartitionedKeyGenerator +hoodie.table.name=hudi_non_part_cow +hoodie.datasource.write.hive_style_partitioning=false diff --git a/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie_partition_metadata b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie_partition_metadata new file mode 100644 index 00000000000..f2149eb6cd5 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Fri Dec 17 11:05:23 UTC 2021 +commitTime=20211217110514527 +partitionDepth=0 diff --git a/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/d0875d00-483d-4e8b-bbbe-c520366c47a0-0_0-6-11_20211217110514527.parquet b/plugin/trino-hudi/src/test/resources/hudi_non_part_cow/d0875d00-483d-4e8b-bbbe-c520366c47a0-0_0-6-11_20211217110514527.parquet new file mode 100644 index 0000000000000000000000000000000000000000..52de8719bf62d7a762237ff4fae1887ee250f57a GIT binary patch literal 436273 zcmeI*e`uWNeE{(HWLb6_&a7@qpiYCE#*#@B|G!hNQ%>`%SxPumNtabk2}dC zlFm7u?S+yplp(e@Y{P{9(SO>EkaijLuMtKGgTe|eW!>2FM=62Obj$wPMl%XqG8lW` z(~mprPPU|+WS#CKjQsw1p6~O0KF|BU&nxNPyT^tf&xGya{o&(}hHYVc5d7*_4+qO% z$8+Q3>Hc*8z(D`N(7-_Y>_Fyh|DoW|Zv5?+!>B|Ug!hE!PKO;)^^WjxSL#?)Hk0Yk z3}rIuvzdXj{X=y&C4*C&Di<|S$z*DBe0+R*X1pW#e)^Yx8Rr)@{uhbxO5A!-ZhbH~ z71p$$O=r)f>Tb@|iT}}qHy-JDWNGP2H4hg)3vJ1Hp}N zyf|=ax$Tkgvo|$;uvoYrHC6tvUg(*FH!i;SukS9UmyW$KaU}SD__6QY7e{Ij4~OX* z_^GJ3LD2Qq!}mqwQR^I}f=Ds?9(~=?s6G5zBK&ruCjPmfc|KjtX{@?5W{pcrp2OhZVo~NJvlmAG5Ma<@V`Z`rOdjTsqx5aPCa5cOZXmvUhTFGT%FOwm*Gl zaBym1Xd>O4&Zc_@do!7A&4~3~nJ7MUHD5|KDo`ZVs89{d>2~85gpE7Cy29V88ULS0 zr&D}lzcCct7yM-I%l99xHwd2Ovdt;{`y8n}( z`n9|NuJ?!ka4;zT)rntrMMa}?_Gma*+l}e*@l#(N3WC9Jw*A|oxZL{7K2+1#&!Y>(sUKed%+;~KkFGX%-T0$FJyNg5!==JV{^9&wI<>YG zf^xQ)IBjuryyWZAm9kTX>$O|5#r#yEmkun zJkoi#<8XUtU7xT$*^&HOTk@T@p2HXF+%weK6bZX-HRM<;B1*cP9U@qa(nihoqd(t8K~uNknBY3~u$#qO&V%E24?pdZ)e8R-F+pxAez}HldEeYlkY)YLB0Cpw~K*zv$?B?a+mW&3g?C6pjAg zo7C$lzjCxP`pf5H)quZp^wtCZ-McG8RDJ4}L;PbR`SrG*@7{f(dIGLqj)*pzrt2j7 z#<8xjedP>Yy(E;|`RhdT{Y1|j$1YT6+R9SYM!lK1PW&GquRxS9GUedc;^Y3u$76=d zWL$mBzkP3opmI4X*ZoQ&`O`$t+xK25PpP%@YMmrrPgbI>yOfnfzMe?F(AM*MGU{vX z6dCumc56_lnIELO!nPI5%B8W~!V8Jyn~9zuq%JH^i*+Z*IuTz!Q5l`e#kCwU8lV45 z^t^l`%DHx?i*v5sS|{h^eJ{Oa(;dy_0NaC)LnY^O1}HS z?&Z0u`*e3aQAF!C^W(~FxKd*A%EVN;*aov{RBU=SKbKqVe(=dJt;SrP&8jBQsN8Y_ zPj)X{otukR7fZ$2g_-WBPIu=DQ&A$(rd)n{;_6(f`@y)#bEh|oSDthm60kB+HzeHJ z>>S-}lrA?U|_2pZW+N$yWMS18qXt#WD-s+Iqpx)XI zPi>P`$J+*#Z@xF$PK34FsM^ZWu3L3@?S`wiW__!to(odPt`}!Z`C{LEA(x-)i_KE) z@o68VKDKfTJ2_V<%x9y6C@PkXPr&G=HkVI*_~9>ncKCP4Mzfzw2dQ5v6leM-qVxRn z^4)S@xzci-)Qhj4cp(0|K0G`;GB7ecnmhm4%<%Bwe6&71{BZgEbX?$EyqF)pG`ukV z#6WqoZv+SsAV7cs0RjXF5FkK+009C72oTu1z{RasQG);h0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB=C82#hu$h7be@5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0((@z8^7++fsqyg0t5&UAV7cs0RjXF z5FkLH1q8-gppUvpfB*pk1PBlyK!5-N0=pqFvm1?Qg8%^n1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB=CF1-ygFhBS5%AV7cs0RjXF5FoHl;L^Hvs{{xTAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72y9ls z(>a@!QjP!t0t5&UAV6RQ0RaL82oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZ;DZW`eb4~v6Cgl<009C72y8;YBgC6PQiK2j0t5&UAV7cs0RjXF5Fl_X zftg$Fw2uG*0t5&UAV7cs0RnqMz&ix)iD8fg0RjXF5FkK+K$U=pMyfC@5+Fc;z)lNz z)%^B^{Yq(jZfX%AK!5-N0t5&UAV7e?CI$SSc#}|y5g3;3nMz8_*8BtU=w0RjXFG@Zam(`77S0tA{~ zz=fwthtMO_O*(MWCP09|CIn2*n?O>8009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FoH#z>6-{i&!T>fB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBly&>RAnnj=kF5+Fc;009C72oNAZfB=DgC*b2e_uY`{5CH-N2oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXFw7kGf%L@P?K!5-N0vicQN z0t5&UAV7e?9TOP2V=Zf#009C72oNAZfB*pk1PBn=tbn&X*sPRt1PBlyK!5-N0t5&U zAV7cs0RjXF>_Gu9QQCupA|(O@2oNAZfB*pk1PBlyK!5-N0$UgG>0MhlQ-c5j0t5&U zAV7cs0RjYeR=|^YJL@Yt0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkLH*#&$sce9V6{0R^sK!5-N0t5&UAV7cs0RjXF5FkK+009C7nq6R1A6-;_ z(^Kyz)9hm{e*y&dhJdT--WUg25FkK+009C72oNAZfB*pk1PHW>z)Y)jR6hw2AVA=b z2<(Ty{P+~qJ96k~lmGz&1PBlyK!5-N0t5&UAV7cs0RjYeOJGxv&+S$++95!I0D=7^ zV2asKqp2?h2oNAZfB=Ek5%B3CtuxkoN`L?X0t5&U*t)=ecuHpLy;g$&0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&&Sm0uV;|WKA009C72oNAZfB*pk1PBmlfPhb0X#k=S1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72<()=%uWRsmjD3*1PBlyK!5-N0t5&UAV7cs0RjZJCosCb zYHAT6K!5-N0t5&UAV7csfmRXlqjRf_w0;sGK!Ct*3wT}eZi{P=009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+Kr0Ef?uWUxZqHlk_|{hf1PBlyK!5-N0t5&U zxbp&@j=J+>AOivf2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7e?jtKZD+Z|yQlmGz&1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNB!PT=CYb*lsjY(?PGRthUifB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkKc{|JoipRVW$0RjXF5FkK+009C72oNAZfB*pk1PJV~z}ODQ7oGrt))w%PWNVMV z-V-1|fB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfWU4Fc-7u+YHz14 zZ=Js#wY^ryEAaQ)h{%Tk0RjXF5FkK+009ESe&BWx5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyKwwV_c&Cm%IV_c=W-5!# zzqGG9)8xb9%y04mls*9h1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB=CV z7VsLQ9R?Pj009C72oNAZfB*pk1PBlyK!5-N0?jBe+KfrdnE-)p3HW2tw%Al6K!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ zK$8kwYSO%=O@M%Bz#Rhw2oNAZfB*pk1PBly&|ra)2FDYQz|8{Q;Ob@%YXk@oAV7cs z0RjXF5FoJ61-vF_pAWF^5g%=*2oNAZfB*pk1PBly zK!5-N0t5&UAaLgdX6{_mS|&h%009C72viC9JijUoiv$P|AV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjYCPhhO|x~#`dDDay>D!g1M|E~T#JTgi>2uTP z&!6d^9L!G-4xTxaOAn?8&*#skr_N6fogYe1<|lKRPjtT76$G347o=Y52p$`LJoEp) CsCG~Q literal 0 HcmV?d00001 diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.commit b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.commit new file mode 100644 index 00000000000..18cf55cc1bf --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.commit @@ -0,0 +1,51 @@ +{ + "partitionToWriteStats" : { + "2018/08/31" : [ { + "fileId" : "871677fb-e0e3-46f8-9cc1-fe497e317216-0", + "path" : "2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet", + "prevCommit" : "null", + "numWrites" : 99, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 99, + "totalWriteBytes" : 440747, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "2018/08/31", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 440747, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"stock_ticks\",\"fields\":[{\"name\":\"volume\",\"type\":\"long\"},{\"name\":\"ts\",\"type\":\"string\"},{\"name\":\"symbol\",\"type\":\"string\"},{\"name\":\"year\",\"type\":\"int\"},{\"name\":\"month\",\"type\":\"string\"},{\"name\":\"high\",\"type\":\"double\"},{\"name\":\"low\",\"type\":\"double\"},{\"name\":\"key\",\"type\":\"string\"},{\"name\":\"date\",\"type\":\"string\"},{\"name\":\"close\",\"type\":\"double\"},{\"name\":\"open\",\"type\":\"double\"},{\"name\":\"day\",\"type\":\"string\"}]}", + "deltastreamer.checkpoint.key" : "stock_ticks,0:1668" + }, + "operationType" : "UPSERT", + "fileIdAndRelativePaths" : { + "871677fb-e0e3-46f8-9cc1-fe497e317216-0" : "2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet" + }, + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 750, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "2018/08/31" ] +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.commit.requested b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.commit.requested new file mode 100644 index 00000000000..e69de29bb2d diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.inflight b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.inflight new file mode 100644 index 00000000000..6dc689a285d --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/20211216071453747.inflight @@ -0,0 +1,48 @@ +{ + "partitionToWriteStats" : { + "2018/08/31" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 99, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT", + "fileIdAndRelativePaths" : { + "" : null + }, + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "2018/08/31" ] +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/hoodie.properties b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/hoodie.properties new file mode 100644 index 00000000000..4754c1c23eb --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/.hoodie/hoodie.properties @@ -0,0 +1,13 @@ +#Properties saved on Thu Dec 16 07:14:51 UTC 2021 +#Thu Dec 16 07:14:51 UTC 2021 +hoodie.table.precombine.field=ts +hoodie.table.partition.fields=date +hoodie.table.type=COPY_ON_WRITE +hoodie.archivelog.folder=archived +hoodie.populate.meta.fields=true +hoodie.timeline.layout.version=1 +hoodie.table.version=3 +hoodie.table.recordkey.fields=key +hoodie.table.base.file.format=PARQUET +hoodie.table.keygenerator.class=org.apache.hudi.keygen.SimpleKeyGenerator +hoodie.table.name=stock_ticks_cow diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_cow/2018/08/31/.hoodie_partition_metadata b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/2018/08/31/.hoodie_partition_metadata new file mode 100644 index 00000000000..1aaf9e64d93 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/2018/08/31/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Thu Dec 16 07:14:56 UTC 2021 +commitTime=20211216071453747 +partitionDepth=3 diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_cow/2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet b/plugin/trino-hudi/src/test/resources/stock_ticks_cow/2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet new file mode 100644 index 0000000000000000000000000000000000000000..b97391697e6242ef36b01b70fde44dfc50bf01b3 GIT binary patch literal 440747 zcmeHw2VhlIwsr_1gcd+Z3yyPg9?g@v;-0(gZ|BWW-(& zyCZ@HX%3)N3kZs+PY@|0qM{5qB13uSCb{>Xw$Isp?{n^+_x~^F?6TIkzV)rO_qooS z$qQVckrbD7Nz(P#B*iAh#l&3jKtfE}zv|@o@1Nc}y>(`0>&!OkS(zDa+ht``j`?%@ zC%-0XE0SW8>Lz8kNZMFI+aI5l&>*F@wm2iBbw<1Ntc=XIZL%`6Y=)9!nnJD9JVc-j z8r;8s_QuT#>tk+89lxMznfE7}CY`ES>OZE`f4i8b$rc3t();D)q$JsmXBb?J-S+m_ ziW$|r27X=4pQgPoD9p=jTEA|M+3{CQTNv0LeEr7ee+uho)_5wu$Fx@h zUoI%xmKl7V5uZElg%7v&FHFx&sb9VHHFuizf3BWp{-3+g#!Psw?(XUui&_eIHcEQ5 zvXR7d`do4Ijs04uXJofb&u-Z!BQ+yE<(xnu?;7KnfxY4LdyLqhUj4osKCAG5TW{;% zDlfGtYxfr?->rZDxTbx^t^9fT-EXCx?sMbB<`pkUnEmj?m(Kn=Zq;8;KfUJhnwz$- zzU)ZN&ujkgrmC4^G){p+AMZ)%_uD&9ztSva!M_^b{prL%KaE|wZ%}Mz;rU_l<-oCKjjcZ@~ru~?6?;79mqU-@<*gd%|25)JT$rLyyDJ_9%=QZg4as8K~h&s3)nxfv<1AkEo;fPtafoR zFSK4VB(bdeOtoJ%sFzFAvoCFvk)nIns$KV{pE1bCroQ8no~V%YRs{>B8k@J*n4F$a zX>F_N6=r`huHv%7_ofuLUAgA_pN7BuX#A+unpNlg2wEn#tuCa}yTXt4PyR59i16vlP7qn@a*=}%l%bY=j zGFlEU$jr$qXp@np^|zMk{nA^u&TiSdomLX2me*?JfYGDI7L=q=v!roI%@#dWZuyHz zT2ejfu}bBwB_Vh|BpkjyR@)X6lai2M`Zp#Hg!PV89m~!n`rzUX#mgpaJzZhakakBmwd(!H zqO4huA8LKzxhu~8Q2Vd#pO1ZD|3d?xJ2mQLwG+>-9Xq=E_Io!^s=M>&tM~m>aJJ>N zf1m7ncGiO@b5Hy^`@hNC19KWqy!(m2DsB07fAe{ZA87Dd&5jQ|ePB|Td)r)k{)^wO z-LqhI`%CWm&plT>p8fHzyGwpQ@X(Sy&Hg<8;GvCw&B>{zVUz` zyY;&oF{uf;J;#l#aBga|eHC}#n|^xPspocOwru=G{}&R*{%b~?eK9}&wB^oGN1nO( zr@)G1!?SMB+WPVDFI}5>a(}fI8P`p3UZM098J0A!LitS~m}%`Ku46bq+PBSVpPrs# z8f=rH4H|Mi=#jy-Zp<$k+`n#q-GW-R>(*&;V}9Mbg|!pnW+z-yHKXe6^d5Eds@JO9 zf7gaf8h^66=_MC#-n=#@X2!J}546(^Y3JGTYDwSdMW;cD3!Y~walsSJ;H>TSL+@Ib zp76`g-}U4Rm++~pN~B>Dx>L>i|P&BQ~kyCdWqkDl~;Azv7<5PJ|EcD`=gt`IGVX@#O$Yf z9om0iQf}^)O{pVWuHU}A%arpEx1QMRnHh`XMn0SM?CR!IZx}s%?aD)c-92K{f4aOi zsY71#O><^`y!6=53$LH>#--Q%QS|pCceJ1I!?$l$Y~5y0{|}$)_GXXkZaZ^N*EcHf z=rum|nGrX(e|+kXFTRlYPT(^QypIlDSn<#8 zYty<{IoYP$kbMVcTr+pfh!f4OeCx|+QvT|cxaz9&PHLsDohk`Q>6R9L_xi0`(k|&d z>1OS|8BAJyQY)Q$QsVYi)_O$g|JM40?#+Bpr`6dnHEojg6G$g zn3xT7uB{et>nk=rJ4O5dnxFC~2Op&^?SlkEOZ&Cw^7HST@qFi$gr!+0KFWXNOef7$ zf_2nj_Lok_dtLZSzt0-ghVTh<<4N5fxl z-*wxuUlK1~-|z6rygygg8nY~}%c)VHZ)wy$^`p_%X2$I3w6fOt!j#+-y+$Q&OkL7G zVc%OhZ|9vok$qdg+xmanWJHyoujVH{^83+IBTqdsF>PGYoOL%X-rVkiqO@9n?K(at zA-2+M3)Zar==he!vwuI?enQ2U7dLD^b$aoX*6SNwShL&mL)VUrsXXRikFMLlzs%RvbSrT{XnP0Z#)EfWH-qAnAc9?tKzO!dvuQloM zhN(xMuN7Bge9xJS4t7d>`OT9&ad*dz+t^``h*@ z%WvxR?cR?LZeHA>xG1|p$EDlG-uh0p%!bW!cO7k3`|$ZUymt9zv+lg$y+%g{ExT{y z!#SCKyXNltxPOfvH6NQh;oG^@_x|sjF8lgsZLCoD*~SwaCH`|!%A{Kx=QcXC>&@SO znD^w_7kb@LZQz_~jn?n}`^UWTD?0CbG3j7T%$eh#953zCLew?$mj|j1Te1GPhE<2J zczYpktN(19ci)5S2Fx8b^LWEc-gxYA_NpdbKCAe~cekA`%3XUVZ|1v0 zCZ}$yamkS*+iJekY0tU}`49asw^PEJJ@r4y8k-(JW#9#~E7Vdy+TlNH9C+~h9<%P;eP80cgQrejIc@g?PY=Ck_ulCzuI_umz}0WJTEDC1 zlv*ceKb)8;*M`9Qn<@u(wY@m`@C6qXp18Toj%H8h z9=LmAg9e?_p1;5EQ~Uq;s`)!x20m3~{bgCF8%&-2;mKRJ?HMrO#pB#Ti58G84n+-_w2`sKlC3lfAoZj6E_|ys&q@OyT?`Pe7aGM`(J(k@5KH; zeV;M&i7)3?9JQ%ltIDmGR9w;UOyAchy#C*;q=mn~Q?cvr>8XViX15>K_Ux#p1z&Z# zd)DOz-IkAT68LEEeJ`&&XKw4kL%L-TN$z~&lZ|FZ}X20Emzm( zMoE)HUwY+-G~Soqr=!-TSMRzza8HewDt!LU$4zdU)hQ#bQN5R%<~)9U&lSH;OlJey0z9>-X^Io7U~0Klt_U?map3<*Qq)JpRF~FW>v5 zFtg#jOIHrB6Z3Av%<9jLy+!&6y>6&deaBnPw{6&xPP3-_`rR$>Bv~Dooq5 zbn>9PF0YXL@SLv8UamH>Rf7la=)3l-#{YS@=+M#S?c$c~nx}n*Re9#((Urch_GR43 zdC&awvE+K+JX$!Y@5+TUyZl}8%7ibj&YV^yrN*s8j{M{I;+b)^dk={j6nE^_E4vRG z(!W*4+CAfPkMDc`jq~EF?W~_Ox$mgLrk(OD9m$MM>$bYW(koZKwd~ZfwJqMM{#3hm zeQ!Q{MdbML*GoIC@oo=Pi z&e1V3w=UbUW$(#-u?w!Q`@)f$Gqx2iOx)9XYtyf$Kk;?nL+L})5?<@@&A*qgIQVtL zYiF(5@n^#aKkR(&z$1@!p4I8dikyq8biH}R-ddA47w^1pdG5NDeQVyxo^|4I>a0}< z*LOL+zv8HempAG9?7W<=t2b@>@aUlbJo(MbIf?!L-T1o}uRPg#*O)IqZ!R>&$QTpBd-(zjxR6C#xSFuRB4%dAE7Y;hII=IxgKZKBj~A zU9tX|jSXgX*g7(2+~&Up2;b-j;WF!TD}r{A4<=!O}~ zA81}>--zzbXEsSK_-yOklDXmYbNZ@#-dT6wmPh)H|Ne`S*%R(~;g|RG&n;|!{fy=N zZ)m*ZvRCpZ-Bs(@<%PNXX6{a||NOyiCADjex_DaK^$kW`J#*iK_ZJV{d2#M>t=APk zn^HNiWcDBTWnc6Dz~jU3uYTXg7vG6c8Ti7Pqf=%)>?W_2Hy2kr-7xu;;(K~c{QBH;-nwdbYDw7*ZRUx3moFSr zlGb7Jv=zNR*?aV8y;ZL!6$TEBuheGgC23MHAcv$tNS5 zy`;@wHhuY!+}2&!Jlt~qp7t4u3kvJ*tv2e3VH*y8cTT;kM}HbTyz=9povOL$satA| zX;8i48-ee)G@rTh(@Oi!U!=)v&5zqYDF2oQerIUEk1Z@%(+BXKq5Uov+a@FUol*AW z&>v&#ojRvmwJ{~dgNF7iDH=3%Oj6^Rr0Pv#Q|k09EH2J3D(E+;c=+(55^ZvLK@;uc zCpcHvG&iPT)QIBH%NA7{ zUOb}Aa@F8OVbPGnn55ui#bL$c%a&9OBhwb-4=5=OFSw-Apkc*hG-stdi$@lWC|{pH zAQ(yV#tQWs)H_u>EuleD^?GgN;uGrGl9d#vtxSF!0^q@@h*A6V3D zp?!tMCB+86XoG)ZV$%MIEyWX`5L3@SL1lnmX_h>@lD4>GJLG?aQLlmKO=CLf?PTSM zH=B%?`P$hy`OySzY0ww!{ZH>?POS*V0PU1Js*t=eHf>I=j{0AH_0Jlypu2+&l}pZv z__d0%%;m=TNN?9w!Fy8=v`7H`LX!4CG|QY z{F1Ae&c+LZDQps_6{prO%53h7Pu@{6ZQ}(&IhuYRbokBC2J}-GmKrYwUDkoh9DGnc zc|m;IQx^su=zj^+16$dwK~iiPu+nx_wsU%gCBXU%lcq15(HMQ4^V)^2 z23xM)=)88gMz?kf6Cdo}WkxqQPM(~QwzU&(m)Uw%n8ea!pxkPOn` zG%UZk4;)rpJiMQlV{KW#V0~y;`TT;Es$KhB8|cwJw_oq{n3VI1M-OQ=VB`R;Dq9tf z%`a+Ix~XhW%A;@8zcl!-@qs{~OJVxW+cVH#qj1cjX$!@*9wbhtM<+5tp(H}Di07WRZj zEtR`AYOIdQ5P}Fb0>bq!S8_t@l()+!a4kGdK%SzIaz@yiD=k3SBMzsi9{9U(5lr8aV4EI>lQYP&N7 zlBn56S32C}U0Y`i`w`p{uuY2VM*2ar2^xB2^b<| zSBmWJfaSfEN{=+5q+-)RuzI1zfG=`WrZ>z`B4;Y%bpkR4Y^*{$7qBeCr?YcYlW<;K zK8v#l%CIzx2-ITGzv0Cio&Sln#_YclxGcIzK}uRgV`g}JHYAOp+_k(hLa6`d&m{oC?Nv4(1mXZ?#TD z#<&Z_HVpt2-eMjQ1l(}w_GT9{3T1HKa96>U3$AUfeSD~Ky2{E!Yub=vK&In6?P*+}JMk@*o4<;?iNw8ce?uTJ^Z0irxo9 ztZ0nNKX(S?>3GOhZRr0jiIb>dvJ6u;+p1&9mJuvGs(K8xS+6{X>9QAjx^e^@a(|9& z1wF|aCI~B)@u&-!3Tw&;3M7c2%2+(VM?!NrI!mhd4jbTa`hdPd~9O21Y zQ~0>(7Y%)jjww--K4(Dc-&**Bv+KeXvQM+SI@`L!c^W4ch?@|OSvGqh`ZUB6HrRkx zWnvhhADJKmd32r7rC#^wx(#jteT#N}{gDW}$hK$R$ zQ^)}6Ir*e{wmbMyx(Jfv&GzY#Ry9SBf%4+zRFL#=d)bmS1u3dO^7ZR!ZSw^m9oNNB z@1QF~jB8K`2!p>Ae!hc~9$gZhnE?6*_EcZ0)f(W91oUW_pl8WoR*o0irEkdtN|+KbO<_WSS)}d?5^pe>9xZ|#IV{do&ju? zLqcbD9Plp&3LY^{pcNjb6Ucm$E<=iim3T;lQ2V{afG8Eg`xREQc-{`LY^epQxZ;v3U1yG&Q$eXvb&J8`1yttu)bA;D^;cN?9r7Ber@ye}d z84y)7fWuZKkLVpy!XQKv$jG-Si82Z&WY(u)g~F5QuiMv`$^@Te>Ds0K=M%T|pn#L) zMy?MJ9zHsF9i#AWSgk}qKdB*+qk>&f;E947M|wk4N)BDFdI-_5$`D7Bc9G6T%Y25;?NN76siJCNEW7Ji7Cr$^pxkb;{Ot!GkS_1<#xxautB1 zu+BunarGNnYA^#BHmoUs!Ecpe`t8iBv1n(&RlAZBrBfN?0TFF^_1}ab+D1npCUJH!SxOrw-pAn)SBe{E-9d(o_Z&0}?SH zdygT%XPS;9S64jd>6pjJf`=EIy=wl{-mL`#aensOtPtKAzEZDUwnXAGZtjdg%PxVss0;q8! zhC>3k6F-q*tj+XL(6A&^^OW}p#7Z>crbs5tHGsy8xfIcMBZ6kT+#Cv9ZxGf)oy4$I zI+THcElvYy^%9*75Ojk`gYq18=6bL@IVCJ(D4q?ch2aHiyWJu)U3P11#=Od)O(>Lx zFp0Eyr;}ETSxN&0J%thiKbTR~gPldG%EPG|h;l4dl-RlQ*n`e13r z^x6q!usVnEB`4Obcm_&GyLNsiSdg=1_3Uj0lzr6jInlUX6uHI$=p-whmI|ux*+d zPGwjzpcsH-K(b=xdWnK#NXk|II8dlk8@`hY;wF4LhJnC@M@jJVU5vtXz_7q{4Ra(& z31=EAf^k7rf`HJIOPPyJCRWUs2lG2ctPD+4M9ImNw6z;N&7=j#DLK5|$Wa+8>yf-u zy3<7rQhSQ7Ecxy%mZQ`W)aXsU`9(LjanNZ_-b^fla@j0}`ci~g5OA2H%YY}W$yL_!=f<|L%SECmbf-YtGvF*G zu}#$;aTJew>jo!Jq6G40+rQ;uhBp#dI=FTm`Hz3D`JqCG-GM^IFf}T;N{kw69~-ih zj8DUL4prbek+MAte*KXKzNFF2}mq&ek__j;TKj<)Jj2J zZ-y%1-P4j`4IW=QoHOYvJm+f(8Coi~340$|1fkvPdKO|r#9wF~*{hvF@}Ln>rwiG$ z#t8!g7#XEb7luJ#ZwF3bIH9$LNoqB4x&xF)*gAK#n33Y#FbCrV3c0e&jpOb!V?%kT zP7Mmgu6Q`i#F)*I|zaK~*&HApz z&<#kQMB!hU*7qBhJRde*A(Xt57vWOr>_@u?GtsVKgO$3qe z$TnmEGCI@gU@#%;3v8Q`6+aO!_y&gMCQ=?)+nqL1GEK>h6H?bXq$EVOTwSTeU*N^o zg&!^CPWaCsNQ2&7CRNRqb}SF5ReWa^kAPh6kQXmJM@R=x#DxdM7Xi!ksXRNXO)_O! zHsefdk~~;x;UNH%>ck~a38@8kba=gWVIEKFjtDG{2R&K%80 zRtTI*%xNwZo4PKBnDE(OW@m@$b3RiFzi?n05 zVoqiByu)OS9UERba741?6^=|Q5lNW|3Z+*8jmvtZP>RJWxhO*-uayVEo2WC3B0FOvws)KW|a!1BQ~sJ3~S0af2} zCXYcY#|{+rP{wwoHYx_FF`&*lYE&viZVb>nM!}Ytwqd|8w{CG;=b56kx}!fL)5%Lc zIzCO2r5CVm@CWI6XrGAZ=s~|yR>J~>4lMiTI0rmMI@F1V4i^!Za$snLqfl$IX>cZ z9JW7@Lj_m>M4-cEtP_=V0Fk<3BV)P=r$>tI5zD_5HXqFa^v)wf!AC|!6?AJD%&6^* zU=|oBIzn=(Ba)&f*-JM{-C;mv*x#uqqQ-8szI=4L(mmSk3Itpi*l$y#-eLf;n&>ut z8>*D-Y)J9W5Uuoq4h!b`<4kq9J4XTzKlv?I&eSS*oEShnLO7vPZWIG12ExtOG%oEc ztI-#)whkM-7rUN-D76gB)WSgWg1q zajQr3nQUDF9&v<)pIop*$QkG)0jd$wjfiEAl%oPN<{py_Sh8W)nS!vbXdD)B_L#89 z^@^Dz;5j-Vu;WU4ug*cswMlLtc}7_=C+pjU=rJKBpK55}t+nmyFjcK%Uq$hRIs zix5lXi^~}imT+xaEl>S9N5p#;GIE|pa|QIja}p~3kP%vlw~>iujXVRmQKCe^Qv`6h zU|Jf?8aQvfR~bH|P@RjEuY5v5We;&Z+km0koEu_`5qfpOQ)-ql+~QXAih*(l=qX+{ zN6G#HwIe$~`gzemymaZip5Zeb66W31Qw5Ev$+fAd1wLM^RBli(g+K ztmR3h+;4~k&q`N93(4i!laer2i7?t_krv8wHUdU}6AH)W>g+O&9<16Sf&p%6pKaZ*VM`Bfo&BvS|o{`yaj zn+ko(2rL8YyoY6q>bh`sQnjj#!82gn)3{dycZcu<$-5GCqFHVTu2p8TWHBHs2Xv2F z+pv`~sV3i+W2zp)#!AIuQg(Qf&x2U+d}7plajr+(qej0yTA$evL)^KXvl*?)f{fcs zrH`T%^0*vjMQ$E4KYm%}dbhI=lugSq`eNe4-Dr>z_fq z3b+j>L=`hPzf{HI*cT1FM#+nN4H${gr=4nMjBX&`BIuo~Q8ZFLYmlK~FLP!BU^dw3 zD3CbYY;2%Ro!|ae$oA4TO4iZ(iw$}=0 z7Zhj3dZ!o&dsDOo!XBvz50X~32r5MRbP=e_G;h=pa8OPa4EW)*p0OVcXle^P2JDZ# zuo&1mjta~6?7D{@Rs08gpSKQ7f44}tv#2NNQg0r2Gvr91I|x%SelHI8l43GQhB3P7P@%7Co;5Q@uu)iu8}%?KUi9)wIMaGezv7X-^KMCVV6N87@&9h3sD_dsCW znP9#IVb*w$EUt*DQRt0wk-imiO}=u*i2>@eqgJ+@2%zj{h~q!PVR46{9DoPSQP;2s zIq(Q%Z=A(alIic>fpHJ#mL-|7aU;R%h-xiI(=Qy+6suJ>fDvihn@rX2x68DTZfa~p zq1syk6(@4oZ^8z0iJeIDL&9J|rmhAgE8xbKAtZY6%If~ZJoL=^fE_HqmaJcPmzn$SNJm~`FB!4Fm= zjmFE83^6_%Rs=H1hZdGd%#?aA#olESP2`ml)QCYJ zJ<;KYpo$>48tAs=*&&NY zr7dE4QiXy9U7WHA0VPVc)(;G*%d;P{2#CSvv)*P1uo*b)b~XbystF%6z~>I3v`1A! ze3C{LvbT(>i=DUf<#)8M1&{pgLP_Bi?j?TDoj$%?ZvVD3y;rH8k(KI>-XcNDgn36w0eeOUlk~~n)!E?PxF^`DS#ft!^<+voUC1R^*nvQH| z8WA4xl>?_WaD913H*kTfRnjuRQfYWa?Dd|{Ce?0Svdkl!h$vbGH&TjDbf)achPorW zWl`sY=uzq#vSW`ZTBe(!^2!DQ4OvUA1U9tEVzpf7-jX{_^7xbvz8ChjVfHX2;C=$Q zuh8cx*dx$(=yy|nHU~@9IF>OQ4HV0<+N2l&|MN4;iGW)<*sTw*FcA6-o&#Tq27x8o z@%oT;h|&;9IPC<|v-YiQ0;4~YAV0C>7_VYovMehhOtY0GbnK6^V;HVQrW)Szr2P+RC z57-lkt&xl#Y?RI$T|fat#Lbp78tSwB#6hQufqdIDL&vGTZ9bzAH*|<4C|D_*Lvii2 z)};`gwmQuTQ;0CfPHcE|PLO2z_jv3;u{VMg!76sJUGNMDqZ(((0CYcwUT6l3c-EV@ zi+cAcY*BO(`^DCmi)VmcjB{Hl!i8$dr!i7)j7U3u@jkSuAKOM;>WvozfM%k1uewBt z4dQwW8S>{}2rc4n6CmrKR%XPE5aomH01^{0EwA{*VH}+r=xF2G zrQP|X`ody#SCjk!ShdBb?USAToUCo z&Lj>8RO!%=g`DD1EkgU0son-9St=)3sV7A^6yZV^(aW+B_?lCtF7C_=ImyERU95t< z(_z`u;7t^8xm$2?5(&&t`H4j3@hAIx94IJ>4g}i|_z-T+$_}|T{|zf%5J4!7fCPIJRv|&_IXcxD?a<^u|~c)ixB4v>{MU6pR0B4@ZvaqF2^X8IUPCqHhHHdgpvrhz=c9 zHZbeuzJ9_);GH$-r;~3KCsPOGEg1B)J0RrT@*I$0Pw6{2wh`owEJ{gCB;n9u-LnYY zC~D(d7$D38Y!EQE7_N~ACSoh4kbHvd0y2UK3Nd7WQ|$lN7WsQF2ZXe8089?W34%*4 zeXUcB3(pjd%%K%5xg1d~nAeA5Hjm>m=>1(GKD-}^ionVMLBF#aa~cVj6E|m0KumLG z?MX&tCZG_nXi;F?OUZhHG#wQt445?#DG4o+CiV~@^|zb>Vb?Y#3H}l6mAC zBFvEkvj!n%%H|L}tEG?(pndZ}8g*O7SNLvdpynA#8Oba`u8{mt`p;TKfy+w~Wqzoh z=ajFXP-^tkBC5&o6EP|lwhZvFeO=(9P^T8cAY%4uv6veYdMI{Uh9T^PTth7N$P(Hu!6RX_g(+SpsK*WJdh`EPo}Xj{ z(fVBF1J4IC^RB*a7l@anEi4ZY3OIf+*dmY@yoNDw`F4xztq4{()aa*X$hSEQ2S%gy zhmAo$gcAhUq%9>)5j7kFOJv~)EHi`yn0N@idUZQ&Yw>D)(72iD5gGycLT^dm(v-*v zP!<#e>=-B&tnRXg*D~;S(TU1mCC8IVe!l#bBd~5Iym|0TGME+*%m}(>2rdi>^~zQ& zWPIz+NgJ4`wwCr+j2-L5w^CyGbreI^A15mkjkk| zgq|aWH3X-U_C(X292O<)Is}5Ee5olBtl8jr<0B0P@xZVpGBn7z&3U^lAc>|OWGFYJ zIsxp5R667`fD<)LlL0_3AoU4C!npSB-1_MLViz8#F#(C=6d6l5gw+BMEt*75IabQ% z+;hfF6b?|3%NcotgL6rzx<~$V?Py=GFeBzJO-!6rwnc%$sPF_f0=UG>&~Lz?)eKz* z0QYvfPGq#kDsVtm*-{W*i`&d__t~ORgGdZ0?vdz{Xt5;MHHjFJ9}2i#gJ+mWh3?vT zxeE7x=17$T3n zVq=~`tK6)6W}fhnrV^*fvZfLMMB?nIh1KYXXH0-IMp}=yDnA&x~5SsYLVs4EFc=#`{ zs>Sy}Z>K$0M#R{{wnt#Q)Wd_~#ZG}dp0Y5*<=XJf&VBk$lux3~VM6Ix37Zr0;3iGC zc_Ef3X`ebs1*+jRz(UD!Z*Lpgn3NB=fn_-&4I1|AsgPRkd>Z+$|k82g>Mj* z5G|lOrQW^6T?D{Icpeq0T2I74S&L)EJ|Njdh}C89b8fTy|F zY|H%}fjTq_++EXI!48#gdyhmH8#)@0+zLrIU=tcb)$0MUt2MnB_h-LVrYyX=GcV~1 z%#P*-9VuU;R4!8ZPkXG$ynT-l6 z4Y=Gf=Y&)AkvP0?&p^{!BsR2tWvj_{<9Jki5E)?F!x6DSE{E$bkj7+LCdrZRN{8Gy zDWfwgO?5L@b*Kg67(kJy2fMyFVwrhH$ru9=#_s0qKncc~AG}!<0ia|H77dDjSew9g z?h;}VA`lMF1g8&59BAzJ*yPP-h@F2gTGlC9jAkfbgfVg;CT?_~KGiZI{hnZj>+uDl zEM3(@tj1hInAosa|NT}O4o$8H&aGIsqL3>YVKjnh;?g3*L5g_bEO!whhJBL6DoB9k z*pa*=Ee?}|_`s?%Xq}ht;lM}0g;00Er7N)c@zn(an*~Ug)iQY)fUi?|Ftb8}!3FY` zaH9x6lt*KJhY1pA4~$*4M#lhhigkOv@Q8J9BRL{?9QZtZ7?VrFh|!X_-aPT+7&4{Y zqQa^^O6?Y4BY#wI*aPuJDKBvGO8Idj$1;~~G zaRp@yM-5VBfYD(=5uq{#V8E&G0=8fc^YQ}&j&>_j=MxO;Y7K4ODMl3o(ZT??-)>+D zbpk|Q=uXyQ5rWpK;|jzK$8l+Y_<-erT;m2$827^<0*q}2x6m&%!$r?Mw=gw6czw*# z_Cqs6(6SJ?P;29+kWH@?vGLR&Mc7ejO0NO*K4T>eI`g4&cHIDZwF%j?LW=Sc-SthM zY&=%%Sk^hfGitL7GU+xuqjQKnv(d?$XQHOjo9P4#XxKy*OSG9zSj1KUypY*tVw4P$ z+RcUmW~pXzKC_{~3swo(Tw4hX#YiypWxIHB%{j`Xme zq$MMFoI>l#9iCdjmH}^^07|56RiIht*KwGK5^>;f-MHUIB24)1Nl0pG+{y%G4Fk5e z#hzhb_6a$w10!vC;_}8NK+YsVp#oM@WVcsyfFaUI(6~?btqIo}+CGO8Gk!E{AVuOt zi^@N%jY-hQCF{|Jfy+UJ`iMRItS=mT9R`DFM+POx0$A;i$kn7@8L-|XAnJ#niu8r+ zSBimg8EItI9ip0IAoo!{0a4=e^J9BDTiY2R5Zm2=wO$vuL(zxNIU^=LWI$+J;FBb1eT6sy|CPJrpwtIA zMoKd@zsiM*loB7vW{9&%QTNE;p*J{!l)K!ZV-&iOW*xE$gwHuZjNXMxmuX`6)D3uq zcIt5}gNgw{28a*ma{rFUA0b8|+g*q;PWvu;A(V@As_tjvxL2MD7*LIefIRZ8{J!VS zcXH{kyn+By=;XL2j8_6^aba$Qw(qtOH(WQnQ1iCV&02@Dq!uX#_%a}g5X^wSDj-i8!H7AyN9-)_h&j7OY7Jc*QtN0JP|qPQjH9P*=*UIRrh9&t1ic@|M)o;kR6Ky{B8 zd`m7Teie$rop}r3lp+x_z@*tTmv?$Sj0;@vI#ER9=8mM)aDCeCBW_zbZ5bdNXDs># zR$zAuc2@uce)_~PE=RJT~Xt>Y|0ggzA>1Rrhh^lpp>Ct0U&kziFo_iD}a%z$gw0mla$(V7Yx{S z4QhH1GbBZW45J}_pidAnq^L8%__{5fVjVSyeXuxMrLQSQtTNUDG!KlzgZ58zq20ki z)4sArj{)rRrw4(mC9w9uGgJX6V|EPKkRKkxYkjgbZo6ptjRDFxG#vWN50EfVAw4h& zIb<9vI_SmW3V<@pY!P1l!NFmY4LH|+Tjlp@xtRtYkz8!}?RAu5yMgcRyBopanCBO&K5SI5ys0CbOSV2sPH5fz#PRmIO9QrVE0>EXCMcGerDADZE;u#<4uBe@fvq5JwkEx9{MO=g*+Cve5(pRW2shoWX@S_qYKiYe&x$B4yO2E>(zZIvu!e3q)zzsGA(`#M>6K^*_&sH!OI6AjvqlVOp)q*$aFWjw`+Lvx#F6YEm(vF+i*E z6qyBDkq5^Ctu;48qQ{CHp8LoU84d-JO~ zrYZ6&Y2zgUZoJD-PgUr;68bALYMYw_^>~rvGqf0EGLUWgbc6ULMTljtV_78tyww3G zBBc5L$xy^nN9&v|nJP_c8WlJJ44A5{r?Zv?0K|7`b2W(=GsPX5KzIrsE6mR>1Uz0@ z+C4ynea6u`st3lmQiho&zn_GndZ$vFrZm5c%Un#BcTwuI08s;q0d@>{;Od|cUe=!; z^cc&YJw~iG0EtJ4FO%WefXAGFE&;$0!zB3P!8@IS65X0YRga;>CBO{3(qU)W6bcnb z{%M9Lc0Bv4zD_)wItARn3`9wz^l_N9LQ_{W39DQsMQ~E=uPyeYf}zrOkNBGfBqxx| zXas{8fYBw_uX9kDvm=YjSC)}6!EYcAU0|`J*u^p1X-ilJJ^L^FWo%N6i>P_9%`JWF zVuHo2Smf|6wSa5^VS^Yd5df;Cih)uFlv0%t6a$hn;8L8S+nmcXNrOX#MfaW%6_g<| zOlLXv?VHw5m}Frf3IiE5T^UDW4fTS^2<0c{4k+VQ=Hz97&$pOTEJX$INf`oQ>~qG= z2dfV4=2zT2@*zeEs!M?qz(PF5Da8{u!%UlD56mBY5CU_KFdQgaF-VhYi{!_HAb145 z8Y;Vv_Jc7yE`V}jw}}5aBHca9jux~rX?q^aTJd2X(Xx)RnoaSdAn^KH8RX`2ou@)E z?KnvrytKUypKUJ}BWXJES;O5a0oy$47MC8ij}rrUjZ8_0Q+lG^4e+WCoS^fc1rj_^ zj4oUow)f7UXcLz|yOygkqb`6}k`1Wvu9?mbB%C$GjE9f;H3B7v zcp)&^#R9fJRq4;Dn23soQKVZXNp9fw!mXkac$p0fEsq))G%U1#Q>GS1nW5l8fSgP; z24E4hv1;@V_5nviuIYQ-Xg%}>K3#QcK&t4FWv+k0WO0QdpEGcN<-@8F0T__K{s6!Q z4O{twYng)JlDWleeO8LdP%zxh;zNfaTo}{3vlV8!T4QFwuAcD73iftcWQP?iR8yPd>9d)A{QADlfnC#Oy8f$a(g zw`ON*`D-Fh8SNy;HRt#KfTfznEGJ<74sy*Ju1ctsCuk literal 0 HcmV?d00001 From 08a4737cd6ad4214654cffcdb4f0cb51ee9d5c12 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Mon, 20 Dec 2021 19:48:41 +0530 Subject: [PATCH 08/26] Fix partition column and apply filter --- .../parquet/ParquetPageSourceFactory.java | 26 +++- .../io/trino/plugin/hudi/HudiMetadata.java | 21 +++ .../io/trino/plugin/hudi/HudiPageSource.java | 127 ++++++++++++++++++ .../plugin/hudi/HudiPageSourceProvider.java | 56 +++++++- .../trino/plugin/hudi/HudiSplitManager.java | 20 +-- .../io/trino/plugin/hudi/HudiTableHandle.java | 14 ++ .../java/io/trino/plugin/hudi/HudiUtil.java | 97 +++++++++++++ 7 files changed, 341 insertions(+), 20 deletions(-) create mode 100644 plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSource.java diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java index 033050ce73d..dc607cdfc25 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import io.airlift.log.Logger; import io.trino.parquet.Field; import io.trino.parquet.ParquetCorruptionException; import io.trino.parquet.ParquetDataSource; @@ -104,6 +105,7 @@ public class ParquetPageSourceFactory implements HivePageSourceFactory { + private static final Logger log = Logger.get(ParquetPageSourceFactory.class); /** * If this object is passed as one of the columns for {@code createPageSource}, * it will be populated as an additional column containing the index of each @@ -199,6 +201,7 @@ public static ReaderPageSource createPageSource( { // Ignore predicates on partial columns for now. effectivePredicate = effectivePredicate.filter((column, domain) -> column.isBaseColumn()); + log.warn(">>> Creating Parquet Page Source with columns: %s, predicate: %s", columns, effectivePredicate); MessageType fileSchema; MessageType requestedSchema; @@ -213,6 +216,7 @@ public static ReaderPageSource createPageSource( ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource); FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); fileSchema = fileMetaData.getSchema(); + log.warn(">>> File Schema: " + fileSchema.toString()); Optional message = projectSufficientColumns(columns) .map(projection -> projection.get().stream() @@ -227,12 +231,14 @@ public static ReaderPageSource createPageSource( .reduce(MessageType::union); requestedSchema = message.orElse(new MessageType(fileSchema.getName(), ImmutableList.of())); + log.warn(">>> Requested Schema: " + requestedSchema); messageColumn = getColumnIO(fileSchema, requestedSchema); - + log.warn(">>> Ignore stats: " + options.isIgnoreStatistics()); Map, RichColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, requestedSchema); TupleDomain parquetTupleDomain = options.isIgnoreStatistics() ? TupleDomain.all() : getParquetTupleDomain(descriptorsByPath, effectivePredicate, fileSchema, useColumnNames); + log.warn(">>> ParquetTupleDomain: %s", parquetTupleDomain.toString()); Predicate parquetPredicate = buildPredicate(requestedSchema, parquetTupleDomain, descriptorsByPath, timeZone); @@ -251,6 +257,7 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq } nextStart += block.getRowCount(); } + log.warn("Message Column: %s, Predicate: %s, ColumnIndexes: %s", messageColumn, parquetPredicate.toString(), columnIndexes.build()); parquetReader = new ParquetReader( Optional.ofNullable(fileMetaData.getCreatedBy()), messageColumn, @@ -290,10 +297,12 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq Optional readerProjections = projectBaseColumns(columns); List baseColumns = readerProjections.map(projection -> - projection.get().stream() - .map(HiveColumnHandle.class::cast) - .collect(toUnmodifiableList())) + projection.get().stream() + .map(HiveColumnHandle.class::cast) + .collect(toUnmodifiableList())) .orElse(columns); + log.warn("Reader columns: %s", readerProjections.orElse(null)); + log.warn("Base columns: %s", baseColumns); for (HiveColumnHandle column : baseColumns) { checkArgument(column == PARQUET_ROW_INDEX_COLUMN || column.getColumnType() == REGULAR, "column type must be REGULAR: %s", column); @@ -317,6 +326,10 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq } } + log.warn("Trino types: %s", trinoTypes.build()); + log.warn("Internal fields: %s", internalFields.build()); + log.warn("Row index: %s", rowIndexColumns.build()); + ConnectorPageSource parquetPageSource = new ParquetPageSource( parquetReader, trinoTypes.build(), @@ -412,6 +425,8 @@ public static TupleDomain getParquetTupleDomain( return TupleDomain.none(); } + descriptorsByPath.forEach((key, value) -> log.warn(">>> descriptorsByPath KEY: %s, VALUE: %s", key, value.toString())); + ImmutableMap.Builder predicate = ImmutableMap.builder(); for (Entry entry : effectivePredicate.getDomains().get().entrySet()) { HiveColumnHandle columnHandle = entry.getKey(); @@ -423,12 +438,14 @@ public static TupleDomain getParquetTupleDomain( RichColumnDescriptor descriptor; if (useColumnNames) { descriptor = descriptorsByPath.get(ImmutableList.of(columnHandle.getName())); + log.warn(">>> Descriptor: %s, Column Handle: %s", descriptor, columnHandle); } else { org.apache.parquet.schema.Type parquetField = getParquetType(columnHandle, fileSchema, false); if (parquetField == null || !parquetField.isPrimitive()) { // Parquet file has fewer column than partition // Or the field is a complex type + log.warn(String.valueOf(">>> Parquet file has fewer column than partition: " + parquetField == null)); continue; } descriptor = descriptorsByPath.get(ImmutableList.of(parquetField.getName())); @@ -437,6 +454,7 @@ public static TupleDomain getParquetTupleDomain( predicate.put(descriptor, entry.getValue()); } } + log.warn(">>> Predicates: " + predicate.build()); return TupleDomain.withColumnDomains(predicate.build()); } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java index 705fec13987..35802c39647 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadata.java @@ -33,6 +33,8 @@ import io.trino.spi.connector.ConnectorTableHandle; import io.trino.spi.connector.ConnectorTableMetadata; import io.trino.spi.connector.ConnectorTableProperties; +import io.trino.spi.connector.Constraint; +import io.trino.spi.connector.ConstraintApplicationResult; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.connector.TableNotFoundException; import io.trino.spi.predicate.TupleDomain; @@ -64,6 +66,7 @@ import static io.trino.plugin.hive.util.HiveUtil.hiveColumnHandles; import static io.trino.plugin.hive.util.HiveUtil.isHiveSystemSchema; import static io.trino.plugin.hudi.HudiErrorCode.HUDI_UNKNOWN_TABLE_TYPE; +import static io.trino.plugin.hudi.HudiUtil.splitPredicate; import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static java.util.function.Function.identity; @@ -123,6 +126,24 @@ public ConnectorTableMetadata getTableMetadata(ConnectorSession session, Connect return getTableMetadata(session, hudiTableHandle.getSchemaTableName()); } + @Override + public Optional> applyFilter(ConnectorSession session, ConnectorTableHandle tableHandle, Constraint constraint) + { + HudiTableHandle handle = (HudiTableHandle) tableHandle; + if (constraint.getSummary().equals(handle.getPredicate())) { + return Optional.empty(); + } + + List> predicate = splitPredicate(constraint.getSummary()); + TupleDomain unenforcedPredicate = predicate.get(1); + HudiTableHandle newHudiTableHandle = handle.withPredicate(constraint.getSummary().transformKeys(HiveColumnHandle.class::cast)); + + return Optional.of(new ConstraintApplicationResult<>( + newHudiTableHandle, + unenforcedPredicate, + false)); + } + private ConnectorTableMetadata getTableMetadata(ConnectorSession session, SchemaTableName tableName) { Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSource.java new file mode 100644 index 00000000000..e6ed5096baa --- /dev/null +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSource.java @@ -0,0 +1,127 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.spi.Page; +import io.trino.spi.TrinoException; +import io.trino.spi.block.Block; +import io.trino.spi.block.RunLengthEncodedBlock; +import io.trino.spi.connector.ConnectorPageSource; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_BAD_DATA; +import static java.util.Objects.requireNonNull; + +public class HudiPageSource + implements ConnectorPageSource +{ + private final List columnHandles; + private final ConnectorPageSource pageSource; + private final Map partitionBlocks; + + public HudiPageSource( + List columnHandles, + Map partitionBlocks, + ConnectorPageSource pageSource) + { + this.columnHandles = requireNonNull(columnHandles, "columnHandles is null"); + this.pageSource = requireNonNull(pageSource, "pageSource is null"); + this.partitionBlocks = requireNonNull(partitionBlocks, "partitionBlocks is null"); + } + + @Override + public long getCompletedBytes() + { + return pageSource.getCompletedBytes(); + } + + @Override + public long getReadTimeNanos() + { + return pageSource.getReadTimeNanos(); + } + + @Override + public boolean isFinished() + { + return pageSource.isFinished(); + } + + @Override + public Page getNextPage() + { + try { + Page page = pageSource.getNextPage(); + if (page == null) { + return null; + } + int positionCount = page.getPositionCount(); + + int dataColumnIndex = 0; + int columnIndex = 0; + Block[] blocksWithPartitionColumns = new Block[columnHandles.size()]; + for (HiveColumnHandle columnHandle : columnHandles) { + if (columnHandle.isPartitionKey()) { + Block partitionValue = partitionBlocks.get(columnHandle.getName()); + blocksWithPartitionColumns[columnIndex++] = new RunLengthEncodedBlock(partitionValue, positionCount); + } + else { + blocksWithPartitionColumns[columnIndex++] = (page.getBlock(dataColumnIndex)); + dataColumnIndex++; + } + } + return new Page(positionCount, blocksWithPartitionColumns); + } + catch (TrinoException e) { + closeWithSuppression(e); + throw e; + } + catch (RuntimeException e) { + closeWithSuppression(e); + throw new TrinoException(HUDI_BAD_DATA, e); + } + } + + @Override + public long getSystemMemoryUsage() + { + return pageSource.getSystemMemoryUsage(); + } + + @Override + public void close() + throws IOException + { + pageSource.close(); + } + + private void closeWithSuppression(Throwable throwable) + { + requireNonNull(throwable, "throwable is null"); + try { + close(); + } + catch (Exception e) { + // Self-suppression not permitted + if (e != throwable) { + throwable.addSuppressed(e); + } + } + } +} diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java index db0368cb2e2..fb978ccdc61 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java @@ -35,6 +35,7 @@ import io.trino.plugin.hive.parquet.ParquetPageSource; import io.trino.plugin.hive.parquet.ParquetReaderConfig; import io.trino.spi.TrinoException; +import io.trino.spi.block.Block; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorPageSource; import io.trino.spi.connector.ConnectorPageSourceProvider; @@ -44,6 +45,7 @@ import io.trino.spi.connector.ConnectorTransactionHandle; import io.trino.spi.connector.DynamicFilter; import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.predicate.Utils; import io.trino.spi.security.ConnectorIdentity; import io.trino.spi.type.Type; import org.apache.hadoop.conf.Configuration; @@ -67,7 +69,9 @@ import java.util.Map; import java.util.Optional; import java.util.TimeZone; +import java.util.stream.Collectors; +import static com.google.common.base.Preconditions.checkArgument; import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; import static io.trino.parquet.ParquetTypeUtils.getColumnIO; import static io.trino.parquet.ParquetTypeUtils.getDescriptors; @@ -86,9 +90,11 @@ import static io.trino.plugin.hudi.HudiErrorCode.HUDI_BAD_DATA; import static io.trino.plugin.hudi.HudiErrorCode.HUDI_CANNOT_OPEN_SPLIT; import static io.trino.plugin.hudi.HudiErrorCode.HUDI_MISSING_DATA; +import static io.trino.plugin.hudi.HudiUtil.convertPartitionValue; import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toList; +import static java.util.stream.Collectors.toMap; import static java.util.stream.Collectors.toUnmodifiableList; public class HudiPageSourceProvider @@ -133,15 +139,20 @@ public ConnectorPageSource createPageSource( List hiveColumns = columns.stream() .map(HiveColumnHandle.class::cast) .collect(toList()); + // just send regular columns to create parquet page source + // for partition columns, separate blocks will be created + List regularColumns = hiveColumns.stream() + .filter(columnHandle -> !columnHandle.isPartitionKey()) + .collect(Collectors.toList()); Configuration configuration = hdfsEnvironment.getConfiguration(new HdfsContext(session), path); - return createParquetPageSource( + ConnectorPageSource dataPageSource = createParquetPageSource( hdfsEnvironment, configuration, path, start, length, estimatedFileSize, - hiveColumns, + regularColumns, true, split.getPredicate(), fileFormatDataSourceStats, @@ -149,6 +160,11 @@ public ConnectorPageSource createPageSource( parquetReaderOptions, session.getIdentity(), split.getPartitionKeys()); + + return new HudiPageSource( + hiveColumns, + convertPartitionValues(hiveColumns, split.getPartitionKeys()), // create blocks for partition values + dataPageSource); } private static ConnectorPageSource createParquetPageSource( @@ -167,7 +183,7 @@ private static ConnectorPageSource createParquetPageSource( ConnectorIdentity identity, List partitionKeys) { - log.debug(">>> Creating Parquet Page Source with partition keys: " + partitionKeys); + log.warn(">>> Creating Parquet Page Source with partition keys: %s, columns: %s, predicate: %s", partitionKeys, columns, effectivePredicate); ParquetDataSource dataSource = null; // TODO: Reuse some elements of ParquetPageSourceFactory and extract the try block to a new HudiParquetReader class. @@ -179,6 +195,7 @@ private static ConnectorPageSource createParquetPageSource( ParquetMetadata parquetMetadata = hdfsEnvironment.doAs(identity, () -> MetadataReader.readFooter(theDataSource)); FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); MessageType fileSchema = fileMetaData.getSchema(); + log.warn(">>> File Schema: " + fileSchema.toString()); Optional message = projectSufficientColumns(columns) .map(projection -> projection.get().stream() @@ -193,12 +210,15 @@ private static ConnectorPageSource createParquetPageSource( .reduce(MessageType::union); MessageType requestedSchema = message.orElse(new MessageType(fileSchema.getName(), ImmutableList.of())); + log.warn(">>> Requested Schema: " + requestedSchema); MessageColumnIO messageColumn = getColumnIO(fileSchema, requestedSchema); Map, RichColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, requestedSchema); + log.warn(">>> Ignore stats: " + options.isIgnoreStatistics()); TupleDomain parquetTupleDomain = options.isIgnoreStatistics() ? TupleDomain.all() : getParquetTupleDomain(descriptorsByPath, effectivePredicate, fileSchema, useParquetColumnNames); + log.warn(">>> ParquetTupleDomain: %s", parquetTupleDomain.toString()); Predicate parquetPredicate = buildPredicate(requestedSchema, parquetTupleDomain, descriptorsByPath, timeZone); @@ -217,6 +237,8 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq } nextStart += block.getRowCount(); } + + log.warn("Message Column: %s, Predicate: %s, ColumnIndexes: %s", messageColumn, parquetPredicate.toString(), columnIndexes.build()); ParquetReader parquetReader = new ParquetReader( Optional.ofNullable(fileMetaData.getCreatedBy()), messageColumn, @@ -229,16 +251,17 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq parquetPredicate, columnIndexes.build()); Optional readerProjections = projectBaseColumns(columns); + log.warn("Reade columns: %s", readerProjections.orElse(null)); List baseColumns = readerProjections.map(projection -> projection.get().stream() .map(HiveColumnHandle.class::cast) .collect(toUnmodifiableList())) .orElse(columns); + log.warn("Base columns: %s", baseColumns); - // TODO: add a check for patition column type - /*for (HiveColumnHandle column : baseColumns) { + for (HiveColumnHandle column : baseColumns) { checkArgument(column == PARQUET_ROW_INDEX_COLUMN || column.getColumnType() == REGULAR, "column type must be REGULAR: %s", column); - }*/ + } ImmutableList.Builder trinoTypes = ImmutableList.builder(); ImmutableList.Builder> internalFields = ImmutableList.builder(); @@ -258,6 +281,10 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq } } + log.warn("Trino types: %s", trinoTypes.build()); + log.warn("Internal fields: %s", internalFields.build()); + log.warn("Row index: %s", rowIndexColumns.build()); + return new ParquetPageSource( parquetReader, trinoTypes.build(), @@ -287,4 +314,21 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq throw new TrinoException(HUDI_CANNOT_OPEN_SPLIT, message, e); } } + + private Map convertPartitionValues( + List allColumns, + List partitionKeys) + { + log.warn(">>> Converting partition values for columns: %s, partition key: %s", allColumns, partitionKeys); + return allColumns.stream() + .filter(HiveColumnHandle::isPartitionKey) + .collect(toMap( + HiveColumnHandle::getName, + columnHandle -> Utils.nativeValueToBlock( + columnHandle.getType(), + convertPartitionValue( + columnHandle.getName(), + partitionKeys.get(0).getValue(), + columnHandle.getType().getTypeSignature())))); + } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index e1dda79fd4a..de3969bfdae 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -85,7 +85,7 @@ public ConnectorSplitSource getSplits( DynamicFilter dynamicFilter, Constraint constraint) { - log.debug(" >>>> Getting Splits <<<< "); + log.warn(" >>>> Getting Splits <<<< "); HiveIdentity identity = new HiveIdentity(session); HudiTableHandle hudiTable = (HudiTableHandle) tableHandle; SchemaTableName tableName = hudiTable.getSchemaTableName(); @@ -103,13 +103,13 @@ public ConnectorSplitSource getSplits( Configuration conf = hdfsEnvironment.getConfiguration(context, new Path(table.getStorage().getLocation())); HoodieTableMetaClient metaClient = hudiTable.getMetaClient().orElseGet(() -> getMetaClient(conf, hudiTable.getBasePath())); List partitionValues = getPartitionsWritten(metaClient.getActiveTimeline()); - log.debug("Fetched partitions from Hudi: " + partitionValues); - hudiTable.getPartitions().ifPresent(p -> p.forEach(p1 -> log.debug("Partitions from TableHandle: " + p1))); + log.warn("Fetched partitions from Hudi: " + partitionValues); + hudiTable.getPartitions().ifPresent(p -> p.forEach(p1 -> log.warn("Partitions from TableHandle: " + p1))); List columnNames = table.getPartitionColumns().stream() .map(Column::getName) .collect(toImmutableList()); - log.debug("Column Names: " + columnNames); + log.warn("Column Names: " + columnNames); HudiSplitSource splitSource; String tablePath = table.getStorage().getLocation(); Optional fileStatuses = Optional.empty(); @@ -119,14 +119,14 @@ public ConnectorSplitSource getSplits( .stream() .map(HiveUtil::toPartitionValues) .collect(toImmutableList()); - log.debug("Partition Names: " + partitionNames); + log.warn("Partition Names: " + partitionNames); Optional partition = metastore.getPartition(identity, table, partitionNames.get(0)); - log.debug("Fetched partitions from Metastore: " + partition.get()); + log.warn("Fetched partitions from Metastore: " + partition.get()); Properties schema = getPartitionSchema(table, partition); String dataDir = schema.getProperty(META_TABLE_LOCATION); - log.debug("Partition schema: " + schema); + log.warn("Partition schema: " + schema); List partitionKeys = getPartitionKeys(table, partition); partitionKeys.forEach(p -> log.warn("Fetched partitions from HiveUtil: " + p)); @@ -147,17 +147,17 @@ public ConnectorSplitSource getSplits( private static Optional getFileStatuses(FileSystem fs, Configuration conf, String tablePath, Optional fileStatuses, Properties schema) { InputFormat inputFormat = HiveUtil.getInputFormat(conf, schema, false); - log.debug(">>> Check for inputFormat: " + isHudiParquetInputFormat(inputFormat)); + log.warn(">>> Check for inputFormat: " + isHudiParquetInputFormat(inputFormat)); try { if (isHudiParquetInputFormat(inputFormat)) { fileStatuses = Optional.of(((HoodieParquetInputFormat) inputFormat).listStatus(toJobConf(conf))); } if (fileStatuses.isPresent()) { - log.debug(">>> Total Files: " + fileStatuses.get().length); + log.warn(">>> Total Files: " + fileStatuses.get().length); if (fileStatuses.get().length == 0 && fs != null) { fileStatuses = Optional.of(fs.listStatus(new Path(tablePath))); - log.debug(">>> Total Files: " + fileStatuses.get().length); + log.warn(">>> Total Files: " + fileStatuses.get().length); } } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java index 0ce83b2e585..db5012f9fb1 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java @@ -32,6 +32,7 @@ import java.util.Optional; import java.util.stream.Collectors; +import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; public class HudiTableHandle @@ -140,6 +141,19 @@ public SchemaTableName getSchemaTableName() return new SchemaTableName(schemaName, tableName); } + HudiTableHandle withPredicate(TupleDomain predicate) + { + checkState(this.predicate.isAll(), "There is already a predicate."); + return new HudiTableHandle( + schemaName, + tableName, + basePath, + tableType, + predicate, + partitions, + metaClient); + } + @Override public String toString() { diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java index fcd18de72b0..d6cfe4fe036 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java @@ -14,20 +14,48 @@ package io.trino.plugin.hudi; +import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; +import io.trino.plugin.hive.HiveColumnHandle; import io.trino.plugin.hive.metastore.Partition; import io.trino.plugin.hive.metastore.Table; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.type.Decimals; +import io.trino.spi.type.StandardTypes; +import io.trino.spi.type.TypeSignature; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.InputFormat; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.hadoop.HoodieParquetInputFormat; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeParseException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Properties; +import static com.google.common.base.Preconditions.checkArgument; +import static io.airlift.slice.Slices.utf8Slice; import static io.trino.plugin.hive.metastore.MetastoreUtil.getHiveSchema; +import static io.trino.plugin.hudi.HudiErrorCode.HUDI_INVALID_PARTITION_VALUE; +import static java.lang.Double.parseDouble; +import static java.lang.Float.floatToRawIntBits; +import static java.lang.Float.parseFloat; +import static java.lang.Long.parseLong; +import static java.lang.String.format; public class HudiUtil { + private static final Logger log = Logger.get(HudiUtil.class); + private HudiUtil() {} public static HoodieTableMetaClient getMetaClient(Configuration conf, String basePath) @@ -47,4 +75,73 @@ public static Properties getPartitionSchema(Table table, Optional par } return getHiveSchema(partition.get(), table); } + + public static List> splitPredicate( + TupleDomain predicate) + { + Map partitionColumnPredicates = new HashMap<>(); + Map regularColumnPredicates = new HashMap<>(); + + Optional> domains = predicate.getDomains(); + domains.ifPresent(columnHandleDomainMap -> columnHandleDomainMap.forEach((key, value) -> { + HiveColumnHandle columnHandle = (HiveColumnHandle) key; + if (columnHandle.isPartitionKey()) { + partitionColumnPredicates.put(key, value); + } + else { + regularColumnPredicates.put(key, value); + } + })); + + return ImmutableList.of( + TupleDomain.withColumnDomains(partitionColumnPredicates), + TupleDomain.withColumnDomains(regularColumnPredicates)); + } + + public static Object convertPartitionValue( + String partitionColumnName, + String partitionValue, + TypeSignature partitionDataType) + { + log.warn(">>> convertPartitionValue column: %s, value: %s, dataType: %s", partitionColumnName, partitionValue, partitionDataType); + if (partitionValue == null) { + return null; + } + + String typeBase = partitionDataType.getBase(); + log.warn(">>> Base Type: %s", typeBase); + try { + switch (typeBase) { + case StandardTypes.TINYINT: + case StandardTypes.SMALLINT: + case StandardTypes.INTEGER: + case StandardTypes.BIGINT: + return parseLong(partitionValue); + case StandardTypes.REAL: + return (long) floatToRawIntBits(parseFloat(partitionValue)); + case StandardTypes.DOUBLE: + return parseDouble(partitionValue); + case StandardTypes.VARCHAR: + case StandardTypes.VARBINARY: + return utf8Slice(partitionValue); + case StandardTypes.DATE: + return LocalDate.parse(partitionValue, DateTimeFormatter.ISO_LOCAL_DATE).toEpochDay(); + case StandardTypes.TIMESTAMP: + return Timestamp.valueOf(partitionValue).toLocalDateTime().toEpochSecond(ZoneOffset.UTC) * 1_000; + case StandardTypes.BOOLEAN: + checkArgument(partitionValue.equalsIgnoreCase("true") || partitionValue.equalsIgnoreCase("false")); + return Boolean.valueOf(partitionValue); + case StandardTypes.DECIMAL: + return Decimals.parse(partitionValue).getObject(); + default: + throw new TrinoException(HUDI_INVALID_PARTITION_VALUE, + format("Unsupported data type '%s' for partition column %s", partitionDataType, partitionColumnName)); + } + } + catch (IllegalArgumentException | DateTimeParseException e) { + throw new TrinoException(HUDI_INVALID_PARTITION_VALUE, + format("Can not parse partition value '%s' of type '%s' for partition column '%s'", + partitionValue, partitionDataType, partitionColumnName)); + } + } } From 0d2017fe8da178763518a4d6d3e36fb5e59cd00f Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Wed, 15 Dec 2021 19:05:30 -0800 Subject: [PATCH 09/26] Add splits to split source --- plugin/trino-hudi/pom.xml | 4 ++ .../java/io/trino/plugin/hudi/HudiConfig.java | 15 +++++++ .../plugin/hudi/HudiSessionProperties.java | 15 ++++++- .../io/trino/plugin/hudi/HudiSplitSource.java | 44 +++++++++++++++---- .../etc/catalog/hive.properties | 3 +- .../etc/catalog/hudi.properties | 1 + .../trino-server-dev/etc/config.properties | 4 +- .../etc/event-listener.properties | 2 +- 8 files changed, 74 insertions(+), 14 deletions(-) diff --git a/plugin/trino-hudi/pom.xml b/plugin/trino-hudi/pom.xml index 0d66076edc4..c47eb8f4a97 100644 --- a/plugin/trino-hudi/pom.xml +++ b/plugin/trino-hudi/pom.xml @@ -65,6 +65,10 @@ io.airlift log + + io.airlift + units + com.google.guava guava diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java index 68420d45924..7ca0579488a 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java @@ -16,6 +16,7 @@ import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; +import io.airlift.units.DataSize; import org.apache.hudi.common.model.HoodieFileFormat; import javax.validation.constraints.NotNull; @@ -26,6 +27,7 @@ public class HudiConfig { private HoodieFileFormat fileFormat = PARQUET; private boolean metadataEnabled; + private DataSize maxSplitSize = DataSize.ofBytes(128 * 1024 * 1024); @NotNull public HoodieFileFormat getFileFormat() @@ -53,4 +55,17 @@ public boolean isMetadataEnabled() { return this.metadataEnabled; } + + @Config("hudi.max_split_size") + public HudiConfig setMaxSplitSize(DataSize size) + { + this.maxSplitSize = size; + return this; + } + + @NotNull + public DataSize getMaxSplitSize() + { + return this.maxSplitSize; + } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java index 9932c203764..d1a4f21c39e 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java @@ -15,6 +15,7 @@ package io.trino.plugin.hudi; import com.google.common.collect.ImmutableList; +import io.airlift.units.DataSize; import io.trino.plugin.base.session.SessionPropertiesProvider; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.session.PropertyMetadata; @@ -24,6 +25,7 @@ import java.util.List; +import static io.trino.plugin.base.session.PropertyMetadataUtil.dataSizeProperty; import static io.trino.spi.session.PropertyMetadata.booleanProperty; import static io.trino.spi.session.PropertyMetadata.enumProperty; @@ -32,6 +34,7 @@ public class HudiSessionProperties { private static final String FILE_FORMAT = "file_format"; private static final String METADATA_ENABLED = "metadata_enabled"; + private static final String MAX_SPLIT_SIZE = "max_split_size"; private final List> sessionProperties; @@ -49,7 +52,12 @@ public HudiSessionProperties(HudiConfig hudiConfig) METADATA_ENABLED, "For Hudi tables prefer to fetch the list of files from its metadata", hudiConfig.isMetadataEnabled(), - false)); + false), + dataSizeProperty( + MAX_SPLIT_SIZE, + "Max split size", + hudiConfig.getMaxSplitSize(), + true)); } @Override @@ -67,4 +75,9 @@ public static boolean isHudiMetadataEnabled(ConnectorSession session) { return session.getProperty(METADATA_ENABLED, Boolean.class); } + + public static DataSize getMaxSplitSize(ConnectorSession session) + { + return session.getProperty(MAX_SPLIT_SIZE, DataSize.class); + } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index 9e34fbfae31..cdce5e7fc93 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -16,8 +16,10 @@ import com.google.common.collect.ImmutableList; import io.airlift.log.Logger; +import io.airlift.units.DataSize; import io.trino.plugin.hive.HivePartitionKey; import io.trino.spi.connector.ConnectorPartitionHandle; +import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitSource; import org.apache.hadoop.conf.Configuration; @@ -47,11 +49,12 @@ public class HudiSplitSource implements ConnectorSplitSource { private static final Logger log = Logger.get(HudiSplitSource.class); - + private final Configuration conf; private final HudiTableHandle tableHandle; private final HoodieTableMetaClient metaClient; private final HoodieTableFileSystemView fileSystemView; private final Iterator hoodieBaseFileIterator; + private final DataSize maxSplitSize; private final List partitionKeys; private final boolean metadataEnabled; private final Optional fileStatuses; @@ -59,6 +62,7 @@ public class HudiSplitSource private final String dataDir; public HudiSplitSource( + ConnectorSession session, HudiTableHandle tableHandle, Configuration conf, List partitionKeys, @@ -67,6 +71,8 @@ public HudiSplitSource( String tablePath, String dataDir) { + requireNonNull(session, "session is null"); + this.conf = requireNonNull(conf, "conf is null"); this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); this.partitionKeys = requireNonNull(partitionKeys, "partitionKeys is null"); this.metadataEnabled = metadataEnabled; @@ -91,6 +97,7 @@ public HudiSplitSource( else { this.hoodieBaseFileIterator = fileSystemView.getLatestBaseFiles(partition).iterator(); } + this.maxSplitSize = HudiSessionProperties.getMaxSplitSize(session); } @Override @@ -98,6 +105,7 @@ public CompletableFuture getNextBatch(ConnectorPartitionHan { log.debug("Getting next batch with partitionKeys: " + partitionKeys); List splits = new ArrayList<>(); + long maxSplitBytes = maxSplitSize.toBytes(); Iterator baseFileIterator = limit(hoodieBaseFileIterator, maxSize); while (baseFileIterator.hasNext()) { HoodieBaseFile baseFile = baseFileIterator.next(); @@ -107,14 +115,32 @@ public CompletableFuture getNextBatch(ConnectorPartitionHan /*String[] name = new String[] {"localhost:" + DFS_DATANODE_DEFAULT_PORT}; String[] host = new String[] {"localhost"}; BlockLocation[] blockLocations = new BlockLocation[] {new BlockLocation(name, host, 0L, fileStatus.getLen())};*/ - splits.add(new HudiSplit( - baseFile.getPath(), - 0L, - baseFile.getFileLen(), - baseFile.getFileSize(), - ImmutableList.of(), - tableHandle.getPredicate(), - partitionKeys)); + long remainingFileBytes = baseFile.getFileSize(); + long start = 0; + long splitBytes; + log.debug(String.format("Base file %s, %d bytes", baseFile.getPath(), remainingFileBytes)); + while (remainingFileBytes > 0) { + if (remainingFileBytes <= maxSplitBytes) { + splitBytes = remainingFileBytes; + } + else if (maxSplitBytes * 2 >= remainingFileBytes) { + splitBytes = remainingFileBytes / 2; + } + else { + splitBytes = maxSplitBytes; + } + log.debug(String.format("Split: start=%d len=%d remaining=%d", start, splitBytes, remainingFileBytes)); + splits.add(new HudiSplit( + baseFile.getPath(), + start, + splitBytes, + baseFile.getFileSize(), + ImmutableList.of(), + tableHandle.getPredicate(), + partitionKeys)); + start += splitBytes; + remainingFileBytes -= splitBytes; + } } return completedFuture(new ConnectorSplitBatch(splits, isFinished())); diff --git a/testing/trino-server-dev/etc/catalog/hive.properties b/testing/trino-server-dev/etc/catalog/hive.properties index 3f2d401d4ce..3e12a2f6c25 100644 --- a/testing/trino-server-dev/etc/catalog/hive.properties +++ b/testing/trino-server-dev/etc/catalog/hive.properties @@ -10,8 +10,9 @@ connector.name=hive # Configuration appropriate for Hive as started by product test environment, e.g. # trino-product-tests-launcher/bin/run-launcher env up --environment singlenode --without-trino # On Mac, this additionally requires that you add " hadoop-master" to /etc/hosts -hive.metastore.uri=thrift://localhost:9083 +hive.metastore.uri=thrift://hivemetastore:9083 hive.hdfs.socks-proxy=localhost:1180 +hive.config.resources=/Users/ethan/Work/data/conf/core-site.xml,/Users/ethan/Work/data/conf/hdfs-site.xml # Fail-fast in development hive.metastore.thrift.client.max-retry-time=1s diff --git a/testing/trino-server-dev/etc/catalog/hudi.properties b/testing/trino-server-dev/etc/catalog/hudi.properties index 33ba981c9bc..0b50f174bf9 100644 --- a/testing/trino-server-dev/etc/catalog/hudi.properties +++ b/testing/trino-server-dev/etc/catalog/hudi.properties @@ -14,4 +14,5 @@ connector.name=hudi hive.metastore.uri=thrift://localhost:9083 +hudi.max_split_size=16MB #hive.config.resources=/Users/sagars/core-site.xml,/Users/sagars/hdfs-site.xml diff --git a/testing/trino-server-dev/etc/config.properties b/testing/trino-server-dev/etc/config.properties index fd2beb31dba..36eaa0a681b 100644 --- a/testing/trino-server-dev/etc/config.properties +++ b/testing/trino-server-dev/etc/config.properties @@ -10,9 +10,9 @@ node.id=ffffffff-ffff-ffff-ffff-ffffffffffff node.environment=test node.internal-address=localhost experimental.concurrent-startup=true -http-server.http.port=8080 +http-server.http.port=9090 -discovery.uri=http://localhost:8080 +discovery.uri=http://localhost:9090 exchange.http-client.max-connections=1000 exchange.http-client.max-connections-per-server=1000 diff --git a/testing/trino-server-dev/etc/event-listener.properties b/testing/trino-server-dev/etc/event-listener.properties index 74fdeb5c470..21e08c10eef 100644 --- a/testing/trino-server-dev/etc/event-listener.properties +++ b/testing/trino-server-dev/etc/event-listener.properties @@ -1,3 +1,3 @@ event-listener.name=http http-event-listener.log-created=true -http-event-listener.connect-ingest-uri=http://localhost:8090 +http-event-listener.connect-ingest-uri=http://localhost:9091 From 4602d02b9a88693427e84c8000feda1aa626a73b Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 16 Dec 2021 19:36:47 -0800 Subject: [PATCH 10/26] Fix configuration for list status --- .../trino/plugin/hudi/HudiSplitManager.java | 64 ++++++++++++------- .../io/trino/plugin/hudi/HudiSplitSource.java | 2 +- 2 files changed, 41 insertions(+), 25 deletions(-) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index de3969bfdae..ffa7e5783ab 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -39,7 +39,9 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.FileInputFormat; import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.JobConf; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.hadoop.HoodieParquetInputFormat; @@ -47,10 +49,13 @@ import java.io.IOException; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Properties; +import java.util.stream.Collectors; import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.Maps.fromProperties; import static io.trino.plugin.hive.util.ConfigurationUtils.toJobConf; import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeys; import static io.trino.plugin.hudi.HudiSessionProperties.isHudiMetadataEnabled; @@ -131,39 +136,50 @@ public ConnectorSplitSource getSplits( List partitionKeys = getPartitionKeys(table, partition); partitionKeys.forEach(p -> log.warn("Fetched partitions from HiveUtil: " + p)); - fileStatuses = getFileStatuses(fs, conf, dataDir, fileStatuses, schema); - splitSource = new HudiSplitSource(hudiTable, conf, partitionKeys, isHudiMetadataEnabled(session), fileStatuses, tablePath, dataDir); + InputFormat inputFormat = HiveUtil.getInputFormat(conf, schema, false); + log.warn(">>> Check for inputFormat: " + isHudiParquetInputFormat(inputFormat)); + log.warn(">>> Conf: "); + printConf(conf); + + try { + if (isHudiParquetInputFormat(inputFormat)) { + JobConf jobConf = toJobConf(conf); + String allAbsolutePartitonPaths = String.join(",", partitionValues.stream().map( + relativePartitionPath -> tablePath + "/" + relativePartitionPath + ).collect(Collectors.toList())); + FileInputFormat.setInputPaths(jobConf, allAbsolutePartitonPaths); + // Pass SerDes and Table parameters into input format configuration + fromProperties(schema).forEach(jobConf::set); + log.warn(">>> Updated conf: "); + printConf(jobConf); + fileStatuses = Optional.of(((HoodieParquetInputFormat) inputFormat).listStatus(jobConf)); + } + if (fileStatuses.isPresent()) { + log.warn(">>> Total Files: " + fileStatuses.get().length); + if (fileStatuses.get().length == 0 && fs != null) { + fileStatuses = Optional.of(fs.listStatus(new Path(dataDir))); + log.warn(">>> Total Files: " + fileStatuses.get().length); + } + } + log.warn(">>> Total Splits: " + inputFormat.getSplits(toJobConf(conf), 0).length); + } + catch (IOException e) { + e.printStackTrace(); + } + splitSource = new HudiSplitSource(session, hudiTable, conf, partitionKeys, isHudiMetadataEnabled(session), fileStatuses, tablePath, dataDir); } else { // no partitions, so data dir is same as table path - Properties schema = getPartitionSchema(table, Optional.empty()); - fileStatuses = getFileStatuses(fs, conf, tablePath, fileStatuses, schema); - splitSource = new HudiSplitSource(hudiTable, conf, ImmutableList.of(), isHudiMetadataEnabled(session), fileStatuses, tablePath, tablePath); + splitSource = new HudiSplitSource(session, hudiTable, conf, ImmutableList.of(), isHudiMetadataEnabled(session), fileStatuses, tablePath, tablePath); } return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); } - private static Optional getFileStatuses(FileSystem fs, Configuration conf, String tablePath, Optional fileStatuses, Properties schema) + void printConf(Configuration conf) { - InputFormat inputFormat = HiveUtil.getInputFormat(conf, schema, false); - log.warn(">>> Check for inputFormat: " + isHudiParquetInputFormat(inputFormat)); - - try { - if (isHudiParquetInputFormat(inputFormat)) { - fileStatuses = Optional.of(((HoodieParquetInputFormat) inputFormat).listStatus(toJobConf(conf))); - } - if (fileStatuses.isPresent()) { - log.warn(">>> Total Files: " + fileStatuses.get().length); - if (fileStatuses.get().length == 0 && fs != null) { - fileStatuses = Optional.of(fs.listStatus(new Path(tablePath))); - log.warn(">>> Total Files: " + fileStatuses.get().length); - } - } - } - catch (IOException e) { - e.printStackTrace(); + for (Map.Entry entry : conf) { + log.debug("%s=%s\n", entry.getKey(), entry.getValue()); } - return fileStatuses; } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index cdce5e7fc93..e5c8db70223 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -97,7 +97,7 @@ public HudiSplitSource( else { this.hoodieBaseFileIterator = fileSystemView.getLatestBaseFiles(partition).iterator(); } - this.maxSplitSize = HudiSessionProperties.getMaxSplitSize(session); + this.maxSplitSize = DataSize.ofBytes(32 * 1024 * 1024); } @Override From 47ecc78868fc7feca035c78b4eaa1a092f742018 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 16 Dec 2021 23:45:24 -0800 Subject: [PATCH 11/26] Fix logic using HoodieParquetInputFormat to get splits and support bootstrap --- .../trino/plugin/hudi/HudiSplitManager.java | 76 +++++++++---------- .../io/trino/plugin/hudi/HudiSplitSource.java | 74 ++++++------------ 2 files changed, 59 insertions(+), 91 deletions(-) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index ffa7e5783ab..04fd8517f2f 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -36,13 +36,14 @@ import io.trino.spi.connector.TableNotFoundException; import io.trino.spi.predicate.TupleDomain; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.FileInputFormat; -import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.HoodieParquetInputFormat; import javax.inject.Inject; @@ -61,7 +62,6 @@ import static io.trino.plugin.hudi.HudiSessionProperties.isHudiMetadataEnabled; import static io.trino.plugin.hudi.HudiUtil.getMetaClient; import static io.trino.plugin.hudi.HudiUtil.getPartitionSchema; -import static io.trino.plugin.hudi.HudiUtil.isHudiParquetInputFormat; import static java.util.Objects.requireNonNull; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_LOCATION; import static org.apache.hudi.common.table.timeline.TimelineUtils.getPartitionsWritten; @@ -117,7 +117,8 @@ public ConnectorSplitSource getSplits( log.warn("Column Names: " + columnNames); HudiSplitSource splitSource; String tablePath = table.getStorage().getLocation(); - Optional fileStatuses = Optional.empty(); + List partitionKeys = ImmutableList.of(); + Properties schema; if (!columnNames.isEmpty()) { List> partitionNames = metastore.getPartitionNamesByFilter(identity, tableName.getSchemaName(), tableName.getTableName(), columnNames, TupleDomain.all()) .orElseThrow(() -> new TableNotFoundException(hudiTable.getSchemaTableName())) @@ -127,59 +128,52 @@ public ConnectorSplitSource getSplits( log.warn("Partition Names: " + partitionNames); Optional partition = metastore.getPartition(identity, table, partitionNames.get(0)); + schema = getPartitionSchema(table, partition); log.warn("Fetched partitions from Metastore: " + partition.get()); - Properties schema = getPartitionSchema(table, partition); - String dataDir = schema.getProperty(META_TABLE_LOCATION); log.warn("Partition schema: " + schema); - List partitionKeys = getPartitionKeys(table, partition); + partitionKeys = getPartitionKeys(table, partition); partitionKeys.forEach(p -> log.warn("Fetched partitions from HiveUtil: " + p)); - InputFormat inputFormat = HiveUtil.getInputFormat(conf, schema, false); - log.warn(">>> Check for inputFormat: " + isHudiParquetInputFormat(inputFormat)); - log.warn(">>> Conf: "); - printConf(conf); - - try { - if (isHudiParquetInputFormat(inputFormat)) { - JobConf jobConf = toJobConf(conf); - String allAbsolutePartitonPaths = String.join(",", partitionValues.stream().map( - relativePartitionPath -> tablePath + "/" + relativePartitionPath - ).collect(Collectors.toList())); - FileInputFormat.setInputPaths(jobConf, allAbsolutePartitonPaths); - // Pass SerDes and Table parameters into input format configuration - fromProperties(schema).forEach(jobConf::set); - log.warn(">>> Updated conf: "); - printConf(jobConf); - fileStatuses = Optional.of(((HoodieParquetInputFormat) inputFormat).listStatus(jobConf)); - } - if (fileStatuses.isPresent()) { - log.warn(">>> Total Files: " + fileStatuses.get().length); - if (fileStatuses.get().length == 0 && fs != null) { - fileStatuses = Optional.of(fs.listStatus(new Path(dataDir))); - log.warn(">>> Total Files: " + fileStatuses.get().length); - } - } - log.warn(">>> Total Splits: " + inputFormat.getSplits(toJobConf(conf), 0).length); - } - catch (IOException e) { - e.printStackTrace(); - } - splitSource = new HudiSplitSource(session, hudiTable, conf, partitionKeys, isHudiMetadataEnabled(session), fileStatuses, tablePath, dataDir); + // InputFormat inputFormat = HiveUtil.getInputFormat(conf, schema, false); + // log.debug(">>> Check for inputFormat: " + isHudiParquetInputFormat(inputFormat)); + // log.debug(">>> Conf: "); + // printConf(conf); } else { // no partitions, so data dir is same as table path - splitSource = new HudiSplitSource(session, hudiTable, conf, ImmutableList.of(), isHudiMetadataEnabled(session), fileStatuses, tablePath, tablePath); + schema = getPartitionSchema(table, Optional.empty()); + partitionValues = ImmutableList.of(""); } - return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); + // Non Hudi table should also be compatible with HoodieParquetInputFormat + HoodieParquetInputFormat inputFormat = new HoodieParquetInputFormat(); + inputFormat.setConf(conf); + + JobConf jobConf = toJobConf(conf); + String allAbsolutePartitionPaths = partitionValues.stream().map( + relativePartitionPath -> FSUtils.getPartitionPath(tablePath, relativePartitionPath).toString() + ).collect(Collectors.joining(",")); + FileInputFormat.setInputPaths(jobConf, allAbsolutePartitionPaths); + // Pass SerDes and Table parameters into input format configuration + fromProperties(schema).forEach(jobConf::set); + String dataDir = schema.getProperty(META_TABLE_LOCATION); + try { + InputSplit[] inputSplits = inputFormat.getSplits(jobConf, 0); + log.warn(">>> Total Splits: " + inputSplits.length); + splitSource = new HudiSplitSource(session, hudiTable, conf, partitionKeys, isHudiMetadataEnabled(session), inputSplits, tablePath, dataDir); + return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); + } + catch (IOException e) { + throw new HoodieIOException("Error getting input splits", e); + } } void printConf(Configuration conf) { for (Map.Entry entry : conf) { - log.debug("%s=%s\n", entry.getKey(), entry.getValue()); + log.warn("%s=%s\n", entry.getKey(), entry.getValue()); } } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index e5c8db70223..c1ca8b8b063 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -23,21 +23,22 @@ import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitSource; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputSplit; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Iterator; import java.util.List; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import static com.google.common.collect.Iterators.limit; @@ -53,11 +54,10 @@ public class HudiSplitSource private final HudiTableHandle tableHandle; private final HoodieTableMetaClient metaClient; private final HoodieTableFileSystemView fileSystemView; - private final Iterator hoodieBaseFileIterator; + private final Iterator inputSplitIterator; private final DataSize maxSplitSize; private final List partitionKeys; private final boolean metadataEnabled; - private final Optional fileStatuses; private final String tablePath; private final String dataDir; @@ -67,7 +67,7 @@ public HudiSplitSource( Configuration conf, List partitionKeys, boolean metadataEnabled, - Optional fileStatuses, + InputSplit[] inputSplits, String tablePath, String dataDir) { @@ -76,7 +76,6 @@ public HudiSplitSource( this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); this.partitionKeys = requireNonNull(partitionKeys, "partitionKeys is null"); this.metadataEnabled = metadataEnabled; - this.fileStatuses = fileStatuses; this.tablePath = tablePath; this.dataDir = dataDir; this.metaClient = tableHandle.getMetaClient().orElseGet(() -> getMetaClient(conf, tableHandle.getBasePath())); @@ -89,14 +88,7 @@ public HudiSplitSource( log.debug("Table path: %s \nDirectory: %s", tablePath, dataDir); String partition = FSUtils.getRelativePartitionPath(new Path(tablePath), new Path(dataDir)); log.debug("Partition: %s", partition); - if (fileStatuses.isPresent()) { - log.warn(">>> FileStatus present adding to view: %s", fileStatuses.get().length); - fileSystemView.addFilesToView(fileStatuses.get()); - this.hoodieBaseFileIterator = fileSystemView.fetchLatestBaseFiles(partition).iterator(); - } - else { - this.hoodieBaseFileIterator = fileSystemView.getLatestBaseFiles(partition).iterator(); - } + this.inputSplitIterator = Arrays.stream(inputSplits).iterator(); this.maxSplitSize = DataSize.ofBytes(32 * 1024 * 1024); } @@ -104,46 +96,28 @@ public HudiSplitSource( public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) { log.debug("Getting next batch with partitionKeys: " + partitionKeys); - List splits = new ArrayList<>(); - long maxSplitBytes = maxSplitSize.toBytes(); - Iterator baseFileIterator = limit(hoodieBaseFileIterator, maxSize); - while (baseFileIterator.hasNext()) { - HoodieBaseFile baseFile = baseFileIterator.next(); - log.warn(">>>> Base File: " + baseFile); - FileStatus fileStatus = baseFile.getFileStatus(); - log.warn(">>>> FileStatus: " + fileStatus.toString()); - /*String[] name = new String[] {"localhost:" + DFS_DATANODE_DEFAULT_PORT}; - String[] host = new String[] {"localhost"}; - BlockLocation[] blockLocations = new BlockLocation[] {new BlockLocation(name, host, 0L, fileStatus.getLen())};*/ - long remainingFileBytes = baseFile.getFileSize(); - long start = 0; - long splitBytes; - log.debug(String.format("Base file %s, %d bytes", baseFile.getPath(), remainingFileBytes)); - while (remainingFileBytes > 0) { - if (remainingFileBytes <= maxSplitBytes) { - splitBytes = remainingFileBytes; - } - else if (maxSplitBytes * 2 >= remainingFileBytes) { - splitBytes = remainingFileBytes / 2; - } - else { - splitBytes = maxSplitBytes; - } - log.debug(String.format("Split: start=%d len=%d remaining=%d", start, splitBytes, remainingFileBytes)); - splits.add(new HudiSplit( - baseFile.getPath(), - start, - splitBytes, - baseFile.getFileSize(), + List connectorSplits = new ArrayList<>(); + Iterator batchSplitIterator = limit(inputSplitIterator, maxSize); + while (batchSplitIterator.hasNext()) { + FileSplit fileSplit = (FileSplit) batchSplitIterator.next(); + log.debug(String.format(">>>> File split: %s start=%d len=%d", + fileSplit.getPath(), fileSplit.getStart(), fileSplit.getLength())); + try { + connectorSplits.add(new HudiSplit( + fileSplit.getPath().toString(), + fileSplit.getStart(), + fileSplit.getLength(), + metaClient.getFs().getLength(fileSplit.getPath()), ImmutableList.of(), tableHandle.getPredicate(), partitionKeys)); - start += splitBytes; - remainingFileBytes -= splitBytes; + } + catch (IOException e) { + log.error("Error getting file size: " + e.getMessage()); } } - return completedFuture(new ConnectorSplitBatch(splits, isFinished())); + return completedFuture(new ConnectorSplitBatch(connectorSplits, isFinished())); } @Override @@ -155,6 +129,6 @@ public void close() @Override public boolean isFinished() { - return !hoodieBaseFileIterator.hasNext(); + return !inputSplitIterator.hasNext(); } } From 8097d615aba8df440b48037cc3483c04d9e06368 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Sun, 19 Dec 2021 22:09:46 -0800 Subject: [PATCH 12/26] Add flag to control how to fetch splits --- .../java/io/trino/plugin/hudi/HudiConfig.java | 15 +++ .../plugin/hudi/HudiSessionProperties.java | 11 ++ .../java/io/trino/plugin/hudi/HudiUtil.java | 121 ++++++++++++++++++ 3 files changed, 147 insertions(+) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java index 7ca0579488a..4a4b92edcff 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java @@ -27,6 +27,7 @@ public class HudiConfig { private HoodieFileFormat fileFormat = PARQUET; private boolean metadataEnabled; + private boolean splitInSource = false; private DataSize maxSplitSize = DataSize.ofBytes(128 * 1024 * 1024); @NotNull @@ -68,4 +69,18 @@ public DataSize getMaxSplitSize() { return this.maxSplitSize; } + + @Config("hudi.split_in_source") + @ConfigDescription("Whether to split files in the HudiSplitSource. If false, done in HudiSplitManager.") + public HudiConfig setSplitInSource(boolean splitInSource) + { + this.splitInSource = splitInSource; + return this; + } + + @NotNull + public boolean isSplitInSource() + { + return this.splitInSource; + } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java index d1a4f21c39e..3510ab5d231 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java @@ -35,6 +35,7 @@ public class HudiSessionProperties private static final String FILE_FORMAT = "file_format"; private static final String METADATA_ENABLED = "metadata_enabled"; private static final String MAX_SPLIT_SIZE = "max_split_size"; + private static final String SPLIT_IN_SOURCE = "split_in_source"; private final List> sessionProperties; @@ -53,6 +54,11 @@ public HudiSessionProperties(HudiConfig hudiConfig) "For Hudi tables prefer to fetch the list of files from its metadata", hudiConfig.isMetadataEnabled(), false), + booleanProperty( + SPLIT_IN_SOURCE, + "Whether to split files in the HudiSplitSource. If false, done in HudiSplitManager.", + hudiConfig.isSplitInSource(), + false), dataSizeProperty( MAX_SPLIT_SIZE, "Max split size", @@ -80,4 +86,9 @@ public static DataSize getMaxSplitSize(ConnectorSession session) { return session.getProperty(MAX_SPLIT_SIZE, DataSize.class); } + + public static boolean isSplitInSource(ConnectorSession session) + { + return session.getProperty(SPLIT_IN_SOURCE, Boolean.class); + } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java index d6cfe4fe036..4ec566f1424 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java @@ -27,15 +27,27 @@ import io.trino.spi.type.StandardTypes; import io.trino.spi.type.TypeSignature; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.BlockLocation; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.net.NetworkTopology; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.hadoop.HoodieParquetInputFormat; +import org.apache.hudi.hadoop.PathWithBootstrapFileStatus; +import java.io.IOException; import java.sql.Timestamp; import java.time.LocalDate; import java.time.ZoneOffset; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeParseException; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -51,9 +63,12 @@ import static java.lang.Float.parseFloat; import static java.lang.Long.parseLong; import static java.lang.String.format; +import static org.apache.hadoop.mapred.FileInputFormat.NUM_INPUT_FILES; public class HudiUtil { + private static final double SPLIT_SLOP = 1.1; // 10% slop + private static final Logger log = Logger.get(HudiUtil.class); private HudiUtil() {} @@ -144,4 +159,110 @@ public static Object convertPartitionValue( partitionValue, partitionDataType, partitionColumnName)); } } + + public static InputSplit[] getSplits(FileStatus[] fileStatuses, JobConf job, int numSplits, + HoodieParquetInputFormat inputFormat) throws IOException { + // Save the number of input files for metrics/loadgen + job.setLong(NUM_INPUT_FILES, fileStatuses.length); + long totalSize = 0; // compute total size + for (FileStatus file: fileStatuses) { // check we have valid files + if (file.isDirectory()) { + throw new IOException("Not a file: " + file.getPath()); + } + totalSize += file.getLen(); + } + + long goalSize = totalSize / (numSplits == 0 ? 1 : numSplits); + long minSize = Math.max(job.getLong(org.apache.hadoop.mapreduce.lib.input. + FileInputFormat.SPLIT_MINSIZE, 1), 1); + + // generate splits + ArrayList splits = new ArrayList(numSplits); + NetworkTopology clusterMap = new NetworkTopology(); + for (FileStatus file: fileStatuses) { + Path path = file.getPath(); + long length = file.getLen(); + if (length != 0) { + FileSystem fs = path.getFileSystem(job); + BlockLocation[] blkLocations; + if (file instanceof LocatedFileStatus) { + blkLocations = ((LocatedFileStatus) file).getBlockLocations(); + } else { + blkLocations = fs.getFileBlockLocations(file, 0, length); + } + if (isSplitable(fs, path)) { + long blockSize = file.getBlockSize(); + long splitSize = computeSplitSize(goalSize, minSize, blockSize); + + long bytesRemaining = length; + while (((double) bytesRemaining)/splitSize > SPLIT_SLOP) { + String[][] splitHosts = getSplitHostsAndCachedHosts(blkLocations, + length-bytesRemaining, splitSize, clusterMap); + splits.add(makeSplit(path, length-bytesRemaining, splitSize, + splitHosts[0], splitHosts[1])); + bytesRemaining -= splitSize; + } + + if (bytesRemaining != 0) { + String[][] splitHosts = getSplitHostsAndCachedHosts(blkLocations, length + - bytesRemaining, bytesRemaining, clusterMap); + splits.add(makeSplit(path, length - bytesRemaining, bytesRemaining, + splitHosts[0], splitHosts[1])); + } + } else { + String[][] splitHosts = getSplitHostsAndCachedHosts(blkLocations,0,length,clusterMap); + splits.add(makeSplit(path, 0, length, splitHosts[0], splitHosts[1])); + } + } else { + //Create empty hosts array for zero length files + splits.add(makeSplit(path, 0, length, new String[0])); + } + } + return splits.toArray(new FileSplit[splits.size()]); + } + + private static boolean isSplitable(FileSystem fs, Path filename) { + return !(filename instanceof PathWithBootstrapFileStatus); + } + + private static long computeSplitSize(long goalSize, long minSize, + long blockSize) { + return Math.max(minSize, Math.min(goalSize, blockSize)); + } + + private static FileSplit makeSplit(Path file, long start, long length, + String[] hosts) { + return new FileSplit(file, start, length, hosts); + } + + private static FileSplit makeSplit(Path file, long start, long length, + String[] hosts, String[] inMemoryHosts) { + return new FileSplit(file, start, length, hosts, inMemoryHosts); + } + + private static String[][] getSplitHostsAndCachedHosts(BlockLocation[] blkLocations, + long offset, long splitSize, NetworkTopology clusterMap) + throws IOException { + + int startIndex = getBlockIndex(blkLocations, offset); + + return new String[][]{blkLocations[startIndex].getHosts(), + blkLocations[startIndex].getCachedHosts()}; + } + + private static int getBlockIndex(BlockLocation[] blkLocations, + long offset) { + for (int i = 0 ; i < blkLocations.length; i++) { + // is the offset inside this block? + if ((blkLocations[i].getOffset() <= offset) && + (offset < blkLocations[i].getOffset() + blkLocations[i].getLength())){ + return i; + } + } + BlockLocation last = blkLocations[blkLocations.length -1]; + long fileLength = last.getOffset() + last.getLength() -1; + throw new IllegalArgumentException("Offset " + offset + + " is outside of file (0.." + + fileLength + ")"); + } } From 025e26952e1e9d3fbea17a3b49eebbd2d8ce646f Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Mon, 20 Dec 2021 22:59:14 -0800 Subject: [PATCH 13/26] Refactor file listing and Hudi split --- .../java/io/trino/plugin/hudi/HudiConfig.java | 2 +- .../trino/plugin/hudi/HudiSplitManager.java | 26 ++-- .../io/trino/plugin/hudi/HudiSplitSource.java | 125 ++++++++++++------ .../java/io/trino/plugin/hudi/HudiUtil.java | 118 ++++++++--------- 4 files changed, 153 insertions(+), 118 deletions(-) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java index 4a4b92edcff..aad3edc21e8 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java @@ -27,7 +27,7 @@ public class HudiConfig { private HoodieFileFormat fileFormat = PARQUET; private boolean metadataEnabled; - private boolean splitInSource = false; + private boolean splitInSource; private DataSize maxSplitSize = DataSize.ofBytes(128 * 1024 * 1024); @NotNull diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index 04fd8517f2f..f58020b19ad 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -38,37 +38,30 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapred.FileInputFormat; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.hadoop.HoodieParquetInputFormat; import javax.inject.Inject; import java.io.IOException; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; -import java.util.stream.Collectors; +import java.util.regex.Pattern; import static com.google.common.collect.ImmutableList.toImmutableList; -import static com.google.common.collect.Maps.fromProperties; -import static io.trino.plugin.hive.util.ConfigurationUtils.toJobConf; import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeys; import static io.trino.plugin.hudi.HudiSessionProperties.isHudiMetadataEnabled; import static io.trino.plugin.hudi.HudiUtil.getMetaClient; import static io.trino.plugin.hudi.HudiUtil.getPartitionSchema; import static java.util.Objects.requireNonNull; -import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_LOCATION; import static org.apache.hudi.common.table.timeline.TimelineUtils.getPartitionsWritten; public class HudiSplitManager implements ConnectorSplitManager { + public static final Pattern HOODIE_CONSUME_MODE_PATTERN_STRING = Pattern.compile("hoodie\\.(.*)\\.consume\\.mode"); private static final Logger log = Logger.get(HudiSplitManager.class); private final HudiTransactionManager transactionManager; @@ -106,8 +99,14 @@ public ConnectorSplitSource getSplits( e.printStackTrace(); } Configuration conf = hdfsEnvironment.getConfiguration(context, new Path(table.getStorage().getLocation())); + Map valByRegex = conf.getValByRegex(HOODIE_CONSUME_MODE_PATTERN_STRING.pattern()); + log.debug("Hoodie consume mode: " + valByRegex); HoodieTableMetaClient metaClient = hudiTable.getMetaClient().orElseGet(() -> getMetaClient(conf, hudiTable.getBasePath())); List partitionValues = getPartitionsWritten(metaClient.getActiveTimeline()); + log.debug("HudiSplitManager ref: " + this.toString()); + log.debug("Table ref: " + table.toString()); + log.debug("HoodieTableMetaClient ref: " + metaClient.toString()); + log.debug("HoodieTableMetaClient base path: " + metaClient.getBasePath()); log.warn("Fetched partitions from Hudi: " + partitionValues); hudiTable.getPartitions().ifPresent(p -> p.forEach(p1 -> log.warn("Partitions from TableHandle: " + p1))); @@ -119,6 +118,7 @@ public ConnectorSplitSource getSplits( String tablePath = table.getStorage().getLocation(); List partitionKeys = ImmutableList.of(); Properties schema; + Iterator relativePartitionPaths; if (!columnNames.isEmpty()) { List> partitionNames = metastore.getPartitionNamesByFilter(identity, tableName.getSchemaName(), tableName.getTableName(), columnNames, TupleDomain.all()) .orElseThrow(() -> new TableNotFoundException(hudiTable.getSchemaTableName())) @@ -147,6 +147,7 @@ public ConnectorSplitSource getSplits( partitionValues = ImmutableList.of(""); } + /* // Non Hudi table should also be compatible with HoodieParquetInputFormat HoodieParquetInputFormat inputFormat = new HoodieParquetInputFormat(); inputFormat.setConf(conf); @@ -168,6 +169,11 @@ public ConnectorSplitSource getSplits( catch (IOException e) { throw new HoodieIOException("Error getting input splits", e); } + */ + relativePartitionPaths = partitionValues.iterator(); + splitSource = new HudiSplitSource(session, hudiTable, conf, partitionKeys, relativePartitionPaths, + isHudiMetadataEnabled(session), tablePath); + return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); } void printConf(Configuration conf) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index c1ca8b8b063..9fd3172dae9 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -23,25 +23,25 @@ import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitSource; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.mapred.FileSplit; -import org.apache.hadoop.mapred.InputSplit; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.engine.HoodieLocalEngineContext; -import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.exception.HoodieIOException; import java.io.IOException; +import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.Arrays; import java.util.Iterator; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; -import static com.google.common.collect.Iterators.limit; import static io.trino.plugin.hudi.HudiUtil.getMetaClient; import static java.util.Objects.requireNonNull; import static java.util.concurrent.CompletableFuture.completedFuture; @@ -52,43 +52,35 @@ public class HudiSplitSource private static final Logger log = Logger.get(HudiSplitSource.class); private final Configuration conf; private final HudiTableHandle tableHandle; + private final FileSystem fileSystem; private final HoodieTableMetaClient metaClient; - private final HoodieTableFileSystemView fileSystemView; - private final Iterator inputSplitIterator; private final DataSize maxSplitSize; private final List partitionKeys; private final boolean metadataEnabled; private final String tablePath; - private final String dataDir; + private HoodieTableFileSystemView fileSystemView; + private ArrayDeque baseFiles; + private Iterator relativePartitionPaths; public HudiSplitSource( ConnectorSession session, HudiTableHandle tableHandle, Configuration conf, List partitionKeys, + Iterator relativePartitionPaths, boolean metadataEnabled, - InputSplit[] inputSplits, - String tablePath, - String dataDir) + String tablePath) { requireNonNull(session, "session is null"); this.conf = requireNonNull(conf, "conf is null"); this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); this.partitionKeys = requireNonNull(partitionKeys, "partitionKeys is null"); + this.relativePartitionPaths = requireNonNull(relativePartitionPaths, "relativePartitionPaths is null"); this.metadataEnabled = metadataEnabled; this.tablePath = tablePath; - this.dataDir = dataDir; this.metaClient = tableHandle.getMetaClient().orElseGet(() -> getMetaClient(conf, tableHandle.getBasePath())); - HoodieEngineContext engineContext = new HoodieLocalEngineContext(conf); - HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() - .enable(metadataEnabled) - .build(); - this.fileSystemView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, metaClient, metadataConfig); - - log.debug("Table path: %s \nDirectory: %s", tablePath, dataDir); - String partition = FSUtils.getRelativePartitionPath(new Path(tablePath), new Path(dataDir)); - log.debug("Partition: %s", partition); - this.inputSplitIterator = Arrays.stream(inputSplits).iterator(); + this.fileSystem = metaClient.getFs(); + log.debug("Table path: %s \nDirectory: %s", tablePath); this.maxSplitSize = DataSize.ofBytes(32 * 1024 * 1024); } @@ -96,28 +88,13 @@ public HudiSplitSource( public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) { log.debug("Getting next batch with partitionKeys: " + partitionKeys); - List connectorSplits = new ArrayList<>(); - Iterator batchSplitIterator = limit(inputSplitIterator, maxSize); - while (batchSplitIterator.hasNext()) { - FileSplit fileSplit = (FileSplit) batchSplitIterator.next(); - log.debug(String.format(">>>> File split: %s start=%d len=%d", - fileSplit.getPath(), fileSplit.getStart(), fileSplit.getLength())); - try { - connectorSplits.add(new HudiSplit( - fileSplit.getPath().toString(), - fileSplit.getStart(), - fileSplit.getLength(), - metaClient.getFs().getLength(fileSplit.getPath()), - ImmutableList.of(), - tableHandle.getPredicate(), - partitionKeys)); - } - catch (IOException e) { - log.error("Error getting file size: " + e.getMessage()); - } + try { + List connectorSplits = getSplitsForSnapshotMode(maxSize); + return completedFuture(new ConnectorSplitBatch(connectorSplits, isFinished())); + } + catch (IOException e) { + throw new HoodieIOException("Failed to get next batch of splits", e); } - - return completedFuture(new ConnectorSplitBatch(connectorSplits, isFinished())); } @Override @@ -129,6 +106,66 @@ public void close() @Override public boolean isFinished() { - return !inputSplitIterator.hasNext(); + return !relativePartitionPaths.hasNext() && baseFiles.isEmpty(); + } + + private List getSplitsForSnapshotMode(int maxSize) throws IOException + { + if (this.fileSystemView == null) { + // First time calling this + // Load the timeline and file status only once + HoodieEngineContext engineContext = new HoodieLocalEngineContext(conf); + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() + .enable(metadataEnabled) + .build(); + // Scan the file system to load the instants from timeline + log.debug("Loading file system view for " + metaClient.getBasePath()); + this.fileSystemView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, metaClient, metadataConfig); + this.baseFiles = new ArrayDeque<>(); + } + + List batchHudiSplits = new ArrayList<>(); + int remaining = maxSize; + + log.debug("Target number of splits: " + maxSize); + + while (remaining > 0 && !isFinished()) { + if (baseFiles.isEmpty()) { + if (relativePartitionPaths.hasNext()) { + String relativePartitionPath = relativePartitionPaths.next(); + // TODO: skip partitions that are filtered out based on the predicate + baseFiles.addAll(fileSystemView.getLatestBaseFiles(relativePartitionPath) + .collect(Collectors.toList())); + } + } + + while (remaining > 0 && !baseFiles.isEmpty()) { + HoodieBaseFile baseFile = baseFiles.pollFirst(); + log.debug(String.format("Remaining: %d base file: %s", remaining, baseFile.getPath())); + List fileSplits = HudiUtil.getSplits(fileSystem, baseFile.getFileStatus()); + fileSplits.forEach(fileSplit -> { + try { + log.debug(String.format(">>>> File split: %s start=%d len=%d", + fileSplit.getPath(), fileSplit.getStart(), fileSplit.getLength())); + batchHudiSplits.add(new HudiSplit( + fileSplit.getPath().toString(), + fileSplit.getStart(), + fileSplit.getLength(), + metaClient.getFs().getLength(fileSplit.getPath()), + ImmutableList.of(), + tableHandle.getPredicate(), + partitionKeys)); + } + catch (IOException e) { + throw new HoodieIOException("Unable to add splits for " + fileSplit.getPath().toString(), e); + } + }); + remaining -= fileSplits.size(); + } + } + + log.info("Number of Hudi splits generated in the batch: " + batchHudiSplits.size()); + + return batchHudiSplits; } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java index 4ec566f1424..ca40900ffb4 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java @@ -34,8 +34,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.InputFormat; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.net.NetworkTopology; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.hadoop.HoodieParquetInputFormat; @@ -63,7 +61,6 @@ import static java.lang.Float.parseFloat; import static java.lang.Long.parseLong; import static java.lang.String.format; -import static org.apache.hadoop.mapred.FileInputFormat.NUM_INPUT_FILES; public class HudiUtil { @@ -160,90 +157,84 @@ public static Object convertPartitionValue( } } - public static InputSplit[] getSplits(FileStatus[] fileStatuses, JobConf job, int numSplits, - HoodieParquetInputFormat inputFormat) throws IOException { - // Save the number of input files for metrics/loadgen - job.setLong(NUM_INPUT_FILES, fileStatuses.length); - long totalSize = 0; // compute total size - for (FileStatus file: fileStatuses) { // check we have valid files - if (file.isDirectory()) { - throw new IOException("Not a file: " + file.getPath()); - } - totalSize += file.getLen(); + public static List getSplits(FileSystem fs, FileStatus fileStatus) throws IOException + { + if (fileStatus.isDirectory()) { + throw new IOException("Not a file: " + fileStatus.getPath()); } - long goalSize = totalSize / (numSplits == 0 ? 1 : numSplits); - long minSize = Math.max(job.getLong(org.apache.hadoop.mapreduce.lib.input. - FileInputFormat.SPLIT_MINSIZE, 1), 1); + Path path = fileStatus.getPath(); + long length = fileStatus.getLen(); // generate splits - ArrayList splits = new ArrayList(numSplits); + List splits = new ArrayList<>(); NetworkTopology clusterMap = new NetworkTopology(); - for (FileStatus file: fileStatuses) { - Path path = file.getPath(); - long length = file.getLen(); - if (length != 0) { - FileSystem fs = path.getFileSystem(job); - BlockLocation[] blkLocations; - if (file instanceof LocatedFileStatus) { - blkLocations = ((LocatedFileStatus) file).getBlockLocations(); - } else { - blkLocations = fs.getFileBlockLocations(file, 0, length); - } - if (isSplitable(fs, path)) { - long blockSize = file.getBlockSize(); - long splitSize = computeSplitSize(goalSize, minSize, blockSize); + if (length != 0) { + BlockLocation[] blkLocations; + if (fileStatus instanceof LocatedFileStatus) { + blkLocations = ((LocatedFileStatus) fileStatus).getBlockLocations(); + } + else { + blkLocations = fs.getFileBlockLocations(fileStatus, 0, length); + } + if (isSplitable(fs, path)) { + long splitSize = fileStatus.getBlockSize(); - long bytesRemaining = length; - while (((double) bytesRemaining)/splitSize > SPLIT_SLOP) { - String[][] splitHosts = getSplitHostsAndCachedHosts(blkLocations, - length-bytesRemaining, splitSize, clusterMap); - splits.add(makeSplit(path, length-bytesRemaining, splitSize, - splitHosts[0], splitHosts[1])); - bytesRemaining -= splitSize; - } + long bytesRemaining = length; + while (((double) bytesRemaining) / splitSize > SPLIT_SLOP) { + String[][] splitHosts = getSplitHostsAndCachedHosts(blkLocations, + length - bytesRemaining, splitSize, clusterMap); + splits.add(makeSplit(path, length - bytesRemaining, splitSize, + splitHosts[0], splitHosts[1])); + bytesRemaining -= splitSize; + } - if (bytesRemaining != 0) { - String[][] splitHosts = getSplitHostsAndCachedHosts(blkLocations, length - - bytesRemaining, bytesRemaining, clusterMap); - splits.add(makeSplit(path, length - bytesRemaining, bytesRemaining, - splitHosts[0], splitHosts[1])); - } - } else { - String[][] splitHosts = getSplitHostsAndCachedHosts(blkLocations,0,length,clusterMap); - splits.add(makeSplit(path, 0, length, splitHosts[0], splitHosts[1])); + if (bytesRemaining != 0) { + String[][] splitHosts = getSplitHostsAndCachedHosts(blkLocations, length + - bytesRemaining, bytesRemaining, clusterMap); + splits.add(makeSplit(path, length - bytesRemaining, bytesRemaining, + splitHosts[0], splitHosts[1])); } - } else { - //Create empty hosts array for zero length files - splits.add(makeSplit(path, 0, length, new String[0])); + } + else { + String[][] splitHosts = getSplitHostsAndCachedHosts(blkLocations, 0, length, clusterMap); + splits.add(makeSplit(path, 0, length, splitHosts[0], splitHosts[1])); } } - return splits.toArray(new FileSplit[splits.size()]); + else { + //Create empty hosts array for zero length files + splits.add(makeSplit(path, 0, length, new String[0])); + } + return splits; } - private static boolean isSplitable(FileSystem fs, Path filename) { + private static boolean isSplitable(FileSystem fs, Path filename) + { return !(filename instanceof PathWithBootstrapFileStatus); } private static long computeSplitSize(long goalSize, long minSize, - long blockSize) { + long blockSize) + { return Math.max(minSize, Math.min(goalSize, blockSize)); } private static FileSplit makeSplit(Path file, long start, long length, - String[] hosts) { + String[] hosts) + { return new FileSplit(file, start, length, hosts); } private static FileSplit makeSplit(Path file, long start, long length, - String[] hosts, String[] inMemoryHosts) { + String[] hosts, String[] inMemoryHosts) + { return new FileSplit(file, start, length, hosts, inMemoryHosts); } private static String[][] getSplitHostsAndCachedHosts(BlockLocation[] blkLocations, long offset, long splitSize, NetworkTopology clusterMap) - throws IOException { - + throws IOException + { int startIndex = getBlockIndex(blkLocations, offset); return new String[][]{blkLocations[startIndex].getHosts(), @@ -251,16 +242,17 @@ private static String[][] getSplitHostsAndCachedHosts(BlockLocation[] blkLocatio } private static int getBlockIndex(BlockLocation[] blkLocations, - long offset) { - for (int i = 0 ; i < blkLocations.length; i++) { + long offset) + { + for (int i = 0; i < blkLocations.length; i++) { // is the offset inside this block? if ((blkLocations[i].getOffset() <= offset) && - (offset < blkLocations[i].getOffset() + blkLocations[i].getLength())){ + (offset < blkLocations[i].getOffset() + blkLocations[i].getLength())) { return i; } } - BlockLocation last = blkLocations[blkLocations.length -1]; - long fileLength = last.getOffset() + last.getLength() -1; + BlockLocation last = blkLocations[blkLocations.length - 1]; + long fileLength = last.getOffset() + last.getLength() - 1; throw new IllegalArgumentException("Offset " + offset + " is outside of file (0.." + fileLength + ")"); From bad991cc02052eb993e1af67ddf8a738d1002b83 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Mon, 20 Dec 2021 23:07:48 -0800 Subject: [PATCH 14/26] Remove unused variables --- .../trino/plugin/hudi/HudiSplitManager.java | 41 +------------------ .../io/trino/plugin/hudi/HudiSplitSource.java | 13 ++---- 2 files changed, 5 insertions(+), 49 deletions(-) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index f58020b19ad..bdf70e9428b 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -36,13 +36,11 @@ import io.trino.spi.connector.TableNotFoundException; import io.trino.spi.predicate.TupleDomain; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hudi.common.table.HoodieTableMetaClient; import javax.inject.Inject; -import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -91,13 +89,7 @@ public ConnectorSplitSource getSplits( Table table = metastore.getTable(identity, tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(tableName)); HdfsEnvironment.HdfsContext context = new HdfsEnvironment.HdfsContext(session); - FileSystem fs = null; - try { - fs = hdfsEnvironment.getFileSystem(context, new Path(table.getStorage().getLocation())); - } - catch (IOException e) { - e.printStackTrace(); - } + Configuration conf = hdfsEnvironment.getConfiguration(context, new Path(table.getStorage().getLocation())); Map valByRegex = conf.getValByRegex(HOODIE_CONSUME_MODE_PATTERN_STRING.pattern()); log.debug("Hoodie consume mode: " + valByRegex); @@ -115,7 +107,6 @@ public ConnectorSplitSource getSplits( .collect(toImmutableList()); log.warn("Column Names: " + columnNames); HudiSplitSource splitSource; - String tablePath = table.getStorage().getLocation(); List partitionKeys = ImmutableList.of(); Properties schema; Iterator relativePartitionPaths; @@ -135,11 +126,6 @@ public ConnectorSplitSource getSplits( partitionKeys = getPartitionKeys(table, partition); partitionKeys.forEach(p -> log.warn("Fetched partitions from HiveUtil: " + p)); - - // InputFormat inputFormat = HiveUtil.getInputFormat(conf, schema, false); - // log.debug(">>> Check for inputFormat: " + isHudiParquetInputFormat(inputFormat)); - // log.debug(">>> Conf: "); - // printConf(conf); } else { // no partitions, so data dir is same as table path @@ -147,32 +133,9 @@ public ConnectorSplitSource getSplits( partitionValues = ImmutableList.of(""); } - /* - // Non Hudi table should also be compatible with HoodieParquetInputFormat - HoodieParquetInputFormat inputFormat = new HoodieParquetInputFormat(); - inputFormat.setConf(conf); - - JobConf jobConf = toJobConf(conf); - String allAbsolutePartitionPaths = partitionValues.stream().map( - relativePartitionPath -> FSUtils.getPartitionPath(tablePath, relativePartitionPath).toString() - ).collect(Collectors.joining(",")); - FileInputFormat.setInputPaths(jobConf, allAbsolutePartitionPaths); - // Pass SerDes and Table parameters into input format configuration - fromProperties(schema).forEach(jobConf::set); - String dataDir = schema.getProperty(META_TABLE_LOCATION); - try { - InputSplit[] inputSplits = inputFormat.getSplits(jobConf, 0); - log.warn(">>> Total Splits: " + inputSplits.length); - splitSource = new HudiSplitSource(session, hudiTable, conf, partitionKeys, isHudiMetadataEnabled(session), inputSplits, tablePath, dataDir); - return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); - } - catch (IOException e) { - throw new HoodieIOException("Error getting input splits", e); - } - */ relativePartitionPaths = partitionValues.iterator(); splitSource = new HudiSplitSource(session, hudiTable, conf, partitionKeys, relativePartitionPaths, - isHudiMetadataEnabled(session), tablePath); + isHudiMetadataEnabled(session)); return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index 9fd3172dae9..4dce003c447 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -16,7 +16,6 @@ import com.google.common.collect.ImmutableList; import io.airlift.log.Logger; -import io.airlift.units.DataSize; import io.trino.plugin.hive.HivePartitionKey; import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorSession; @@ -54,13 +53,11 @@ public class HudiSplitSource private final HudiTableHandle tableHandle; private final FileSystem fileSystem; private final HoodieTableMetaClient metaClient; - private final DataSize maxSplitSize; private final List partitionKeys; private final boolean metadataEnabled; - private final String tablePath; + private final Iterator relativePartitionPaths; private HoodieTableFileSystemView fileSystemView; private ArrayDeque baseFiles; - private Iterator relativePartitionPaths; public HudiSplitSource( ConnectorSession session, @@ -68,8 +65,7 @@ public HudiSplitSource( Configuration conf, List partitionKeys, Iterator relativePartitionPaths, - boolean metadataEnabled, - String tablePath) + boolean metadataEnabled) { requireNonNull(session, "session is null"); this.conf = requireNonNull(conf, "conf is null"); @@ -77,11 +73,8 @@ public HudiSplitSource( this.partitionKeys = requireNonNull(partitionKeys, "partitionKeys is null"); this.relativePartitionPaths = requireNonNull(relativePartitionPaths, "relativePartitionPaths is null"); this.metadataEnabled = metadataEnabled; - this.tablePath = tablePath; this.metaClient = tableHandle.getMetaClient().orElseGet(() -> getMetaClient(conf, tableHandle.getBasePath())); this.fileSystem = metaClient.getFs(); - log.debug("Table path: %s \nDirectory: %s", tablePath); - this.maxSplitSize = DataSize.ofBytes(32 * 1024 * 1024); } @Override @@ -100,7 +93,7 @@ public CompletableFuture getNextBatch(ConnectorPartitionHan @Override public void close() { - // TODO: close file iterable + fileSystemView.close(); } @Override From 068f4a6a5e00a2bf3d010632cdfb0a64bf02328a Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Mon, 20 Dec 2021 23:56:49 -0800 Subject: [PATCH 15/26] Fix partition keys --- .../trino/plugin/hudi/HudiSplitManager.java | 39 +++++++------------ .../io/trino/plugin/hudi/HudiSplitSource.java | 26 +++++++------ 2 files changed, 30 insertions(+), 35 deletions(-) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index bdf70e9428b..f7ad90dd48a 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -22,7 +22,6 @@ import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.Partition; import io.trino.plugin.hive.metastore.Table; import io.trino.plugin.hive.util.HiveUtil; import io.trino.spi.connector.ConnectorSession; @@ -41,18 +40,18 @@ import javax.inject.Inject; -import java.util.Iterator; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; -import java.util.Properties; +import java.util.function.Function; import java.util.regex.Pattern; +import java.util.stream.Collectors; import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeys; import static io.trino.plugin.hudi.HudiSessionProperties.isHudiMetadataEnabled; import static io.trino.plugin.hudi.HudiUtil.getMetaClient; -import static io.trino.plugin.hudi.HudiUtil.getPartitionSchema; import static java.util.Objects.requireNonNull; import static org.apache.hudi.common.table.timeline.TimelineUtils.getPartitionsWritten; @@ -107,34 +106,26 @@ public ConnectorSplitSource getSplits( .collect(toImmutableList()); log.warn("Column Names: " + columnNames); HudiSplitSource splitSource; - List partitionKeys = ImmutableList.of(); - Properties schema; - Iterator relativePartitionPaths; + Map> partitionMap; if (!columnNames.isEmpty()) { - List> partitionNames = metastore.getPartitionNamesByFilter(identity, tableName.getSchemaName(), tableName.getTableName(), columnNames, TupleDomain.all()) - .orElseThrow(() -> new TableNotFoundException(hudiTable.getSchemaTableName())) - .stream() - .map(HiveUtil::toPartitionValues) - .collect(toImmutableList()); - log.warn("Partition Names: " + partitionNames); + List rawPartitionNames = metastore.getPartitionNamesByFilter(identity, tableName.getSchemaName(), tableName.getTableName(), columnNames, TupleDomain.all()) + .orElseThrow(() -> new TableNotFoundException(hudiTable.getSchemaTableName())); - Optional partition = metastore.getPartition(identity, table, partitionNames.get(0)); - schema = getPartitionSchema(table, partition); + log.warn("rawPartitionNames: " + rawPartitionNames); - log.warn("Fetched partitions from Metastore: " + partition.get()); - log.warn("Partition schema: " + schema); - - partitionKeys = getPartitionKeys(table, partition); - partitionKeys.forEach(p -> log.warn("Fetched partitions from HiveUtil: " + p)); + // relative partition path -> Hive partition + partitionMap = rawPartitionNames.stream() + .collect(Collectors.toMap(Function.identity(), relativePartitionPath -> getPartitionKeys(table, + metastore.getPartition(identity, table, HiveUtil.toPartitionValues(relativePartitionPath))))); } else { // no partitions, so data dir is same as table path - schema = getPartitionSchema(table, Optional.empty()); partitionValues = ImmutableList.of(""); + partitionMap = new HashMap<>(); + partitionMap.put("", new ArrayList<>()); } - relativePartitionPaths = partitionValues.iterator(); - splitSource = new HudiSplitSource(session, hudiTable, conf, partitionKeys, relativePartitionPaths, + splitSource = new HudiSplitSource(session, hudiTable, conf, partitionMap, isHudiMetadataEnabled(session)); return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index 4dce003c447..82622756301 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -36,8 +36,10 @@ import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.HashMap; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; @@ -53,34 +55,35 @@ public class HudiSplitSource private final HudiTableHandle tableHandle; private final FileSystem fileSystem; private final HoodieTableMetaClient metaClient; - private final List partitionKeys; + private final Map> partitionMap; private final boolean metadataEnabled; private final Iterator relativePartitionPaths; + private final Map baseFileToPartitionMap; + private final ArrayDeque baseFiles = new ArrayDeque<>(); private HoodieTableFileSystemView fileSystemView; - private ArrayDeque baseFiles; public HudiSplitSource( ConnectorSession session, HudiTableHandle tableHandle, Configuration conf, - List partitionKeys, - Iterator relativePartitionPaths, + Map> partitionMap, boolean metadataEnabled) { requireNonNull(session, "session is null"); this.conf = requireNonNull(conf, "conf is null"); this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); - this.partitionKeys = requireNonNull(partitionKeys, "partitionKeys is null"); - this.relativePartitionPaths = requireNonNull(relativePartitionPaths, "relativePartitionPaths is null"); + this.partitionMap = partitionMap; + this.relativePartitionPaths = requireNonNull(partitionMap.keySet().iterator(), "relativePartitionPaths is null"); this.metadataEnabled = metadataEnabled; this.metaClient = tableHandle.getMetaClient().orElseGet(() -> getMetaClient(conf, tableHandle.getBasePath())); this.fileSystem = metaClient.getFs(); + this.baseFileToPartitionMap = new HashMap<>(); } @Override public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) { - log.debug("Getting next batch with partitionKeys: " + partitionKeys); + log.debug("Getting next batch with partitionKeys: " + partitionMap.keySet()); try { List connectorSplits = getSplitsForSnapshotMode(maxSize); return completedFuture(new ConnectorSplitBatch(connectorSplits, isFinished())); @@ -114,7 +117,6 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce // Scan the file system to load the instants from timeline log.debug("Loading file system view for " + metaClient.getBasePath()); this.fileSystemView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, metaClient, metadataConfig); - this.baseFiles = new ArrayDeque<>(); } List batchHudiSplits = new ArrayList<>(); @@ -126,9 +128,11 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce if (baseFiles.isEmpty()) { if (relativePartitionPaths.hasNext()) { String relativePartitionPath = relativePartitionPaths.next(); + List baseFilesToAdd = fileSystemView.getLatestBaseFiles(relativePartitionPath) + .collect(Collectors.toList()); + baseFilesToAdd.forEach(baseFile -> baseFileToPartitionMap.put(baseFile, relativePartitionPath)); // TODO: skip partitions that are filtered out based on the predicate - baseFiles.addAll(fileSystemView.getLatestBaseFiles(relativePartitionPath) - .collect(Collectors.toList())); + baseFiles.addAll(baseFilesToAdd); } } @@ -147,7 +151,7 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce metaClient.getFs().getLength(fileSplit.getPath()), ImmutableList.of(), tableHandle.getPredicate(), - partitionKeys)); + partitionMap.get(baseFileToPartitionMap.get(baseFile)))); } catch (IOException e) { throw new HoodieIOException("Unable to add splits for " + fileSplit.getPath().toString(), e); From 7fc6d47acef0de8fbb36d3ebb43010fcb2c96a73 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Tue, 21 Dec 2021 00:33:44 -0800 Subject: [PATCH 16/26] Use empty partition keys for now --- .../java/io/trino/plugin/hudi/HudiSplitManager.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index f7ad90dd48a..7afd5e4e919 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -18,12 +18,12 @@ import io.airlift.log.Logger; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitSource; import io.trino.plugin.hive.HdfsEnvironment; +import io.trino.plugin.hive.HivePartition; import io.trino.plugin.hive.HivePartitionKey; import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.Table; -import io.trino.plugin.hive.util.HiveUtil; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.ConnectorSplitSource; @@ -33,7 +33,6 @@ import io.trino.spi.connector.DynamicFilter; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.connector.TableNotFoundException; -import io.trino.spi.predicate.TupleDomain; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -44,12 +43,10 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.function.Function; import java.util.regex.Pattern; import java.util.stream.Collectors; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeys; import static io.trino.plugin.hudi.HudiSessionProperties.isHudiMetadataEnabled; import static io.trino.plugin.hudi.HudiUtil.getMetaClient; import static java.util.Objects.requireNonNull; @@ -108,6 +105,7 @@ public ConnectorSplitSource getSplits( HudiSplitSource splitSource; Map> partitionMap; if (!columnNames.isEmpty()) { + /* List rawPartitionNames = metastore.getPartitionNamesByFilter(identity, tableName.getSchemaName(), tableName.getTableName(), columnNames, TupleDomain.all()) .orElseThrow(() -> new TableNotFoundException(hudiTable.getSchemaTableName())); @@ -117,6 +115,10 @@ public ConnectorSplitSource getSplits( partitionMap = rawPartitionNames.stream() .collect(Collectors.toMap(Function.identity(), relativePartitionPath -> getPartitionKeys(table, metastore.getPartition(identity, table, HiveUtil.toPartitionValues(relativePartitionPath))))); + + */ + partitionMap = hudiTable.getPartitions().get().stream() + .collect(Collectors.toMap(HivePartition::getPartitionId, e -> new ArrayList<>())); } else { // no partitions, so data dir is same as table path @@ -125,6 +127,7 @@ public ConnectorSplitSource getSplits( partitionMap.put("", new ArrayList<>()); } + log.debug("Partition map: " + partitionMap); splitSource = new HudiSplitSource(session, hudiTable, conf, partitionMap, isHudiMetadataEnabled(session)); return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); From fd6e4dc2f65a3497942e2b33472f7a4cbcb3f073 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Tue, 21 Dec 2021 08:23:16 -0800 Subject: [PATCH 17/26] Consider bootstrap base file when getting the file status --- .../src/main/java/io/trino/plugin/hudi/HudiSplitSource.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index 82622756301..8ac6077b073 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -32,6 +32,7 @@ import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; import java.io.IOException; import java.util.ArrayDeque; @@ -139,7 +140,9 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce while (remaining > 0 && !baseFiles.isEmpty()) { HoodieBaseFile baseFile = baseFiles.pollFirst(); log.debug(String.format("Remaining: %d base file: %s", remaining, baseFile.getPath())); - List fileSplits = HudiUtil.getSplits(fileSystem, baseFile.getFileStatus()); + + List fileSplits = HudiUtil.getSplits( + fileSystem, HoodieInputFormatUtils.getFileStatus(baseFile)); fileSplits.forEach(fileSplit -> { try { log.debug(String.format(">>>> File split: %s start=%d len=%d", From 7d5e8d51a62a06c18d6c0ea25fa5682f1991f53d Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Tue, 21 Dec 2021 15:58:41 -0800 Subject: [PATCH 18/26] Fix partition keys --- .../trino/plugin/hudi/HudiSplitManager.java | 53 +++++++++++-------- 1 file changed, 30 insertions(+), 23 deletions(-) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index 7afd5e4e919..0bed4fe8d4b 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -18,12 +18,13 @@ import io.airlift.log.Logger; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitSource; import io.trino.plugin.hive.HdfsEnvironment; -import io.trino.plugin.hive.HivePartition; import io.trino.plugin.hive.HivePartitionKey; import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.Partition; import io.trino.plugin.hive.metastore.Table; +import io.trino.plugin.hive.util.HiveUtil; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.ConnectorSplitSource; @@ -33,23 +34,28 @@ import io.trino.spi.connector.DynamicFilter; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.predicate.TupleDomain; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import javax.inject.Inject; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.Properties; import java.util.regex.Pattern; -import java.util.stream.Collectors; import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeys; import static io.trino.plugin.hudi.HudiSessionProperties.isHudiMetadataEnabled; import static io.trino.plugin.hudi.HudiUtil.getMetaClient; +import static io.trino.plugin.hudi.HudiUtil.getPartitionSchema; import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_LOCATION; import static org.apache.hudi.common.table.timeline.TimelineUtils.getPartitionsWritten; public class HudiSplitManager @@ -85,8 +91,8 @@ public ConnectorSplitSource getSplits( Table table = metastore.getTable(identity, tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(tableName)); HdfsEnvironment.HdfsContext context = new HdfsEnvironment.HdfsContext(session); - - Configuration conf = hdfsEnvironment.getConfiguration(context, new Path(table.getStorage().getLocation())); + String tablePath = table.getStorage().getLocation(); + Configuration conf = hdfsEnvironment.getConfiguration(context, new Path(tablePath)); Map valByRegex = conf.getValByRegex(HOODIE_CONSUME_MODE_PATTERN_STRING.pattern()); log.debug("Hoodie consume mode: " + valByRegex); HoodieTableMetaClient metaClient = hudiTable.getMetaClient().orElseGet(() -> getMetaClient(conf, hudiTable.getBasePath())); @@ -103,28 +109,29 @@ public ConnectorSplitSource getSplits( .collect(toImmutableList()); log.warn("Column Names: " + columnNames); HudiSplitSource splitSource; - Map> partitionMap; + Map> partitionMap = new HashMap<>(); if (!columnNames.isEmpty()) { - /* - List rawPartitionNames = metastore.getPartitionNamesByFilter(identity, tableName.getSchemaName(), tableName.getTableName(), columnNames, TupleDomain.all()) - .orElseThrow(() -> new TableNotFoundException(hudiTable.getSchemaTableName())); - - log.warn("rawPartitionNames: " + rawPartitionNames); - - // relative partition path -> Hive partition - partitionMap = rawPartitionNames.stream() - .collect(Collectors.toMap(Function.identity(), relativePartitionPath -> getPartitionKeys(table, - metastore.getPartition(identity, table, HiveUtil.toPartitionValues(relativePartitionPath))))); - - */ - partitionMap = hudiTable.getPartitions().get().stream() - .collect(Collectors.toMap(HivePartition::getPartitionId, e -> new ArrayList<>())); + List> partitionNames = metastore.getPartitionNamesByFilter(identity, tableName.getSchemaName(), tableName.getTableName(), columnNames, TupleDomain.all()) + .orElseThrow(() -> new TableNotFoundException(hudiTable.getSchemaTableName())) + .stream() + .map(HiveUtil::toPartitionValues) + .collect(toImmutableList()); + log.warn("Partition Names: " + partitionNames); + + for (List partitionName : partitionNames) { + Optional partition1 = metastore.getPartition(identity, table, partitionName); + Properties schema1 = getPartitionSchema(table, partition1); + String dataDir1 = schema1.getProperty(META_TABLE_LOCATION); + log.warn(">>> basePath: %s, dataDir1: %s", tablePath, dataDir1); + String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(tablePath), new Path(dataDir1)); + List partitionKeys1 = getPartitionKeys(table, partition1); + partitionMap.putIfAbsent(relativePartitionPath, partitionKeys1); + partitionKeys1.forEach(p -> log.warn(">>> Fetched partitions from HiveUtil: " + p)); + } } else { // no partitions, so data dir is same as table path - partitionValues = ImmutableList.of(""); - partitionMap = new HashMap<>(); - partitionMap.put("", new ArrayList<>()); + partitionMap.put("", ImmutableList.of()); } log.debug("Partition map: " + partitionMap); From ae1f0eaefb78bb6800804493cee38d8f4cde35e5 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Tue, 21 Dec 2021 22:38:00 -0800 Subject: [PATCH 19/26] Add argument for dynamic filter --- .../main/java/io/trino/plugin/hudi/HudiSplitManager.java | 2 +- .../main/java/io/trino/plugin/hudi/HudiSplitSource.java | 7 ++++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index 0bed4fe8d4b..a18bf31b59f 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -136,7 +136,7 @@ public ConnectorSplitSource getSplits( log.debug("Partition map: " + partitionMap); splitSource = new HudiSplitSource(session, hudiTable, conf, partitionMap, - isHudiMetadataEnabled(session)); + isHudiMetadataEnabled(session), dynamicFilter); return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index 8ac6077b073..6c079bb7fe3 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -21,6 +21,7 @@ import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitSource; +import io.trino.spi.connector.DynamicFilter; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.mapred.FileSplit; @@ -61,6 +62,7 @@ public class HudiSplitSource private final Iterator relativePartitionPaths; private final Map baseFileToPartitionMap; private final ArrayDeque baseFiles = new ArrayDeque<>(); + private final DynamicFilter dynamicFilter; private HoodieTableFileSystemView fileSystemView; public HudiSplitSource( @@ -68,7 +70,8 @@ public HudiSplitSource( HudiTableHandle tableHandle, Configuration conf, Map> partitionMap, - boolean metadataEnabled) + boolean metadataEnabled, + DynamicFilter dynamicFilter) { requireNonNull(session, "session is null"); this.conf = requireNonNull(conf, "conf is null"); @@ -79,11 +82,13 @@ public HudiSplitSource( this.metaClient = tableHandle.getMetaClient().orElseGet(() -> getMetaClient(conf, tableHandle.getBasePath())); this.fileSystem = metaClient.getFs(); this.baseFileToPartitionMap = new HashMap<>(); + this.dynamicFilter = dynamicFilter; } @Override public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) { + log.debug("Dynamic filter: " + dynamicFilter.getColumnsCovered()); log.debug("Getting next batch with partitionKeys: " + partitionMap.keySet()); try { List connectorSplits = getSplitsForSnapshotMode(maxSize); From 1f478e06cdb8a90be8621eb99f1ed223e80e1a1a Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Wed, 22 Dec 2021 22:11:56 -0800 Subject: [PATCH 20/26] Add batch processing in split source --- .../trino/plugin/hudi/HudiSplitManager.java | 51 +---- .../io/trino/plugin/hudi/HudiSplitSource.java | 181 ++++++++++++++---- 2 files changed, 142 insertions(+), 90 deletions(-) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index a18bf31b59f..13bc7ecf3de 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -14,17 +14,12 @@ package io.trino.plugin.hudi; -import com.google.common.collect.ImmutableList; import io.airlift.log.Logger; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitSource; import io.trino.plugin.hive.HdfsEnvironment; -import io.trino.plugin.hive.HivePartitionKey; import io.trino.plugin.hive.authentication.HiveIdentity; -import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.Partition; import io.trino.plugin.hive.metastore.Table; -import io.trino.plugin.hive.util.HiveUtil; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.ConnectorSplitSource; @@ -34,29 +29,18 @@ import io.trino.spi.connector.DynamicFilter; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.connector.TableNotFoundException; -import io.trino.spi.predicate.TupleDomain; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import javax.inject.Inject; -import java.util.HashMap; -import java.util.List; import java.util.Map; -import java.util.Optional; -import java.util.Properties; import java.util.regex.Pattern; -import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeys; import static io.trino.plugin.hudi.HudiSessionProperties.isHudiMetadataEnabled; import static io.trino.plugin.hudi.HudiUtil.getMetaClient; -import static io.trino.plugin.hudi.HudiUtil.getPartitionSchema; import static java.util.Objects.requireNonNull; -import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_LOCATION; -import static org.apache.hudi.common.table.timeline.TimelineUtils.getPartitionsWritten; public class HudiSplitManager implements ConnectorSplitManager @@ -96,46 +80,13 @@ public ConnectorSplitSource getSplits( Map valByRegex = conf.getValByRegex(HOODIE_CONSUME_MODE_PATTERN_STRING.pattern()); log.debug("Hoodie consume mode: " + valByRegex); HoodieTableMetaClient metaClient = hudiTable.getMetaClient().orElseGet(() -> getMetaClient(conf, hudiTable.getBasePath())); - List partitionValues = getPartitionsWritten(metaClient.getActiveTimeline()); log.debug("HudiSplitManager ref: " + this.toString()); log.debug("Table ref: " + table.toString()); log.debug("HoodieTableMetaClient ref: " + metaClient.toString()); log.debug("HoodieTableMetaClient base path: " + metaClient.getBasePath()); - log.warn("Fetched partitions from Hudi: " + partitionValues); hudiTable.getPartitions().ifPresent(p -> p.forEach(p1 -> log.warn("Partitions from TableHandle: " + p1))); - List columnNames = table.getPartitionColumns().stream() - .map(Column::getName) - .collect(toImmutableList()); - log.warn("Column Names: " + columnNames); - HudiSplitSource splitSource; - Map> partitionMap = new HashMap<>(); - if (!columnNames.isEmpty()) { - List> partitionNames = metastore.getPartitionNamesByFilter(identity, tableName.getSchemaName(), tableName.getTableName(), columnNames, TupleDomain.all()) - .orElseThrow(() -> new TableNotFoundException(hudiTable.getSchemaTableName())) - .stream() - .map(HiveUtil::toPartitionValues) - .collect(toImmutableList()); - log.warn("Partition Names: " + partitionNames); - - for (List partitionName : partitionNames) { - Optional partition1 = metastore.getPartition(identity, table, partitionName); - Properties schema1 = getPartitionSchema(table, partition1); - String dataDir1 = schema1.getProperty(META_TABLE_LOCATION); - log.warn(">>> basePath: %s, dataDir1: %s", tablePath, dataDir1); - String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(tablePath), new Path(dataDir1)); - List partitionKeys1 = getPartitionKeys(table, partition1); - partitionMap.putIfAbsent(relativePartitionPath, partitionKeys1); - partitionKeys1.forEach(p -> log.warn(">>> Fetched partitions from HiveUtil: " + p)); - } - } - else { - // no partitions, so data dir is same as table path - partitionMap.put("", ImmutableList.of()); - } - - log.debug("Partition map: " + partitionMap); - splitSource = new HudiSplitSource(session, hudiTable, conf, partitionMap, + HudiSplitSource splitSource = new HudiSplitSource(identity, metastore, hudiTable, conf, isHudiMetadataEnabled(session), dynamicFilter); return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index 6c079bb7fe3..9b78f90cb42 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -15,36 +15,53 @@ package io.trino.plugin.hudi; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterators; import io.airlift.log.Logger; import io.trino.plugin.hive.HivePartitionKey; +import io.trino.plugin.hive.authentication.HiveIdentity; +import io.trino.plugin.hive.metastore.Column; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.Partition; +import io.trino.plugin.hive.metastore.Table; +import io.trino.plugin.hive.util.HiveUtil; import io.trino.spi.connector.ConnectorPartitionHandle; -import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitSource; import io.trino.spi.connector.DynamicFilter; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.predicate.TupleDomain; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.fs.Path; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeys; import static io.trino.plugin.hudi.HudiUtil.getMetaClient; import static java.util.Objects.requireNonNull; import static java.util.concurrent.CompletableFuture.completedFuture; @@ -53,31 +70,39 @@ public class HudiSplitSource implements ConnectorSplitSource { private static final Logger log = Logger.get(HudiSplitSource.class); + private final HiveIdentity identity; + private final HiveMetastore metastore; + private final SchemaTableName tableName; private final Configuration conf; private final HudiTableHandle tableHandle; + private final Table table; private final FileSystem fileSystem; private final HoodieTableMetaClient metaClient; private final Map> partitionMap; private final boolean metadataEnabled; - private final Iterator relativePartitionPaths; private final Map baseFileToPartitionMap; private final ArrayDeque baseFiles = new ArrayDeque<>(); private final DynamicFilter dynamicFilter; + private final int partitionBatchNum = 100; private HoodieTableFileSystemView fileSystemView; + private Iterator> partitionNames; public HudiSplitSource( - ConnectorSession session, + HiveIdentity identity, + HiveMetastore metastore, HudiTableHandle tableHandle, Configuration conf, - Map> partitionMap, boolean metadataEnabled, DynamicFilter dynamicFilter) { - requireNonNull(session, "session is null"); + this.identity = identity; + this.metastore = metastore; + this.tableName = tableHandle.getSchemaTableName(); + this.table = metastore.getTable(identity, tableName.getSchemaName(), tableName.getTableName()) + .orElseThrow(() -> new TableNotFoundException(tableName)); this.conf = requireNonNull(conf, "conf is null"); this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); - this.partitionMap = partitionMap; - this.relativePartitionPaths = requireNonNull(partitionMap.keySet().iterator(), "relativePartitionPaths is null"); + this.partitionMap = new HashMap<>(); this.metadataEnabled = metadataEnabled; this.metaClient = tableHandle.getMetaClient().orElseGet(() -> getMetaClient(conf, tableHandle.getBasePath())); this.fileSystem = metaClient.getFs(); @@ -108,12 +133,13 @@ public void close() @Override public boolean isFinished() { - return !relativePartitionPaths.hasNext() && baseFiles.isEmpty(); + return !partitionNames.hasNext() && baseFiles.isEmpty(); } private List getSplitsForSnapshotMode(int maxSize) throws IOException { if (this.fileSystemView == null) { + HoodieTimer timer = new HoodieTimer().startTimer(); // First time calling this // Load the timeline and file status only once HoodieEngineContext engineContext = new HoodieLocalEngineContext(conf); @@ -123,6 +149,28 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce // Scan the file system to load the instants from timeline log.debug("Loading file system view for " + metaClient.getBasePath()); this.fileSystemView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, metaClient, metadataConfig); + log.warn(String.format("Finish in %d ms to load table view", timer.endTimer())); + + timer = new HoodieTimer().startTimer(); + List columnNames = table.getPartitionColumns().stream() + .map(Column::getName) + .collect(toImmutableList()); + log.warn("Column Names: " + columnNames); + HudiSplitSource splitSource; + Map> partitionMap = new HashMap<>(); + if (!columnNames.isEmpty()) { + partitionNames = metastore.getPartitionNamesByFilter(identity, tableName.getSchemaName(), tableName.getTableName(), columnNames, TupleDomain.all()) + .orElseThrow(() -> new TableNotFoundException(tableHandle.getSchemaTableName())) + .stream() + .map(HiveUtil::toPartitionValues) + .collect(toImmutableList()).iterator(); + } + else { + // no partitions, so data dir is same as table path + partitionNames = Collections.singletonList(Collections.singletonList("")).iterator(); + } + + log.warn(String.format("Finish in %d ms. Partition Names: %s", timer.endTimer(), partitionNames.toString())); } List batchHudiSplits = new ArrayList<>(); @@ -130,47 +178,100 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce log.debug("Target number of splits: " + maxSize); - while (remaining > 0 && !isFinished()) { + // Only process one batch now + if (remaining > 0 && !isFinished()) { if (baseFiles.isEmpty()) { - if (relativePartitionPaths.hasNext()) { - String relativePartitionPath = relativePartitionPaths.next(); - List baseFilesToAdd = fileSystemView.getLatestBaseFiles(relativePartitionPath) - .collect(Collectors.toList()); - baseFilesToAdd.forEach(baseFile -> baseFileToPartitionMap.put(baseFile, relativePartitionPath)); - // TODO: skip partitions that are filtered out based on the predicate - baseFiles.addAll(baseFilesToAdd); - } + HoodieTimer timer1 = new HoodieTimer().startTimer(); + + List> batchPartitionNames = new ArrayList<>(); + Iterators.limit(partitionNames, partitionBatchNum).forEachRemaining(batchPartitionNames::add); + + Map> batchKeyMap = + batchPartitionNames.stream().parallel() + .map(partitionNames -> getPartitionPathToKey( + identity, metastore, table, table.getStorage().getLocation(), partitionNames)) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + partitionMap.putAll(batchKeyMap); + + log.warn(String.format("Finish in %d ms to get partition keys", timer1.endTimer())); + + timer1 = new HoodieTimer().startTimer(); + List> baseFilesToAdd = batchKeyMap.keySet().stream().parallel() + .flatMap(relativePartitionPath -> fileSystemView.getLatestBaseFiles(relativePartitionPath) + .map(baseFile -> new ImmutablePair<>(baseFile, relativePartitionPath)) + .collect(Collectors.toList()).stream()) + .collect(Collectors.toList()); + + baseFilesToAdd.forEach(e -> baseFileToPartitionMap.put(e.getKey(), e.getValue())); + // TODO: skip partitions that are filtered out based on the predicate + baseFiles.addAll(baseFilesToAdd.stream().map(Pair::getKey).collect(Collectors.toList())); + log.warn(String.format("Finish in %d ms to get base files", timer1.endTimer())); } + HoodieTimer timer = new HoodieTimer().startTimer(); + List batchBaseFiles = new ArrayList<>(); while (remaining > 0 && !baseFiles.isEmpty()) { HoodieBaseFile baseFile = baseFiles.pollFirst(); - log.debug(String.format("Remaining: %d base file: %s", remaining, baseFile.getPath())); - - List fileSplits = HudiUtil.getSplits( - fileSystem, HoodieInputFormatUtils.getFileStatus(baseFile)); - fileSplits.forEach(fileSplit -> { - try { - log.debug(String.format(">>>> File split: %s start=%d len=%d", - fileSplit.getPath(), fileSplit.getStart(), fileSplit.getLength())); - batchHudiSplits.add(new HudiSplit( - fileSplit.getPath().toString(), - fileSplit.getStart(), - fileSplit.getLength(), - metaClient.getFs().getLength(fileSplit.getPath()), - ImmutableList.of(), - tableHandle.getPredicate(), - partitionMap.get(baseFileToPartitionMap.get(baseFile)))); - } - catch (IOException e) { - throw new HoodieIOException("Unable to add splits for " + fileSplit.getPath().toString(), e); - } - }); - remaining -= fileSplits.size(); + batchBaseFiles.add(baseFile); + remaining--; } + + List hudiSplitsToAdd = batchBaseFiles.stream().parallel() + .flatMap(baseFile -> { + List hudiSplits = new ArrayList<>(); + try { + hudiSplits = HudiUtil.getSplits( + fileSystem, HoodieInputFormatUtils.getFileStatus(baseFile)) + .stream() + .flatMap(fileSplit -> { + List result = new ArrayList<>(); + try { + result.add(new HudiSplit( + fileSplit.getPath().toString(), + fileSplit.getStart(), + fileSplit.getLength(), + metaClient.getFs().getLength(fileSplit.getPath()), + ImmutableList.of(), + tableHandle.getPredicate(), + partitionMap.get(baseFileToPartitionMap.get(baseFile)))); + } + catch (IOException e) { + throw new HoodieIOException(String.format( + "Unable to get Hudi split for %s, start=%d len=%d", + baseFile.getPath(), fileSplit.getStart(), fileSplit.getLength()), e); + } + return result.stream(); + }) + .collect(Collectors.toList()); + } + catch (IOException e) { + throw new HoodieIOException("Unable to get splits for " + baseFile.getPath(), e); + } + return hudiSplits.stream(); + }) + .collect(Collectors.toList()); + batchHudiSplits.addAll(hudiSplitsToAdd); + log.warn(String.format("Finish in %d ms to get splits", timer.endTimer())); } log.info("Number of Hudi splits generated in the batch: " + batchHudiSplits.size()); return batchHudiSplits; } + + private Pair> getPartitionPathToKey( + HiveIdentity identity, + HiveMetastore metastore, + Table table, + String tablePath, + List partitionName) + { + Optional partition1 = metastore.getPartition(identity, table, partitionName); + String dataDir1 = partition1.isPresent() + ? partition1.get().getStorage().getLocation() + : tablePath; + log.warn(">>> basePath: %s, dataDir1: %s", tablePath, dataDir1); + String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(tablePath), new Path(dataDir1)); + return new ImmutablePair<>(relativePartitionPath, getPartitionKeys(table, partition1)); + } } From af87fa92be22f1528c3bb3caccd053e32ad337d5 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Wed, 22 Dec 2021 23:01:06 -0800 Subject: [PATCH 21/26] Add logs for partition info from metastore --- .../java/io/trino/plugin/hudi/HudiSplitSource.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index 9b78f90cb42..af9d6dfafdb 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -158,19 +158,24 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce log.warn("Column Names: " + columnNames); HudiSplitSource splitSource; Map> partitionMap = new HashMap<>(); + List fullPartitionNames = new ArrayList<>(); + List> partitionNameElements = new ArrayList<>(); if (!columnNames.isEmpty()) { - partitionNames = metastore.getPartitionNamesByFilter(identity, tableName.getSchemaName(), tableName.getTableName(), columnNames, TupleDomain.all()) - .orElseThrow(() -> new TableNotFoundException(tableHandle.getSchemaTableName())) + fullPartitionNames = metastore.getPartitionNamesByFilter(identity, tableName.getSchemaName(), tableName.getTableName(), columnNames, TupleDomain.all()) + .orElseThrow(() -> new TableNotFoundException(tableHandle.getSchemaTableName())); + partitionNameElements = fullPartitionNames .stream() .map(HiveUtil::toPartitionValues) - .collect(toImmutableList()).iterator(); + .collect(toImmutableList()); + partitionNames = partitionNameElements.iterator(); } else { // no partitions, so data dir is same as table path partitionNames = Collections.singletonList(Collections.singletonList("")).iterator(); } - log.warn(String.format("Finish in %d ms. Partition Names: %s", timer.endTimer(), partitionNames.toString())); + log.warn(String.format("Finish in %d ms. Partition Names: %s", timer.endTimer(), fullPartitionNames)); + log.warn(String.format("Partition Name elements: %s", partitionNameElements)); } List batchHudiSplits = new ArrayList<>(); From c55ff2176321b33884aff07f5f7266af66bee552 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 23 Dec 2021 01:08:06 -0800 Subject: [PATCH 22/26] Add a hack to construct partition keys without using hive metastore --- .../java/io/trino/plugin/hudi/HudiConfig.java | 14 ++-- .../plugin/hudi/HudiSessionProperties.java | 12 +-- .../trino/plugin/hudi/HudiSplitManager.java | 8 +- .../io/trino/plugin/hudi/HudiSplitSource.java | 80 ++++++++++++++----- .../etc/catalog/hudi.properties | 1 - 5 files changed, 80 insertions(+), 35 deletions(-) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java index aad3edc21e8..fb8604e35d2 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java @@ -27,7 +27,7 @@ public class HudiConfig { private HoodieFileFormat fileFormat = PARQUET; private boolean metadataEnabled; - private boolean splitInSource; + private boolean shouldSkipMetaStoreForPartition = true; private DataSize maxSplitSize = DataSize.ofBytes(128 * 1024 * 1024); @NotNull @@ -70,17 +70,17 @@ public DataSize getMaxSplitSize() return this.maxSplitSize; } - @Config("hudi.split_in_source") - @ConfigDescription("Whether to split files in the HudiSplitSource. If false, done in HudiSplitManager.") - public HudiConfig setSplitInSource(boolean splitInSource) + @Config("hudi.skip_metastore_for_partition") + @ConfigDescription("Whether to skip metastore for partition") + public HudiConfig setSkipMetaStoreForPartition(boolean shouldSkipMetaStoreForPartition) { - this.splitInSource = splitInSource; + this.shouldSkipMetaStoreForPartition = shouldSkipMetaStoreForPartition; return this; } @NotNull - public boolean isSplitInSource() + public boolean getSkipMetaStoreForPartition() { - return this.splitInSource; + return this.shouldSkipMetaStoreForPartition; } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java index 3510ab5d231..fb77e3a2798 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSessionProperties.java @@ -35,7 +35,7 @@ public class HudiSessionProperties private static final String FILE_FORMAT = "file_format"; private static final String METADATA_ENABLED = "metadata_enabled"; private static final String MAX_SPLIT_SIZE = "max_split_size"; - private static final String SPLIT_IN_SOURCE = "split_in_source"; + private static final String SKIP_METASTORE_FOR_PARTITION = "skip_metastore_for_partition"; private final List> sessionProperties; @@ -55,9 +55,9 @@ public HudiSessionProperties(HudiConfig hudiConfig) hudiConfig.isMetadataEnabled(), false), booleanProperty( - SPLIT_IN_SOURCE, - "Whether to split files in the HudiSplitSource. If false, done in HudiSplitManager.", - hudiConfig.isSplitInSource(), + SKIP_METASTORE_FOR_PARTITION, + "Whether to skip metastore for partition.", + hudiConfig.getSkipMetaStoreForPartition(), false), dataSizeProperty( MAX_SPLIT_SIZE, @@ -87,8 +87,8 @@ public static DataSize getMaxSplitSize(ConnectorSession session) return session.getProperty(MAX_SPLIT_SIZE, DataSize.class); } - public static boolean isSplitInSource(ConnectorSession session) + public static boolean shouldSkipMetaStoreForPartition(ConnectorSession session) { - return session.getProperty(SPLIT_IN_SOURCE, Boolean.class); + return session.getProperty(SKIP_METASTORE_FOR_PARTITION, Boolean.class); } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index 13bc7ecf3de..0b66e634093 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -20,6 +20,7 @@ import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.Table; +import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.ConnectorSplitSource; @@ -29,6 +30,7 @@ import io.trino.spi.connector.DynamicFilter; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.predicate.TupleDomain; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -39,6 +41,7 @@ import java.util.regex.Pattern; import static io.trino.plugin.hudi.HudiSessionProperties.isHudiMetadataEnabled; +import static io.trino.plugin.hudi.HudiSessionProperties.shouldSkipMetaStoreForPartition; import static io.trino.plugin.hudi.HudiUtil.getMetaClient; import static java.util.Objects.requireNonNull; @@ -86,8 +89,11 @@ public ConnectorSplitSource getSplits( log.debug("HoodieTableMetaClient base path: " + metaClient.getBasePath()); hudiTable.getPartitions().ifPresent(p -> p.forEach(p1 -> log.warn("Partitions from TableHandle: " + p1))); + TupleDomain effectivePredicate = constraint.getSummary(); + log.debug("effectivePredicate: " + effectivePredicate); + HudiSplitSource splitSource = new HudiSplitSource(identity, metastore, hudiTable, conf, - isHudiMetadataEnabled(session), dynamicFilter); + isHudiMetadataEnabled(session), shouldSkipMetaStoreForPartition(session), dynamicFilter); return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index af9d6dfafdb..96938c69caa 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -80,11 +80,13 @@ public class HudiSplitSource private final HoodieTableMetaClient metaClient; private final Map> partitionMap; private final boolean metadataEnabled; + private final boolean shouldSkipMetaStoreForPartition; private final Map baseFileToPartitionMap; private final ArrayDeque baseFiles = new ArrayDeque<>(); private final DynamicFilter dynamicFilter; - private final int partitionBatchNum = 100; + private final int partitionBatchNum = 32; private HoodieTableFileSystemView fileSystemView; + private List partitionColumnNames; private Iterator> partitionNames; public HudiSplitSource( @@ -93,6 +95,7 @@ public HudiSplitSource( HudiTableHandle tableHandle, Configuration conf, boolean metadataEnabled, + boolean shouldSkipMetaStoreForPartition, DynamicFilter dynamicFilter) { this.identity = identity; @@ -104,6 +107,7 @@ public HudiSplitSource( this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); this.partitionMap = new HashMap<>(); this.metadataEnabled = metadataEnabled; + this.shouldSkipMetaStoreForPartition = shouldSkipMetaStoreForPartition; this.metaClient = tableHandle.getMetaClient().orElseGet(() -> getMetaClient(conf, tableHandle.getBasePath())); this.fileSystem = metaClient.getFs(); this.baseFileToPartitionMap = new HashMap<>(); @@ -152,16 +156,14 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce log.warn(String.format("Finish in %d ms to load table view", timer.endTimer())); timer = new HoodieTimer().startTimer(); - List columnNames = table.getPartitionColumns().stream() + partitionColumnNames = table.getPartitionColumns().stream() .map(Column::getName) .collect(toImmutableList()); - log.warn("Column Names: " + columnNames); - HudiSplitSource splitSource; - Map> partitionMap = new HashMap<>(); + log.warn("Column Names: " + partitionColumnNames); List fullPartitionNames = new ArrayList<>(); List> partitionNameElements = new ArrayList<>(); - if (!columnNames.isEmpty()) { - fullPartitionNames = metastore.getPartitionNamesByFilter(identity, tableName.getSchemaName(), tableName.getTableName(), columnNames, TupleDomain.all()) + if (!partitionColumnNames.isEmpty()) { + fullPartitionNames = metastore.getPartitionNamesByFilter(identity, tableName.getSchemaName(), tableName.getTableName(), partitionColumnNames, TupleDomain.all()) .orElseThrow(() -> new TableNotFoundException(tableHandle.getSchemaTableName())); partitionNameElements = fullPartitionNames .stream() @@ -184,7 +186,7 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce log.debug("Target number of splits: " + maxSize); // Only process one batch now - if (remaining > 0 && !isFinished()) { + while (remaining > 0 && !isFinished()) { if (baseFiles.isEmpty()) { HoodieTimer timer1 = new HoodieTimer().startTimer(); @@ -194,11 +196,11 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce Map> batchKeyMap = batchPartitionNames.stream().parallel() .map(partitionNames -> getPartitionPathToKey( - identity, metastore, table, table.getStorage().getLocation(), partitionNames)) + identity, metastore, table, table.getStorage().getLocation(), partitionColumnNames, partitionNames)) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); partitionMap.putAll(batchKeyMap); - log.warn(String.format("Finish in %d ms to get partition keys", timer1.endTimer())); + log.warn(String.format("Finish in %d ms to get partition keys: %s", timer1.endTimer(), batchKeyMap.toString())); timer1 = new HoodieTimer().startTimer(); List> baseFilesToAdd = batchKeyMap.keySet().stream().parallel() @@ -215,10 +217,9 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce HoodieTimer timer = new HoodieTimer().startTimer(); List batchBaseFiles = new ArrayList<>(); - while (remaining > 0 && !baseFiles.isEmpty()) { + while (!baseFiles.isEmpty()) { HoodieBaseFile baseFile = baseFiles.pollFirst(); batchBaseFiles.add(baseFile); - remaining--; } List hudiSplitsToAdd = batchBaseFiles.stream().parallel() @@ -256,7 +257,11 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce }) .collect(Collectors.toList()); batchHudiSplits.addAll(hudiSplitsToAdd); - log.warn(String.format("Finish in %d ms to get splits", timer.endTimer())); + remaining -= hudiSplitsToAdd.size(); + log.warn(String.format("Finish in %d ms to get batch splits", timer.endTimer())); + if (remaining < hudiSplitsToAdd.size()) { + break; + } } log.info("Number of Hudi splits generated in the batch: " + batchHudiSplits.size()); @@ -269,14 +274,49 @@ private Pair> getPartitionPathToKey( HiveMetastore metastore, Table table, String tablePath, + List columnNames, List partitionName) { - Optional partition1 = metastore.getPartition(identity, table, partitionName); - String dataDir1 = partition1.isPresent() - ? partition1.get().getStorage().getLocation() - : tablePath; - log.warn(">>> basePath: %s, dataDir1: %s", tablePath, dataDir1); - String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(tablePath), new Path(dataDir1)); - return new ImmutablePair<>(relativePartitionPath, getPartitionKeys(table, partition1)); + Optional partition1 = Optional.empty(); + String relativePartitionPath = ""; + List partitionKeys = new ArrayList<>(); + if (!columnNames.isEmpty()) { + if (shouldSkipMetaStoreForPartition) { + StringBuilder partitionPathBuilder = new StringBuilder(); + for (int i = 0; i < columnNames.size(); i++) { + if (partitionPathBuilder.length() > 0) { + partitionPathBuilder.append("/"); + } + String columnName = columnNames.get(i); + String value = partitionName.get(i); + partitionKeys.add(new HivePartitionKey(columnName, value)); + partitionPathBuilder.append(columnName); + partitionPathBuilder.append("="); + partitionPathBuilder.append(value); + } + if (columnNames.size() == 1) { + String value = partitionName.get(0); + if (value.contains("-")) { + relativePartitionPath = value.replace("-", "/"); + } + else { + relativePartitionPath = partitionPathBuilder.toString(); + } + } + else { + relativePartitionPath = partitionPathBuilder.toString(); + } + } + else { + partition1 = metastore.getPartition(identity, table, partitionName); + String dataDir1 = partition1.isPresent() + ? partition1.get().getStorage().getLocation() + : tablePath; + log.warn(">>> basePath: %s, dataDir1: %s", tablePath, dataDir1); + relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(tablePath), new Path(dataDir1)); + partitionKeys = getPartitionKeys(table, partition1); + } + } + return new ImmutablePair<>(relativePartitionPath, partitionKeys); } } diff --git a/testing/trino-server-dev/etc/catalog/hudi.properties b/testing/trino-server-dev/etc/catalog/hudi.properties index 0b50f174bf9..33ba981c9bc 100644 --- a/testing/trino-server-dev/etc/catalog/hudi.properties +++ b/testing/trino-server-dev/etc/catalog/hudi.properties @@ -14,5 +14,4 @@ connector.name=hudi hive.metastore.uri=thrift://localhost:9083 -hudi.max_split_size=16MB #hive.config.resources=/Users/sagars/core-site.xml,/Users/sagars/hdfs-site.xml From f246f0a518431a9d61f3a56708b7727e479b080d Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Mon, 27 Dec 2021 15:33:05 +0530 Subject: [PATCH 23/26] Refactoring and more tests --- .../java/io/trino/plugin/hudi/HudiConfig.java | 4 +- .../plugin/hudi/HudiPageSourceProvider.java | 27 ++- .../trino/plugin/hudi/HudiSplitManager.java | 22 +-- .../io/trino/plugin/hudi/HudiSplitSource.java | 171 ++++++++++-------- .../java/io/trino/plugin/hudi/HudiUtil.java | 142 +++++++++------ .../hudi/AbstractHudiTestQueryFramework.java | 6 +- .../io/trino/plugin/hudi/TestHudiConfig.java | 59 ++++++ .../io/trino/plugin/hudi/TestHudiSanity.java | 22 ++- .../io/trino/plugin/hudi/TestHudiUtil.java | 64 +++++++ .../.hoodie/20211221030120532.deltacommit | 51 ++++++ .../20211221030120532.deltacommit.inflight | 48 +++++ .../20211221030120532.deltacommit.requested | 0 .../.hoodie/20211227092838847.deltacommit | 55 ++++++ .../20211227092838847.deltacommit.inflight | 71 ++++++++ .../20211227092838847.deltacommit.requested | 0 .../stock_ticks_mor/.hoodie/hoodie.properties | 14 ++ ...30cdb5a2-0_20211221030120532.log.1_0-28-29 | Bin 0 -> 22220 bytes .../2018/08/31/.hoodie_partition_metadata | 4 + ...cdb5a2-0_0-28-26_20211221030120532.parquet | Bin 0 -> 440746 bytes 19 files changed, 593 insertions(+), 167 deletions(-) create mode 100644 plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java create mode 100644 plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiUtil.java create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.inflight create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.requested create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.inflight create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.requested create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/hoodie.properties create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/.167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29 create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/.hoodie_partition_metadata create mode 100644 plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java index fb8604e35d2..7b10e20d7fa 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConfig.java @@ -57,7 +57,7 @@ public boolean isMetadataEnabled() return this.metadataEnabled; } - @Config("hudi.max_split_size") + @Config("hudi.max-split-size") public HudiConfig setMaxSplitSize(DataSize size) { this.maxSplitSize = size; @@ -70,7 +70,7 @@ public DataSize getMaxSplitSize() return this.maxSplitSize; } - @Config("hudi.skip_metastore_for_partition") + @Config("hudi.skip-metastore-for-partition") @ConfigDescription("Whether to skip metastore for partition") public HudiConfig setSkipMetaStoreForPartition(boolean shouldSkipMetaStoreForPartition) { diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java index fb978ccdc61..19d5f4b00ca 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java @@ -131,7 +131,6 @@ public ConnectorPageSource createPageSource( DynamicFilter dynamicFilter) { HudiSplit split = (HudiSplit) connectorSplit; - HudiTableHandle table = (HudiTableHandle) connectorTable; Path path = new Path(split.getPath()); long start = split.getStart(); long length = split.getLength(); @@ -183,7 +182,7 @@ private static ConnectorPageSource createParquetPageSource( ConnectorIdentity identity, List partitionKeys) { - log.warn(">>> Creating Parquet Page Source with partition keys: %s, columns: %s, predicate: %s", partitionKeys, columns, effectivePredicate); + log.debug(">>> Creating Parquet Page Source with partition keys: %s, columns: %s, predicate: %s", partitionKeys, columns, effectivePredicate); ParquetDataSource dataSource = null; // TODO: Reuse some elements of ParquetPageSourceFactory and extract the try block to a new HudiParquetReader class. @@ -195,7 +194,7 @@ private static ConnectorPageSource createParquetPageSource( ParquetMetadata parquetMetadata = hdfsEnvironment.doAs(identity, () -> MetadataReader.readFooter(theDataSource)); FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); MessageType fileSchema = fileMetaData.getSchema(); - log.warn(">>> File Schema: " + fileSchema.toString()); + log.debug(">>> File Schema: " + fileSchema.toString()); Optional message = projectSufficientColumns(columns) .map(projection -> projection.get().stream() @@ -210,15 +209,15 @@ private static ConnectorPageSource createParquetPageSource( .reduce(MessageType::union); MessageType requestedSchema = message.orElse(new MessageType(fileSchema.getName(), ImmutableList.of())); - log.warn(">>> Requested Schema: " + requestedSchema); + log.debug(">>> Requested Schema: " + requestedSchema); MessageColumnIO messageColumn = getColumnIO(fileSchema, requestedSchema); Map, RichColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, requestedSchema); - log.warn(">>> Ignore stats: " + options.isIgnoreStatistics()); + log.debug(">>> Ignore stats: " + options.isIgnoreStatistics()); TupleDomain parquetTupleDomain = options.isIgnoreStatistics() ? TupleDomain.all() : getParquetTupleDomain(descriptorsByPath, effectivePredicate, fileSchema, useParquetColumnNames); - log.warn(">>> ParquetTupleDomain: %s", parquetTupleDomain.toString()); + log.debug(">>> ParquetTupleDomain: %s", parquetTupleDomain.toString()); Predicate parquetPredicate = buildPredicate(requestedSchema, parquetTupleDomain, descriptorsByPath, timeZone); @@ -238,7 +237,7 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq nextStart += block.getRowCount(); } - log.warn("Message Column: %s, Predicate: %s, ColumnIndexes: %s", messageColumn, parquetPredicate.toString(), columnIndexes.build()); + log.debug("Message Column: %s, Predicate: %s, ColumnIndexes: %s", messageColumn, parquetPredicate.toString(), columnIndexes.build()); ParquetReader parquetReader = new ParquetReader( Optional.ofNullable(fileMetaData.getCreatedBy()), messageColumn, @@ -251,13 +250,13 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq parquetPredicate, columnIndexes.build()); Optional readerProjections = projectBaseColumns(columns); - log.warn("Reade columns: %s", readerProjections.orElse(null)); + log.debug("Reade columns: %s", readerProjections.orElse(null)); List baseColumns = readerProjections.map(projection -> projection.get().stream() .map(HiveColumnHandle.class::cast) .collect(toUnmodifiableList())) .orElse(columns); - log.warn("Base columns: %s", baseColumns); + log.debug("Base columns: %s", baseColumns); for (HiveColumnHandle column : baseColumns) { checkArgument(column == PARQUET_ROW_INDEX_COLUMN || column.getColumnType() == REGULAR, "column type must be REGULAR: %s", column); @@ -281,9 +280,9 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq } } - log.warn("Trino types: %s", trinoTypes.build()); - log.warn("Internal fields: %s", internalFields.build()); - log.warn("Row index: %s", rowIndexColumns.build()); + log.debug("Trino types: %s", trinoTypes.build()); + log.debug("Internal fields: %s", internalFields.build()); + log.debug("Row index: %s", rowIndexColumns.build()); return new ParquetPageSource( parquetReader, @@ -319,7 +318,7 @@ private Map convertPartitionValues( List allColumns, List partitionKeys) { - log.warn(">>> Converting partition values for columns: %s, partition key: %s", allColumns, partitionKeys); + log.debug(">>> Converting partition values for columns: %s, partition key: %s", allColumns, partitionKeys); return allColumns.stream() .filter(HiveColumnHandle::isPartitionKey) .collect(toMap( @@ -329,6 +328,6 @@ private Map convertPartitionValues( convertPartitionValue( columnHandle.getName(), partitionKeys.get(0).getValue(), - columnHandle.getType().getTypeSignature())))); + columnHandle.getType().getTypeSignature()).orElse(null)))); } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index 0b66e634093..3420b39a110 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -70,7 +70,7 @@ public ConnectorSplitSource getSplits( DynamicFilter dynamicFilter, Constraint constraint) { - log.warn(" >>>> Getting Splits <<<< "); + log.debug(" >>>> Getting Splits <<<< "); HiveIdentity identity = new HiveIdentity(session); HudiTableHandle hudiTable = (HudiTableHandle) tableHandle; SchemaTableName tableName = hudiTable.getSchemaTableName(); @@ -80,6 +80,7 @@ public ConnectorSplitSource getSplits( HdfsEnvironment.HdfsContext context = new HdfsEnvironment.HdfsContext(session); String tablePath = table.getStorage().getLocation(); Configuration conf = hdfsEnvironment.getConfiguration(context, new Path(tablePath)); + // TODO: Do we need below? Map valByRegex = conf.getValByRegex(HOODIE_CONSUME_MODE_PATTERN_STRING.pattern()); log.debug("Hoodie consume mode: " + valByRegex); HoodieTableMetaClient metaClient = hudiTable.getMetaClient().orElseGet(() -> getMetaClient(conf, hudiTable.getBasePath())); @@ -87,20 +88,19 @@ public ConnectorSplitSource getSplits( log.debug("Table ref: " + table.toString()); log.debug("HoodieTableMetaClient ref: " + metaClient.toString()); log.debug("HoodieTableMetaClient base path: " + metaClient.getBasePath()); - hudiTable.getPartitions().ifPresent(p -> p.forEach(p1 -> log.warn("Partitions from TableHandle: " + p1))); + hudiTable.getPartitions().ifPresent(p -> p.forEach(p1 -> log.debug("Partitions from TableHandle: " + p1))); TupleDomain effectivePredicate = constraint.getSummary(); log.debug("effectivePredicate: " + effectivePredicate); - HudiSplitSource splitSource = new HudiSplitSource(identity, metastore, hudiTable, conf, - isHudiMetadataEnabled(session), shouldSkipMetaStoreForPartition(session), dynamicFilter); + HudiSplitSource splitSource = new HudiSplitSource( + identity, + metastore, + hudiTable, + conf, + isHudiMetadataEnabled(session), + shouldSkipMetaStoreForPartition(session), + dynamicFilter); return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); } - - void printConf(Configuration conf) - { - for (Map.Entry entry : conf) { - log.warn("%s=%s\n", entry.getKey(), entry.getValue()); - } - } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index 96938c69caa..0243de021e1 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; import io.airlift.log.Logger; +import io.trino.plugin.hive.HivePartition; import io.trino.plugin.hive.HivePartitionKey; import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.Column; @@ -62,7 +63,12 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeys; +import static io.trino.plugin.hudi.HudiUtil.buildPartitionKeys; +import static io.trino.plugin.hudi.HudiUtil.buildPartitionValues; import static io.trino.plugin.hudi.HudiUtil.getMetaClient; +import static io.trino.plugin.hudi.HudiUtil.getSplits; +import static java.util.Objects.isNull; +import static java.util.Objects.nonNull; import static java.util.Objects.requireNonNull; import static java.util.concurrent.CompletableFuture.completedFuture; @@ -70,6 +76,7 @@ public class HudiSplitSource implements ConnectorSplitSource { private static final Logger log = Logger.get(HudiSplitSource.class); + private final HiveIdentity identity; private final HiveMetastore metastore; private final SchemaTableName tableName; @@ -85,9 +92,11 @@ public class HudiSplitSource private final ArrayDeque baseFiles = new ArrayDeque<>(); private final DynamicFilter dynamicFilter; private final int partitionBatchNum = 32; + private HoodieTableFileSystemView fileSystemView; - private List partitionColumnNames; - private Iterator> partitionNames; + private List partitionColumns; + private Iterator> metastorePartitions; + private Iterator tableHandlePartitions; public HudiSplitSource( HiveIdentity identity, @@ -131,18 +140,24 @@ public CompletableFuture getNextBatch(ConnectorPartitionHan @Override public void close() { - fileSystemView.close(); + if (nonNull(fileSystem)) { + fileSystemView.close(); + } } @Override public boolean isFinished() { - return !partitionNames.hasNext() && baseFiles.isEmpty(); + if (shouldSkipMetaStoreForPartition) { + return !tableHandlePartitions.hasNext() && baseFiles.isEmpty(); + } + return !metastorePartitions.hasNext() && baseFiles.isEmpty(); } - private List getSplitsForSnapshotMode(int maxSize) throws IOException + private List getSplitsForSnapshotMode(int maxSize) + throws IOException { - if (this.fileSystemView == null) { + if (isNull(this.fileSystemView)) { HoodieTimer timer = new HoodieTimer().startTimer(); // First time calling this // Load the timeline and file status only once @@ -153,31 +168,46 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce // Scan the file system to load the instants from timeline log.debug("Loading file system view for " + metaClient.getBasePath()); this.fileSystemView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, metaClient, metadataConfig); - log.warn(String.format("Finish in %d ms to load table view", timer.endTimer())); + log.debug(String.format("Finish in %d ms to load table view", timer.endTimer())); timer = new HoodieTimer().startTimer(); - partitionColumnNames = table.getPartitionColumns().stream() - .map(Column::getName) - .collect(toImmutableList()); - log.warn("Column Names: " + partitionColumnNames); - List fullPartitionNames = new ArrayList<>(); - List> partitionNameElements = new ArrayList<>(); - if (!partitionColumnNames.isEmpty()) { - fullPartitionNames = metastore.getPartitionNamesByFilter(identity, tableName.getSchemaName(), tableName.getTableName(), partitionColumnNames, TupleDomain.all()) - .orElseThrow(() -> new TableNotFoundException(tableHandle.getSchemaTableName())); - partitionNameElements = fullPartitionNames - .stream() - .map(HiveUtil::toPartitionValues) - .collect(toImmutableList()); - partitionNames = partitionNameElements.iterator(); + partitionColumns = table.getPartitionColumns(); + + if (!partitionColumns.isEmpty()) { + if (shouldSkipMetaStoreForPartition) { + List partitions = tableHandle.getPartitions().orElseGet(ImmutableList::of).stream() + .map(HivePartition::getPartitionId).collect(Collectors.toList()); + log.debug(">>> Partitions tableHandle: %s", partitions); + tableHandlePartitions = partitions.iterator(); + } + else { + List fullPartitionNames = metastore.getPartitionNamesByFilter( + identity, + tableName.getSchemaName(), + tableName.getTableName(), + partitionColumns.stream() + .map(Column::getName) + .collect(Collectors.toList()), + TupleDomain.all()) + .orElseThrow(() -> new TableNotFoundException(tableHandle.getSchemaTableName())); + List> partitionNameElements = fullPartitionNames + .stream() + .map(HiveUtil::toPartitionValues) + .collect(toImmutableList()); + metastorePartitions = partitionNameElements.iterator(); + } } else { // no partitions, so data dir is same as table path - partitionNames = Collections.singletonList(Collections.singletonList("")).iterator(); + if (shouldSkipMetaStoreForPartition) { + tableHandlePartitions = Collections.singletonList("").iterator(); + } + else { + metastorePartitions = Collections.singletonList(Collections.singletonList("")).iterator(); + } } - log.warn(String.format("Finish in %d ms. Partition Names: %s", timer.endTimer(), fullPartitionNames)); - log.warn(String.format("Partition Name elements: %s", partitionNameElements)); + log.debug(String.format("Finish in %d ms to fetch partition names", timer.endTimer())); } List batchHudiSplits = new ArrayList<>(); @@ -190,17 +220,27 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce if (baseFiles.isEmpty()) { HoodieTimer timer1 = new HoodieTimer().startTimer(); - List> batchPartitionNames = new ArrayList<>(); - Iterators.limit(partitionNames, partitionBatchNum).forEachRemaining(batchPartitionNames::add); - - Map> batchKeyMap = - batchPartitionNames.stream().parallel() - .map(partitionNames -> getPartitionPathToKey( - identity, metastore, table, table.getStorage().getLocation(), partitionColumnNames, partitionNames)) - .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + Map> batchKeyMap; + if (shouldSkipMetaStoreForPartition) { + List batchTableHandlePartitions = new ArrayList<>(); + Iterators.limit(tableHandlePartitions, partitionBatchNum).forEachRemaining(batchTableHandlePartitions::add); + batchKeyMap = batchTableHandlePartitions.stream().parallel() + .map(p -> Pair.of(p, buildPartitionKeys(partitionColumns, buildPartitionValues(p)))) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + log.debug(">>> shouldSkipMetaStoreForPartition batchKeyMap: %s", batchKeyMap); + } + else { + List> batchMetastorePartitions = new ArrayList<>(); + Iterators.limit(metastorePartitions, partitionBatchNum).forEachRemaining(batchMetastorePartitions::add); + batchKeyMap = batchMetastorePartitions.stream().parallel() + .map(partitionNames -> getPartitionPathToKey( + identity, metastore, table, table.getStorage().getLocation(), partitionNames)) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + log.debug(">>> batchKeyMap: %s", batchKeyMap); + } partitionMap.putAll(batchKeyMap); - log.warn(String.format("Finish in %d ms to get partition keys: %s", timer1.endTimer(), batchKeyMap.toString())); + log.debug(String.format("Finish in %d ms to get partition keys: %s", timer1.endTimer(), batchKeyMap)); timer1 = new HoodieTimer().startTimer(); List> baseFilesToAdd = batchKeyMap.keySet().stream().parallel() @@ -212,7 +252,7 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce baseFilesToAdd.forEach(e -> baseFileToPartitionMap.put(e.getKey(), e.getValue())); // TODO: skip partitions that are filtered out based on the predicate baseFiles.addAll(baseFilesToAdd.stream().map(Pair::getKey).collect(Collectors.toList())); - log.warn(String.format("Finish in %d ms to get base files", timer1.endTimer())); + log.debug(String.format("Finish in %d ms to get base files", timer1.endTimer())); } HoodieTimer timer = new HoodieTimer().startTimer(); @@ -224,9 +264,9 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce List hudiSplitsToAdd = batchBaseFiles.stream().parallel() .flatMap(baseFile -> { - List hudiSplits = new ArrayList<>(); + List hudiSplits; try { - hudiSplits = HudiUtil.getSplits( + hudiSplits = getSplits( fileSystem, HoodieInputFormatUtils.getFileStatus(baseFile)) .stream() .flatMap(fileSplit -> { @@ -258,7 +298,7 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOExce .collect(Collectors.toList()); batchHudiSplits.addAll(hudiSplitsToAdd); remaining -= hudiSplitsToAdd.size(); - log.warn(String.format("Finish in %d ms to get batch splits", timer.endTimer())); + log.debug(String.format("Finish in %d ms to get batch splits", timer.endTimer())); if (remaining < hudiSplitsToAdd.size()) { break; } @@ -274,49 +314,22 @@ private Pair> getPartitionPathToKey( HiveMetastore metastore, Table table, String tablePath, - List columnNames, - List partitionName) + List partitionValues) { - Optional partition1 = Optional.empty(); - String relativePartitionPath = ""; - List partitionKeys = new ArrayList<>(); - if (!columnNames.isEmpty()) { - if (shouldSkipMetaStoreForPartition) { - StringBuilder partitionPathBuilder = new StringBuilder(); - for (int i = 0; i < columnNames.size(); i++) { - if (partitionPathBuilder.length() > 0) { - partitionPathBuilder.append("/"); - } - String columnName = columnNames.get(i); - String value = partitionName.get(i); - partitionKeys.add(new HivePartitionKey(columnName, value)); - partitionPathBuilder.append(columnName); - partitionPathBuilder.append("="); - partitionPathBuilder.append(value); - } - if (columnNames.size() == 1) { - String value = partitionName.get(0); - if (value.contains("-")) { - relativePartitionPath = value.replace("-", "/"); - } - else { - relativePartitionPath = partitionPathBuilder.toString(); - } - } - else { - relativePartitionPath = partitionPathBuilder.toString(); - } - } - else { - partition1 = metastore.getPartition(identity, table, partitionName); - String dataDir1 = partition1.isPresent() - ? partition1.get().getStorage().getLocation() - : tablePath; - log.warn(">>> basePath: %s, dataDir1: %s", tablePath, dataDir1); - relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(tablePath), new Path(dataDir1)); - partitionKeys = getPartitionKeys(table, partition1); - } - } + log.debug(">>> Inside getPartitionPathToKey: %s", partitionValues); + Optional partition1; + String relativePartitionPath; + List partitionKeys; + partition1 = metastore.getPartition(identity, table, partitionValues); + partition1.ifPresent(p -> log.debug("Partition from metastore: %s", p)); + String dataDir1 = partition1.isPresent() + ? partition1.get().getStorage().getLocation() + : tablePath; + log.debug(">>> basePath: %s, dataDir1: %s", tablePath, dataDir1); + relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(tablePath), new Path(dataDir1)); + log.debug(">>> relativePartitionPath: %s", relativePartitionPath); + partitionKeys = getPartitionKeys(table, partition1); + log.debug(">>> Partition keys: %s", partitionKeys); return new ImmutablePair<>(relativePartitionPath, partitionKeys); } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java index ca40900ffb4..dc213ea9e90 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiUtil.java @@ -15,16 +15,14 @@ package io.trino.plugin.hudi; import com.google.common.collect.ImmutableList; -import io.airlift.log.Logger; import io.trino.plugin.hive.HiveColumnHandle; -import io.trino.plugin.hive.metastore.Partition; -import io.trino.plugin.hive.metastore.Table; +import io.trino.plugin.hive.HivePartitionKey; +import io.trino.plugin.hive.metastore.Column; import io.trino.spi.TrinoException; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.predicate.Domain; import io.trino.spi.predicate.TupleDomain; import io.trino.spi.type.Decimals; -import io.trino.spi.type.StandardTypes; import io.trino.spi.type.TypeSignature; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; @@ -50,24 +48,34 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Properties; import static com.google.common.base.Preconditions.checkArgument; import static io.airlift.slice.Slices.utf8Slice; -import static io.trino.plugin.hive.metastore.MetastoreUtil.getHiveSchema; import static io.trino.plugin.hudi.HudiErrorCode.HUDI_INVALID_PARTITION_VALUE; +import static io.trino.spi.type.StandardTypes.BIGINT; +import static io.trino.spi.type.StandardTypes.BOOLEAN; +import static io.trino.spi.type.StandardTypes.DATE; +import static io.trino.spi.type.StandardTypes.DECIMAL; +import static io.trino.spi.type.StandardTypes.DOUBLE; +import static io.trino.spi.type.StandardTypes.INTEGER; +import static io.trino.spi.type.StandardTypes.REAL; +import static io.trino.spi.type.StandardTypes.SMALLINT; +import static io.trino.spi.type.StandardTypes.TIMESTAMP; +import static io.trino.spi.type.StandardTypes.TINYINT; +import static io.trino.spi.type.StandardTypes.VARBINARY; +import static io.trino.spi.type.StandardTypes.VARCHAR; import static java.lang.Double.parseDouble; import static java.lang.Float.floatToRawIntBits; import static java.lang.Float.parseFloat; import static java.lang.Long.parseLong; import static java.lang.String.format; +import static java.util.Objects.isNull; +import static org.apache.hadoop.hive.common.FileUtils.unescapePathName; public class HudiUtil { private static final double SPLIT_SLOP = 1.1; // 10% slop - private static final Logger log = Logger.get(HudiUtil.class); - private HudiUtil() {} public static HoodieTableMetaClient getMetaClient(Configuration conf, String basePath) @@ -80,14 +88,6 @@ public static boolean isHudiParquetInputFormat(InputFormat inputFormat) return inputFormat instanceof HoodieParquetInputFormat; } - public static Properties getPartitionSchema(Table table, Optional partition) - { - if (partition.isEmpty()) { - return getHiveSchema(table); - } - return getHiveSchema(partition.get(), table); - } - public static List> splitPredicate( TupleDomain predicate) { @@ -110,41 +110,39 @@ public static List> splitPredicate( TupleDomain.withColumnDomains(regularColumnPredicates)); } - public static Object convertPartitionValue( + public static Optional convertPartitionValue( String partitionColumnName, String partitionValue, TypeSignature partitionDataType) { - log.warn(">>> convertPartitionValue column: %s, value: %s, dataType: %s", partitionColumnName, partitionValue, partitionDataType); - if (partitionValue == null) { - return null; + if (isNull(partitionValue)) { + return Optional.empty(); } - String typeBase = partitionDataType.getBase(); - log.warn(">>> Base Type: %s", typeBase); + String baseType = partitionDataType.getBase(); try { - switch (typeBase) { - case StandardTypes.TINYINT: - case StandardTypes.SMALLINT: - case StandardTypes.INTEGER: - case StandardTypes.BIGINT: - return parseLong(partitionValue); - case StandardTypes.REAL: - return (long) floatToRawIntBits(parseFloat(partitionValue)); - case StandardTypes.DOUBLE: - return parseDouble(partitionValue); - case StandardTypes.VARCHAR: - case StandardTypes.VARBINARY: - return utf8Slice(partitionValue); - case StandardTypes.DATE: - return LocalDate.parse(partitionValue, DateTimeFormatter.ISO_LOCAL_DATE).toEpochDay(); - case StandardTypes.TIMESTAMP: - return Timestamp.valueOf(partitionValue).toLocalDateTime().toEpochSecond(ZoneOffset.UTC) * 1_000; - case StandardTypes.BOOLEAN: + switch (baseType) { + case TINYINT: + case SMALLINT: + case INTEGER: + case BIGINT: + return Optional.of(parseLong(partitionValue)); + case REAL: + return Optional.of((long) floatToRawIntBits(parseFloat(partitionValue))); + case DOUBLE: + return Optional.of(parseDouble(partitionValue)); + case VARCHAR: + case VARBINARY: + return Optional.of(utf8Slice(partitionValue)); + case DATE: + return Optional.of(LocalDate.parse(partitionValue, DateTimeFormatter.ISO_LOCAL_DATE).toEpochDay()); + case TIMESTAMP: + return Optional.of(Timestamp.valueOf(partitionValue).toLocalDateTime().toEpochSecond(ZoneOffset.UTC) * 1_000); + case BOOLEAN: checkArgument(partitionValue.equalsIgnoreCase("true") || partitionValue.equalsIgnoreCase("false")); - return Boolean.valueOf(partitionValue); - case StandardTypes.DECIMAL: - return Decimals.parse(partitionValue).getObject(); + return Optional.of(Boolean.valueOf(partitionValue)); + case DECIMAL: + return Optional.of(Decimals.parse(partitionValue).getObject()); default: throw new TrinoException(HUDI_INVALID_PARTITION_VALUE, format("Unsupported data type '%s' for partition column %s", partitionDataType, partitionColumnName)); @@ -157,7 +155,8 @@ public static Object convertPartitionValue( } } - public static List getSplits(FileSystem fs, FileStatus fileStatus) throws IOException + public static List getSplits(FileSystem fs, FileStatus fileStatus) + throws IOException { if (fileStatus.isDirectory()) { throw new IOException("Not a file: " + fileStatus.getPath()); @@ -213,36 +212,31 @@ private static boolean isSplitable(FileSystem fs, Path filename) return !(filename instanceof PathWithBootstrapFileStatus); } - private static long computeSplitSize(long goalSize, long minSize, - long blockSize) + private static long computeSplitSize(long goalSize, long minSize, long blockSize) { return Math.max(minSize, Math.min(goalSize, blockSize)); } - private static FileSplit makeSplit(Path file, long start, long length, - String[] hosts) + private static FileSplit makeSplit(Path file, long start, long length, String[] hosts) { return new FileSplit(file, start, length, hosts); } - private static FileSplit makeSplit(Path file, long start, long length, - String[] hosts, String[] inMemoryHosts) + private static FileSplit makeSplit(Path file, long start, long length, String[] hosts, String[] inMemoryHosts) { return new FileSplit(file, start, length, hosts, inMemoryHosts); } - private static String[][] getSplitHostsAndCachedHosts(BlockLocation[] blkLocations, - long offset, long splitSize, NetworkTopology clusterMap) + private static String[][] getSplitHostsAndCachedHosts(BlockLocation[] blkLocations, long offset, long splitSize, NetworkTopology clusterMap) throws IOException { int startIndex = getBlockIndex(blkLocations, offset); - return new String[][]{blkLocations[startIndex].getHosts(), - blkLocations[startIndex].getCachedHosts()}; + return new String[][] {blkLocations[startIndex].getHosts(), + blkLocations[startIndex].getCachedHosts()}; } - private static int getBlockIndex(BlockLocation[] blkLocations, - long offset) + private static int getBlockIndex(BlockLocation[] blkLocations, long offset) { for (int i = 0; i < blkLocations.length; i++) { // is the offset inside this block? @@ -257,4 +251,40 @@ private static int getBlockIndex(BlockLocation[] blkLocations, " is outside of file (0.." + fileLength + ")"); } + + public static List buildPartitionKeys(List keys, List values) + { + ImmutableList.Builder partitionKeys = ImmutableList.builder(); + for (int i = 0; i < keys.size(); i++) { + String name = keys.get(i).getName(); + String value = values.get(i); + partitionKeys.add(new HivePartitionKey(name, value)); + } + return partitionKeys.build(); + } + + public static List buildPartitionValues(String partitionNames) + { + ImmutableList.Builder values = ImmutableList.builder(); + String[] parts = partitionNames.split("="); + if (parts.length == 1) { + values.add(unescapePathName(partitionNames)); + return values.build(); + } + if (parts.length == 2) { + values.add(unescapePathName(parts[1])); + return values.build(); + } + for (int i = 1; i < parts.length; i++) { + String val = parts[i]; + int j = val.lastIndexOf('/'); + if (j == -1) { + values.add(unescapePathName(val)); + } + else { + values.add(unescapePathName(val.substring(0, j))); + } + } + return values.build(); + } } diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/AbstractHudiTestQueryFramework.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/AbstractHudiTestQueryFramework.java index 1e68a9b1ac7..04a0899e2f5 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/AbstractHudiTestQueryFramework.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/AbstractHudiTestQueryFramework.java @@ -38,7 +38,8 @@ public class AbstractHudiTestQueryFramework public static final String HUDI_SCHEMA = "default"; static final String NON_PARTITIONED_TABLE_NAME = "hudi_non_part_cow"; - static final String PARTITIONED_TABLE_NAME = "stock_ticks_cow"; + static final String PARTITIONED_COW_TABLE_NAME = "stock_ticks_cow"; + static final String PARTITIONED_MOR_TABLE_NAME = "stock_ticks_mor"; private static final String CREATE_NON_PARTITIONED_TABLE_STATEMENT = "CREATE TABLE %s.\"%s\".\"%s\" (\n" + " _hoodie_commit_time varchar,\n" + @@ -88,7 +89,8 @@ public class AbstractHudiTestQueryFramework private static final Map TABLE_NAME_TO_CREATE_STATEMENT = new ImmutableMap.Builder() .put(NON_PARTITIONED_TABLE_NAME, CREATE_NON_PARTITIONED_TABLE_STATEMENT) - .put(PARTITIONED_TABLE_NAME, CREATE_PARTITIONED_TABLE_STATEMENT) + .put(PARTITIONED_COW_TABLE_NAME, CREATE_PARTITIONED_TABLE_STATEMENT) + .put(PARTITIONED_MOR_TABLE_NAME, CREATE_PARTITIONED_TABLE_STATEMENT) .build(); @Override diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java new file mode 100644 index 00000000000..779ef22a040 --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java @@ -0,0 +1,59 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableMap; +import io.airlift.units.DataSize; +import org.testng.annotations.Test; + +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; +import static org.apache.hudi.common.model.HoodieFileFormat.ORC; +import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET; + +public class TestHudiConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(HudiConfig.class) + .setFileFormat(PARQUET) + .setMetadataEnabled(false) + .setMaxSplitSize(DataSize.ofBytes(128 * 1024 * 1024)) + .setSkipMetaStoreForPartition(true)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("hudi.file-format", "ORC") + .put("hudi.max-split-size", "256MB") + .put("hudi.metadata-enabled", "true") + .put("hudi.skip-metastore-for-partition", "false") + .build(); + + HudiConfig expected = new HudiConfig() + .setFileFormat(ORC) + .setMaxSplitSize(DataSize.of(256, DataSize.Unit.MEGABYTE)) + .setMetadataEnabled(true) + .setSkipMetaStoreForPartition(false); + + assertFullMapping(properties, expected); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSanity.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSanity.java index 8aecfb78fd1..bb548e85703 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSanity.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSanity.java @@ -30,10 +30,26 @@ public void readNonPartitionedTable() } @Test - public void readPartitionedTable() + public void readPartitionedCowTable() { - String testQuery = format("SELECT symbol, max(ts) FROM \"%s\" group by symbol HAVING symbol = 'GOOG'", PARTITIONED_TABLE_NAME); + String testQuery = format("SELECT symbol, max(ts) FROM \"%s\" group by symbol HAVING symbol = 'GOOG'", PARTITIONED_COW_TABLE_NAME); String expResults = "SELECT * FROM VALUES('GOOG', '2018-08-31 10:59:00')"; - assertHudiQuery(PARTITIONED_TABLE_NAME, testQuery, expResults, false); + assertHudiQuery(PARTITIONED_COW_TABLE_NAME, testQuery, expResults, false); + } + + @Test + public void readPartitionedMorTable() + { + String testQuery = format("SELECT symbol, max(ts) FROM \"%s\" group by symbol HAVING symbol = 'GOOG'", PARTITIONED_MOR_TABLE_NAME); + String expResults = "SELECT * FROM VALUES('GOOG', '2018-08-31 10:59:00')"; + assertHudiQuery(PARTITIONED_MOR_TABLE_NAME, testQuery, expResults, false); + } + + @Test + public void readPartitionedColumn() + { + String testQuery = format("SELECT dt, count(1) FROM \"%s\" group by dt", PARTITIONED_COW_TABLE_NAME); + String expResults = "SELECT * FROM VALUES('2018/08/31', '99')"; + assertHudiQuery(PARTITIONED_COW_TABLE_NAME, testQuery, expResults, false); } } diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiUtil.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiUtil.java new file mode 100644 index 00000000000..fa00b76e1fc --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiUtil.java @@ -0,0 +1,64 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.hudi; + +import com.google.common.collect.ImmutableList; +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.hadoop.HoodieParquetInputFormat; +import org.testng.annotations.Test; + +import java.util.List; +import java.util.Properties; + +import static io.trino.plugin.hive.HiveStorageFormat.PARQUET; +import static io.trino.plugin.hive.util.HiveUtil.getInputFormat; +import static io.trino.plugin.hudi.HudiUtil.buildPartitionValues; +import static io.trino.plugin.hudi.HudiUtil.isHudiParquetInputFormat; +import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.FILE_INPUT_FORMAT; +import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_LIB; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +public class TestHudiUtil +{ + @Test + public void testIsHudiParquetInputFormat() + { + Properties schema = new Properties(); + schema.setProperty(FILE_INPUT_FORMAT, HoodieParquetInputFormat.class.getName()); + schema.setProperty(SERIALIZATION_LIB, PARQUET.getSerDe()); + + assertTrue(isHudiParquetInputFormat(getInputFormat(new Configuration(false), schema, false))); + } + + @Test + public void testBuildPartitionValues() + { + assertToPartitionValues("partitionColumn1=01/01/2020", ImmutableList.of("01/01/2020")); + assertToPartitionValues("partitionColumn1=01/01/2020/partitioncolumn2=abc", ImmutableList.of("01/01/2020", "abc")); + assertToPartitionValues("ds=2015-12-30/event_type=QueryCompletion", ImmutableList.of("2015-12-30", "QueryCompletion")); + assertToPartitionValues("ds=2015-12-30", ImmutableList.of("2015-12-30")); + assertToPartitionValues("a=1", ImmutableList.of("1")); + assertToPartitionValues("a=1/b=2/c=3", ImmutableList.of("1", "2", "3")); + assertToPartitionValues("pk=!@%23$%25%5E&%2A()%2F%3D", ImmutableList.of("!@#$%^&*()/=")); + assertToPartitionValues("pk=__HIVE_DEFAULT_PARTITION__", ImmutableList.of("__HIVE_DEFAULT_PARTITION__")); + } + + private static void assertToPartitionValues(String partitionName, List expected) + { + List actual = buildPartitionValues(partitionName); + assertEquals(actual, expected); + } +} diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit new file mode 100644 index 00000000000..f9e28873d52 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit @@ -0,0 +1,51 @@ +{ + "partitionToWriteStats" : { + "2018/08/31" : [ { + "fileId" : "167a0e3e-9b94-444f-a178-242230cdb5a2-0", + "path" : "2018/08/31/167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet", + "prevCommit" : "null", + "numWrites" : 99, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 99, + "totalWriteBytes" : 440746, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "2018/08/31", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 440746, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"stock_ticks\",\"fields\":[{\"name\":\"volume\",\"type\":\"long\"},{\"name\":\"ts\",\"type\":\"string\"},{\"name\":\"symbol\",\"type\":\"string\"},{\"name\":\"year\",\"type\":\"int\"},{\"name\":\"month\",\"type\":\"string\"},{\"name\":\"high\",\"type\":\"double\"},{\"name\":\"low\",\"type\":\"double\"},{\"name\":\"key\",\"type\":\"string\"},{\"name\":\"date\",\"type\":\"string\"},{\"name\":\"close\",\"type\":\"double\"},{\"name\":\"open\",\"type\":\"double\"},{\"name\":\"day\",\"type\":\"string\"}]}", + "deltastreamer.checkpoint.key" : "stock_ticks,0:1668" + }, + "operationType" : "UPSERT", + "fileIdAndRelativePaths" : { + "167a0e3e-9b94-444f-a178-242230cdb5a2-0" : "2018/08/31/167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet" + }, + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 1402, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "2018/08/31" ] +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.inflight b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.inflight new file mode 100644 index 00000000000..6dc689a285d --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.inflight @@ -0,0 +1,48 @@ +{ + "partitionToWriteStats" : { + "2018/08/31" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 99, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT", + "fileIdAndRelativePaths" : { + "" : null + }, + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "2018/08/31" ] +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.requested b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.requested new file mode 100644 index 00000000000..e69de29bb2d diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit new file mode 100644 index 00000000000..f1cc26fecc7 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit @@ -0,0 +1,55 @@ +{ + "partitionToWriteStats" : { + "2018/08/31" : [ { + "fileId" : "167a0e3e-9b94-444f-a178-242230cdb5a2-0", + "path" : "2018/08/31/.167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29", + "prevCommit" : "20211221030120532", + "numWrites" : 99, + "numDeletes" : 0, + "numUpdateWrites" : 99, + "numInserts" : 0, + "totalWriteBytes" : 22220, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "2018/08/31", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 22220, + "minEventTime" : null, + "maxEventTime" : null, + "logVersion" : 1, + "logOffset" : 0, + "baseFile" : "167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet", + "logFiles" : [ ".167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29" ] + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"stock_ticks\",\"fields\":[{\"name\":\"volume\",\"type\":\"long\"},{\"name\":\"ts\",\"type\":\"string\"},{\"name\":\"symbol\",\"type\":\"string\"},{\"name\":\"year\",\"type\":\"int\"},{\"name\":\"month\",\"type\":\"string\"},{\"name\":\"high\",\"type\":\"double\"},{\"name\":\"low\",\"type\":\"double\"},{\"name\":\"key\",\"type\":\"string\"},{\"name\":\"date\",\"type\":\"string\"},{\"name\":\"close\",\"type\":\"double\"},{\"name\":\"open\",\"type\":\"double\"},{\"name\":\"day\",\"type\":\"string\"}]}", + "deltastreamer.checkpoint.key" : "stock_ticks,0:3336" + }, + "operationType" : "UPSERT", + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 187, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "2018/08/31" ], + "fileIdAndRelativePaths" : { + "167a0e3e-9b94-444f-a178-242230cdb5a2-0" : "2018/08/31/.167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29" + } +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.inflight b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.inflight new file mode 100644 index 00000000000..724ce56ff0d --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.inflight @@ -0,0 +1,71 @@ +{ + "partitionToWriteStats" : { + "2018/08/31" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "167a0e3e-9b94-444f-a178-242230cdb5a2-0", + "path" : null, + "prevCommit" : "20211221030120532", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 99, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT", + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "2018/08/31" ], + "fileIdAndRelativePaths" : { + "" : null, + "167a0e3e-9b94-444f-a178-242230cdb5a2-0" : null + } +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.requested b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.requested new file mode 100644 index 00000000000..e69de29bb2d diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/hoodie.properties b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/hoodie.properties new file mode 100644 index 00000000000..33392aa182f --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/.hoodie/hoodie.properties @@ -0,0 +1,14 @@ +#Properties saved on Tue Dec 21 03:01:13 UTC 2021 +#Tue Dec 21 03:01:13 UTC 2021 +hoodie.table.precombine.field=ts +hoodie.table.partition.fields=date +hoodie.table.type=MERGE_ON_READ +hoodie.archivelog.folder=archived +hoodie.populate.meta.fields=true +hoodie.compaction.payload.class=org.apache.hudi.common.model.OverwriteWithLatestAvroPayload +hoodie.timeline.layout.version=1 +hoodie.table.version=3 +hoodie.table.recordkey.fields=key +hoodie.table.base.file.format=PARQUET +hoodie.table.keygenerator.class=org.apache.hudi.keygen.SimpleKeyGenerator +hoodie.table.name=stock_ticks_mor diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/.167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29 b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/.167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29 new file mode 100644 index 0000000000000000000000000000000000000000..da3c7bc07ee1189212243e811d255fb1f0cafecc GIT binary patch literal 22220 zcmb`P4{Y1T6~~L)g;4&CPA!8mmDb7H(k9fgo!BlaJ=cz%P{&DZ$89NHlsYaAxN+K~ zK-(dK1_+I+Wq=SM#2--@DyXa04zW#QUDq+hvW-DU2kO{h2z4Ex4z&m|#`f-=Z+_ez zlMi2{=&8x`d-~+=-tT+w?$@TSfp~Y*6#U9R59SQ`o(kU{@jdm5?cRy4V@2;0@8)8m zym{E$;vF5@$p1Y)Q7&9JI5ASVZXABNexz6$9``O;vt9mva6`E~JW?Dils9f1nE*%l zR7)k;8t>?qQVIS!KCyXZ^cwHl7VmJm072nL!^QPOTS^n&CEVqvmM@AoUfeWVb`ns4 z4PIB=>LlFQ(B_GeiIMW?;MmZ_1}7ockCcjo_-C)@4|sjKw1td;w^W9|Joy7p@NujR zc4b`1Z^yT8Tvz5p_Jtl>i$j~MzaJT$pusnmN68;Ujys9JVdR?X9}8x~mUX2f4NxlI zPz|pCnvR|!`tZ<1b@my4UnrHwiGv=$JXRe2fBA=psR@z>r2tB8Ni@caC2ZGL~q z7x9M!;cze{ex6+hmM^&ie81}PYHpr|+K&c(gKeJXbaz*L(C=#txB9}ZfwnnqKF>`4 z&&9s*;y{}x)z%Rj@)ZNc*2ua@ur(MAu5TS`3x!+#LBBuXD-5q|AM&^Qo|!rCb1{~* zhnM(#jXe0_OQ!k4yAGXRaP_VWn6!qpB&+n2c3Pm#3d`?l?#bj<)sJ=WR3jD-j#%sY z*OM!lw1%_B1O}@CVj~p%si#bN)X${VlM*KBB(3dG z1wGCE1DVeHp>93b1l7ocBh}2k$+xzhiZf{qX$h9}lGYZh-P4@Qbj9k&+H;-}iw8%n zuD7<$-20aSCaobY!IECm+G2HhnwO^%4yNF#po$gZ^@0aStoQn#n|<4&Fq77hmS9OQ zX>Au)$m7jt98f)EURV$tp#bS0{TD7{M;{+sI{oX*n4Xj{sm``cVUM>bzP$bkIQcDg zqc8GpWj_5sybLb zDPdBb1=A5zQExFmf_`W3x!K=j(i+kdF6kw$H5X`$ zcqSdKpX&~-1fsU!!IA5iTv{Aj%%nA>C0x==T5B#47uif-u70lPv?msF@!-gHLptsyPpl3vnUa|JxjnPl35XnJ3#9^-3~3E%377PeR&f2H_VpF? zcvr4;ATaL!s)~h$5Mm?M{_P*`>-+ftlU7ekprn(u)`JE5BAt(A>Ys$CW~o#-SUfml z{dN+*N(m;dAuZvOUea1~fxyV-q7I72L-UPXJUE$z$;u>5GDBLzB_?0h+nOunY3|Hr zSJyvYJ1;VF@!-gXTc&cC*l2+@q$OOcw>4MT)7-lv=U}|{nTsJ0j$B;kCf`akX$@%! zm-Ld>nkxe9E0=M=^^R6G+IcbL!K>r7>+;IR_;O}QOSr^_xauvqep7QX^!q%syX&8X zuW9;;7Y$E08L9TlNVPLHA;A&DQ+2f+CO>G5Y&PeBXlGnqR^nLOfT`Ueyn{8QC0wevH5X`%<Z^89S?aK-@Ml9)Iw}1Dc29*n2GY}lHaD)4FLaeRF z5=_rZs8H@dkznti`@HXgJqaeL{#ZK|h>U2Di+%A&cNz;uBae<;C#D5wbR3&2cE(5N zGDBVhhWlsmlAp4khFoQ2;tr(l9cP<{tC0ssF45`BR{DckC>vT@!lim!bAif8C1Va$ z#v2}mD~RnG9=yu6ewN?@Ye=hd5pQcQP#O6ZF8aA2dKIpARJ%Mla(%>il3Oli(i+kd zF6kw$H5XSI`M3j>@#zX97Y~kHs9$$%5-Q`)O`tNgw1f*)26)L&*>VM-rIAT=r|K6P z4{PcP$BPF?t_L6BU-Ox?hO~rBdP!@|1#%x$#AKl914vzWAo zw1i7~Nh`RH)?DZVe$PC7IAlCHf?@Gq-zpShpcOV=T3W(|Iug9(r>wc)+$7dJ;9$fK%~iRu)brrT zh1%#qzfe+O4QUCN>Mgix*Bk+UaFUMq)ISsV{=lp`cyt8A#^#$9iFgDgqMnz4sovHD z2AU(?>7s3T#M}<$!I5j-`Kuxa8qa6a8qyLj=_Rc-7if-j-bDxC@J;GO#9GgTBbR8{ zRuxj=Jt##TG1^}A;nxp;8o!p-7>0F%~`mT*ZgX|1_Hb3}VO z9c;m#yg=o`Ei(^Z9j|TQ79#-Gkd|=aI1z7Kt{`ZRum&|1+50IMe@P^z=_U_pPQFPC*N zSZ}LTK|gP=@!*Ih8n-DSWe-B%Mo&w)RBvl8P#cN#3I}I-4{K*AxVU(5v75-40;#Mv6E15W;36!;gvqK*}BT{p=39{p!wa_ti(Cas>7 zKuITQt!E&}jA$-5P`@rbuI^C8O*I}I$LjQzt8WkA7i7{J(h@G|C9O3VZ*6qDILJGx z$qcL*JUDXUruX%<*eZiHq$OOcw>1~Yj9A{q>Hj+ys}#k={2Aa=)?lDDy3;uahYoMFwR6C9J%o1=eBP!X$@%!m-LcWaMiv=X$Qg4opx}2@+s{S zId4TZKy0M?+pq8c*?nURnY4OR0wtZKwH_=kIHIc@)QuN4ZHNsv9=y7`e%dG23s^&1 z!iB4fcv}w^+zVal;>7%*QOP$zY@`wm&WkJU(2JR#lt8J@)`NvjjjJ3C)+zIWL$v2A zF<5v2d;25eUNTr{YN%-mm+EcH)q!_IU8tyA=cp9})kZ8{QBnMQ!-8WWCN10zRnrnK z=_Re;s$Fe#fZB*g`y5ol_l`|d!SJEXqvJf>4Ogv?ezznZ$~Twz>QeI(FnlQAQeyH` zwyUed58WCUkFK|`H(@nEY^1_H;^cI(p#-ZZB~Yrf^(+LXk?Tu47%lzYHQw6b!4V6t zlEZMF&7?J?C0x==TEX@EnvcCYf}S}E2RGzD-C;bsQ{}dB~YrffU5nGqmsNMgF*3jS;umOT2mAGR!i{aEd zI*aL936*q`9}-lxOWY1SUfml;U<3i zI&s-!5)p&7*9D-Met0^Fc3#=h6;llAE-qu{$ z#aQX!$@P@>I2O`9vy!}eGTBlohAkl7;WwXeyb>-^4XfUQtM&(S2vkNavD^Ex;XKH7lV~ovlZUUkOd*9jvYwv=>is2g!pY*30`>pZ>`k3z)Qq zw1i7~Nh`Q&?>U8_jSEcRk?__;Hv!| zXt;6@)WQ9Q1~2jBIDrfhK%qkx@PI;K+q7i8Ggq)&y8XTEeAzTXTWR z$i!AS(C{y7J1*=w@!-fMZs3M0H*w*{iJq2lsosLC_UBd@R7O|SLFM?s_~uCi#6~LA z4ZGG9tU>{_iC@J;o{=K5evI6Q!W=R8nA}6giG}nT(wKW zFo=w(3#=pmG9Dc_Kybvu>UK_~vbG{{KS<3=sPJigcGX!>)xJFr!(mBkAm!jH&F}8h z9$%>7d35B$ORD<{qR2l}fY*eyyaa4~!}s6+&JPPhnZT!Pr(p!-#`4}?2Q|aorD^2B ztK*egD!9NJ(h@ElUE*y!U=ffTJ=uf?=wdMk`k?C}CCw`x{ zz_1?j;K+5)k~4a4T|L00HKZk6(o0&wRr|v$04Mlug$D-v7m3pa zuzFGgr8-*=7ATEXooNS;#cK%Y+ z=nc?U)6x4dja&b!j G@P7gDfMlQm literal 0 HcmV?d00001 diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/.hoodie_partition_metadata b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/.hoodie_partition_metadata new file mode 100644 index 00000000000..340533d6e68 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Tue Dec 21 03:01:25 UTC 2021 +commitTime=20211221030120532 +partitionDepth=3 diff --git a/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet b/plugin/trino-hudi/src/test/resources/stock_ticks_mor/2018/08/31/167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9fe2112d09bb41e59909e7ea3a3a7b9f24d39300 GIT binary patch literal 440746 zcmeHw31Ah~)qe;fge`!O7%*VKf*^|tge5?Q5Nx3p1r>K>OCV7oEMcpnYyzSpT5(?x zcPolJpao>Nf}&EFPY^*wii%ROQp)npOY+{EnKyUlF6S;Y@Amuuzs%g_oZtDK-#Pcr zvtK(P|I(baq_p$WF1;u%F)b+};p{t75-R@Hpm6BW><-x-a&tR$$j$DYo!cS1Q|H8l z=TBZREzMYwmXOvkt!vw~O;wEj$!RH>8TF0DIXN9V7)tQx?KYL03 z_N?;P-_^M{x!>%k^FPTdS(Mwiap?6ChyS^7_R{>1mlO}kZPmD8odwCE*H4#}Y|kxj zoLlGNgt4ZUXPW;Cu{XKr%A>?hybKD0RWn(6RA*EP2P&(8lIAJ@Uw! z2kLJAc=hiO)%~dM|GuC1-1^h&PyOlj9Y1yX`@Zy;Wnz zm#K}DKkD|vtJ6N&QS-0QUbz3o7m|{1_&n{v-jPX9FH34V@y$m&J+?4y!-T~PzrMaW z>5|1Y4_umZpz-`=pA{uOSo^zMQilBW+9S`l&RFudCO3U>^RFKyzPWcqVs7eV^Kae# z%C;pdXD&MP*=ifEc&pKU`xl+L;>p8ZCnmpoUz0B$+jHU4L*33C@WpktGM@eZob00q z`j%bOt?KvdUVHwtY8SO@-t&>U_iui)*^K@7Ck?;rqM<$RY4gkV54K$Nz`pz5cw@`j zXZxJ>!}hdxHNU%ZYVSqYFKu-B+ULLMHu22AOlk7FuEQoJXa8Zqnxe9jA8T)Ik$mi{ zvGX6_b;YDvL-Vgar~39bsdaYMy?SQ%yJpn9x3t%?d)oi->#1o&)2~kMf7T<1t~&i_ z>hgX?ecrh$b7s@|d9x1}KLxHr51H(U1Yt+MRK#D8q~b@LCO zHfg$LS7y~Zo5n7w@neIoKW*H+@cQ2mxp>W&)oX73;^>V=!5bxmQSGKT^NdA44)vXs_F$E?m#V}- zsC-rK)VB*T8MX5iotFfST(i^w#`o4YZ7VSD^kM8@!wvD^$)>!uX_I~T% z`Yp5JOZ84&dBv<3pI*15Pv?tA?fks^<8>db`=56&atxzyPUkMevWq$vwaXixm)kBk zH+N*aVL4s8w(DRt!H(G@3Ws+Z)}dYYknDCHjIqu}Mc6uC`*Fi2jGt6gmH}qTpb*R! z94fE$C8RB{opyh<%FdD!Iv!FEUY}@eOGwB_DJ=h+kYL2-FltOjTmPVYy7!$mtl`}L zukT(kx6ehhT8+MU?%Mn(`({@^Yu1w1t(N}#lL6QEZl0RjuIAug8~Xm=tmX$g=e|*W zaLQdNHJ{5$nNa(wR`ZwqcJ=WGzCXSC$m74gJMW{CPA8{-KCAWTty8bvkeFHA;PdA< zKeDma^!KkAU+rYWwhwJ=^I_}#i~m)(%k2$z?EWTk^&j3T-Z-lE$kL6aZ%_U3WR>Zo zIv?KL{)%6gb(#Obfe!l~zwq=o^?&>Lqe*vsd)M&CPmDia>)2y!CrzmR@vU2?H~j2h zf8P6j(dl-x|9-sB>G^jaFF5w=g8!s{oWHQi%{M*xTeWRJe%t2WSMJE%TDQj?kL;V? z`__)wY!(B?sncS|GwqI2fDt$^QN+&_uaL8ck5q|-g#itZwvEke3 ze6o68_n~91>G{s9r}LV8wYUDNKc2Y$n`(LYPy775|2~ksapRI>wO)O>^{I@i<)!^u zQrc3Zk{Vmj_emdqRVN`UrC{LXaaGRDYQ4AWu3NKDzJ22How@Cre=_vRlu3V^+i`Eg zci(TDHvZ6~=YF5R^2nGj*LV5w{hyz{IQ96qwN~a_GN(YM!*^gDRyRQ0Rt?>c{aQrkKAz4F0#>n7bbxcDQ-aqjQ%fjeD%iW2@WDx@^Lj zwW|*NcGK9+|L*XyVvot^e@yr;ld*)-&~;Kc02mD0Smd zNlD9&>EJhQ_|Qn&dA+7zWn4EyNlQ*^Z&FW2(%$Muk0}4&Mt?B9S<*M6>!y^1iyqpa zW=K5bsdid!jHeNy<7;_B!p4Oc*GhKum6+W%!}$M(pUNi}ALT9W-4shp`*Glkrnk*~ zqE|-Bn_Z5rD}3=(Ps3D7?5M@;4?T|#I_KFTTbtEMx_?^t>=|nlXTq(gQ8wt7X^ zsXbmgvvvvP>t*srQNTA8l*aFKgX|TJsWi z^juYMN^wTPu|eZgH)SpFma_NdyjKUDKGyZRA=eH4pvBl41D`8Qz31n{A`R}w? zmbJe__kx`d4NBP4=B$FBKAySa%AQ~DS+{@7E8R;=x@PuxbNi%gU#pedq;#Kd zYsVT;DkJKE&D7w>Iw^n`etLsjd6s$cpVBYJaW@K%ybKapt+v~p8bNBixg?Ih0pl8aO-HrdzWm0zX z%;9Ixzo2Q=`R_ctB7fhRtMYeF99;k4si7&4wLJJsoqcy++JFAEUALvaK62KKRkL^9 z@yLz0?AkNu*q;ZVJ$&`6?KkXfH?!XH1&8VxbpBOo~`nHTk{WjG*J!oV8 zhAXS*@9cDL`oXi$E6GD*&+gFY>+S2C-7@#?1C1VgKlPiTV;4`Ddh^Yj z4wY29rru4HtMxkBtj_Jvz4hPJq2GU4wvWvJS5wTkEb%CKj)nKX=7n-)nqcX^oHH zZrP}}A*>V28ho3u;1Bgz*PHkL6?d*~ch8gCr|y_M?)|N&x4-y8wK_uz=G`!0&i9ME z@9Xkf=1*6a&HUoIbVFS~Ek1H%rMk8>OPdk?(5pP8=|1#67{5NdcAs7Ox72yM%12+k z-{Q*oJ#$(%YxH!hya$f%zVOGJQ=9$gcyiTYGrv1H{_U^EXa4m42YZga)$`q-hTQ$p z=Jnq$9{Ivow;mt&%%9t?I{NPXPv7`Qac+})&tEmBLBi`za%(?6>9P?=o_-;%W&b<2 z4!W#H?Hw<-*}idGO7XQdZr=LO2_Khr8#Lg9LEZi`<(c|7zj%4C2YUDVyw~VA+c#KS z)atiC9=rO1UAg_=OBnpqj%)Xi8j}8e+Sv`3j6cw)$H`m9B!5t6&b!&|mb~@awxPN6 z-d$VgvCR+Fdj6Ro*M8OL>lO!>O{_9|+nX~+{N;iw1$Qs(^Y$~f#Jz>Qjx+yV6&?CU&nkG4Y4SHBK~M zTDZvgy!$z=ZH;#~4tuy)^z&}duRo>M-8XjO%;XVWmM8tEMxUDwzxq^{j~+O3>wl`{ z@0p)hNcDMVJChL~amHbXn6Vsq*0Z|2)2#~FIXW@n+P8OX+jD$x;*vi%eDYA;x!X&Y zrta?bVXJ@6dGPbW2eNN$neu%1FaEw_<^Io`Ts(ixj$fPH`ChLxhabAX*ZiJ`R_6V# zMxU$3?x{CpOX+8~tteQZv3JdjUFRP=m^J^M{Tq6p{I=@&yH~X6^Vq$4eO7PY{NCXa z|9Qv z=a$#j-?r_ZAydBoWL(#&H$3^nTZLy9ce`}%if=D#zT@}L4w(LzdPgoOF4#M7S9;?o z_HQq%UuXQevpa3b9Q)^ad+)rx^v2K5EjViQy5h$&st+hz@XKvoFM4bE(J{BzzHQS} zZ_WPm$W!yK=-+OBUh3R>Q@($oveV6ZcFM2W)e~B6%Ni9uMpp|K7w=;<@7?cqjmn<5 znChZ-pv~CSeu?$(zqV`Al2ac~+J0x1m0L5`y|Q5TiW?`@+c0?I@F!0lo;mk!FL`CW z^h&jpP12t&y=BnNpPzZg%YR&uRaS9Bn|G|y1xrVjwd_7)_R2y3*mL-Bqj#Q5E6(3H zrCP6@Bicq!(vFgbEbJxFjS*^iuw|J-5M$1vUJNA;c`0Y6>Miq4Ev*zx08+Lcg zNnKLha8IrA502h=;Hxtl)ja(D$T8I)*m|PwvWKs!H!-tzlNa;9-qvQ`st>B|ZMw{m z*P8FPzgzh&jr`6qejZy|wB~BWcZTt^SYpSV(04|~lS_Y0Y;@v`zO^Qnm5#V^NLk5< z8z-hUPe`lXA~B=EkmAzP!jhsPBTC1NDJe51#}u_NK7K-T4Q+E1i^h*F4NsmCo}5rL zqI5#xkQtLPlcQ#EKop_{3>rhL?^Gy(StnNz;mkO{iE@ZA|Id3d=P^6U8N?iWAa8k5xyP zPN`T@HG<4oP&lltJiO47Y9mIMPBfgA?<^fxG`4bm;jmC7>6@xF%4~F^e#?~1wAzh2 zB_*dcawID)$yk~GXkz+-#Fi;N9RA^i!;q0OynkNNsIBo8nv|9p`l1c}Nl0k8N1VglyMP5Pg@;d4FC%1gLQ4fM&a?R4&aCRt#Es_koM!%@A zxi>j|N7a@a&JM}Z_Vb|2Z-x$_A3Uercsb~b4piab-P-9(l3PA_PRN1zmq0VH6|FPV z5-Wg}x2uYsbE>58sM2yp>mDvYNI3wxvrQz*iiTHVZEE_KDlPA9(<0p)ZHg1aq8=I!COATCUmXqRzBNxAu$>AL`x} zMz=IipOMmXYfsuPv-OV=63dT)N~=}UA53hy^^ZU;WnaGt9Tmq*gpgG=TJGydOUUDw zMa(9JPO6C2)iqi^ei^ORo!3PescIa06-LWNwevbdR590!SW%^K`m~C1{3SFRozbB3 zA5yg+HhDs6`-vlpi^dGgSe<-R^Rj8QYZws~Rc2*ao`R9G}}*rd^A&AWw` zOmB;_SALfPF&Mr7fS9x1nLtcN?paLMD(+?x6U|4L8qAvrE}IygtIUgOwxWN!tq5J} zqVpwV%gn~huX)kgSo@)yUv$2(bkgwAMP>s=Z*rS9x=eIB)_UmL=r&(|lZ-Bibss8P zw|S%dqSImjgQq8C)Soh;q^xK{`!S`3MWfq?Y9}KpRAC7jO)4+$!$+5vju~R)*jP3s zR3FAwzOX2xW}mAs&hOu^V8|8O2^nXVP8ijG*tlUvRkkmlR9Mo!d{f1qjQd_}e17O( zQ}Xlkd*}AfFDT3#Fe*R4^BChjzxNHH|D8WFw4iI~sVIM7{@9V1=Z2p3zkI_$Kc$eW zg+>510|oeSa}hkkd%J}ze9q{Xiv6pA5RZ}yxsWgjz1*)1=uG=Hb9US^8wuw&40(#v zoL@4jwWrzrT~ayIkm?=?Y8FxoC?9H#^GYdx&GJG(%yUKZ{Zoy^0opboBaKePWYM`m zq8ltOoEKY`DD*;lwN?nahT_41ae~B1*<%a*w5`!0iU!`Vwx}8KU_jgr%L9n~X&Nd( zN?G#x;~ls63ZU$HFk$PXq)wH)l`zIb8o$^2@`F-XuD#HxIxqh^p#(FU22>g#DoC-; zfC^QQ^J1VF$S{N$fuQg+V>#HNOGkL4gAO3dyM?Fdu(T&KYC-PWq_MgtLkc3&2pHFU zT*(=&18-MN;6`|cfP6(Ea7Or=D=$FYBaW=w@CB}|!!uC8Il;1)IkoM1;_|kZ+B-v5j8vL%7mM}YnzNwKcZU# zwMnv#gt?NAmn=Gm)o!X#LlX|5gFjR$ETtO1Qec8F0ZWAZN|E0ku(Fqe^hg^@5Stc) z^$QdOfyhmT-Y7$fo~eY_3CtAmv5M(j!m=oz?#^vZ!hLb&EX^J)!^$jTP>V(XmY3M* zf=^^@%)uMM%c6%Al%z#AW{$UKOVSw1-N+k9l$feu!;k4CK+8OJtqJvEwuRLp?%fK2 zLxHXU(6X2)u>BvSxJB+Z$AZD{dkJ}QD;UQ)oL^M@)jA0o6D|IFw6>}CiJ`{xc14BVdlQylQ^C8lVOkh*sh>Vd)o21Ti~d}(+76czFmkzQ#3fT z)zN(wPnvE!Bgl7rqM?r<<-U-@9Jy1XwH4;c`0GxUX`$#xXt8>n0a)RwB6!NAK~rjQ z#$poha|&-&G<}rZL=AX8k;F`_7zmkqMu4{<8YgiScn&P9qTVdS9A3D1;x9WFCZsxd z!9_F`dn^YcEKCmIN-VZv#xm;$Dt5oyYVtX@4#&2e9IfjPGGbj!Ib@%>OF7tZ@rXIa z#VDq4y)4RrSlQ!dJ)RnLG#Az<7Y>mtcm=gz?KT! zZp0yIp(YW+6pBGvXD)44B<7eL$r7inMnrM|?@O)IP%SjjY)chRO!=@!r$}R94+PqR z6ET)7h|tN6<3c|Va?mX;9kE%%=~q&#o;Fn3`*4VrjZym-&VV`{kGQHW{huds z3N=igVX9_ZcP!O1LWM_HkA=3_E01Nm;zgaV90f<*pQBq*Pb!9q!U{4TcL9U2rj6h* zfYn%{hWvTD3cK?~iPBHf7!cMj!*>)Cd6~{bL@kA_BKa#{fY^-Oq$kv8z$^1nondxu zlh}AhkBJ5cf)@oiHbk$9rbV=8cYiXu6r?A6oG_){^ys+_ zZvnhTC%?f+gi~bx855Ifc0Q=I;`iGnG=5H30(5o}9T>eqxfuw`{Lg}NX<-J$P+B^j z4(vnJiGx`she*ET)Z#@gGz7_IC|3e6dZ}Oruv@98a4iC{ya*0SVRxP+O2O{=1>(6t z3T*YAk{na&N24f0KZ{eUK$Ia)Dhn8@=0}wAn+iYU7?FutA~!^;->odf;9X|$R#^<| z7o`~>x~+v5hnpF@>>#v+GJ;N>Ho4(rU)GFIGaaBNbba z;bUGVozmi(M;p@0S7#U$hMdhijDnY~gAdyf&v0z6U|$smj?y(O0lu{6RGR(@1WR|$ zt;1h|!lT7(-|&u67HJFB|0)hM5}ZiF5$V6=48&8ra#TF3Gr3&Iu+9PURPfrNT#DWK zLnVY4J@IANoqHE9LiHe3Nwg=nOH9F;a%^ox;VD*jF6|c?7LK-KRD`n$>CrL)XUSq# zjTh+BkI4f`mUT`D<6RxEx9zqxLmUBx!Kh~mRS6Vjma;iNCpMhu*h>w+DY176M zSB-#M%cfgBSZ|ik0JX|7q4PQp#1{jAM@kc9g-7WGGoP%> zh+^R-9@ZfAem^lFOGWs8rIjpQ_s0*6qfFGh>vIg&*)%eHI5$wDM(ph0HY!cUHXPos zq}o9e?d>YOg4AVrALU;vcS_W?bHD%@a5jt_rBgF}RUrWaPKakW7CcVRirwY!5hAr9 zsxuyW^LN&Tp(m1;FjRPs=-MxyZAq(irAi`RyY(#tvT6o#*vjOQydx?YgiHcC`IaS7 zPT_>j1~ja2cnbaX`uftD5R)uZyEOlN;*}l}aEjc>&Eet0CkJn06yFW6mE`9q7!oxq z*aHQfD8M)}8v-dga=Drz#M858%A9RbHUxD*9u9d#ZyLet)j+;S5kI3fs^V&GJ2iG>p0$O6nG8FfNg zCxfQcYWoe#yTrla2Sc;|7F;lL04_~!P&1$q1FH8J_Isx7II_O#&!!43s@$yEC7@8f zIYAK^phKQ9u4gJW34vrekI3QPLvJjYI|2Knxh+*{-aMoL`LYcJzyurwoj>7GQotQ)9niHcKsU&R$U_u@+6HgX!cZc7!eE6 zcNK%fAm=V_VvDGTA!3QgBEiQC&H>Lrg7a1>PMx86j_Oump$L=?0-ahwNPMwE<|bsT z8@6~QHX=6QG`m6dNdq8Idwee)ZUst~W}?d{nQ4EnE+<5hY9?%~RbYkSfuH zo1&Sp*8mwW_EIFT3BANwmU7d)8(|rVa%@#I)nl=L`h_fcbK%sm<1YO=qW%5f?!5h4|Num zDi616V9K#lQBvnBkQ-fc?sU}%BnSEV9v4Ixl3)mlIPd@hp%G z%=1MQV10V<%p#2KNCS(Q6QvHX-W>r%`ViBK;o1->6u|WXEGA(}15AUurmRP#^-T@b zxOIrTPK~G(L*YQ4ids;NHHmN>5+)H1nCsY!1(<1VC^!rVxdX)92wC|HT+Jem2(Kaz z91pC*;IQ9E7k}nD6VeSVIYJ2EvL^Jx8X8>|>2}3{#~YAb%;Ixzr-e1gc6)5^;Ap&> zITqXsBa)?pg%o#@Ut) zkvpj*ZX%~+1PD%eK!R8AVichRhXt-{xFaD-IM+}Kj7zE#0))O?%3W+Kv0}eG*xv!M zax@KyQjjTSYd3V7DGN?eazwjPqcU{Xqj?9q(?tz}J;hg+0(TaxQR)P0^rzl}q8rCJ zV49OZ6HB06K1;E_6fqY0i+)EP@*x<+DolyQ{9|b3Z#1NBy<$BON)X0ijmV7*nWp@F zmnm2Vd|^$kvQ|Ggj(uG%5`CdN4bqtbcPYtjs?Lbxc+_7vICT;wkvGTwtqwDyk$BP} zwBx9M{0q$w2pvub3W#AaDuhZ540V7FIY}m<;kt(^@tg`M-(+Da+!;uJ?jZ}@yxQ}K zHZRuC;uv5&MSJHfEN^iv+C9-1)?JCIK|I!NFIXf3aTE~kjA@c<=3hk{^0ZozUm zB@l7?cvxmUmxC?sVDar#;DGq@t;bIQ2(c-P* z0+89LmIO0}+JF!S2C{zv3&APa3FT2B3ZAG?QP`RXWM2}W-2i(e{>c-b3)~ju zsX2IPP!{o7mJy^3pHR#?+EK6DL=gp!Y)b|(qqCh3789zz;I=7M@iXDVZ(vkzqUAxg z-E9+)X&^IhNImC>k`UK&^`w%1fgf8JeYBW6@jrhs4f=DLQZ-lJv3#Ib`JGih0(QA$ zUcBfW5gj~H7d{YQ6fD=L%IxSi$(3c*jB~9?_F$ETM*vKz6PGd4#bh z1b;td>L6|5N1#`l7(m`B37VV99ONS}1a2kfHW!XfUl+qn1ne)jvqSf}fGI^>I5-VS z9GVJG6_~hHi#?oJN51dCq(>XhdKHyoPi6eP<7A8<8-6)(WU}NJj%+HCNtq1_(5r;T z6?>!r#o{Y7%CZ2Uao=AN5K5}(TFPb)ff_N4Xu&k;_h3XqkEx8=k3#ey6Oqk$9rj~} z)yS2?I+M{byo4kJlebXRMCFZJuTX^nG6k8dgtUPmJkeY@=*dI=BNuv;ZadNL%uKpj zAea-&BmpqJl#&6od?*d-Z9Zl|*SDI<QQmdo8oEIHv~xr8sM9(Fu7|EBUDRPa|DNo z2iA`V5g79W@?Wqd?U1KRl4@xEQkp%?#UkqsbsEPXNFOu zo{X}c&VWB|p(0Ry+>>F|17OF;e2(Mx2X?3k3y26zxU6-ela3(LG;C$eG~xCL*dD3; zyI~8^9769RB9we&L{$M>!)QhwXC$-0Inhy)%N?;4HOpSuC{2d}li^^eo{1W#&F1nk z>B{uzv@1W~bAj_VJ?bw82&)OU={r!RW#>SOe}rXJ%k?tG~IPw&i|6(uO3^(1(AbucbEaAhU*%|GWvCG* z%p`szcUF>3UTb5FGW?cRo`4vGUI@6%hqb`PIK5FjEMeNggRDmG@USxIG}eJrt1BO? z^n%Khp9qVMnrXy~e-vfTPT_iXI3-Rgb@q)I^$HNNEjny7Kt@Q^j1QhjvWK9-MwZ$^ zhc(8Gl0Isbyh&0)1s~;@${6-0Fvep&+RxVl#s`d(Rf`y-XuVVFSPo7e!ef>; zOuRd6C8&R+)N_c?k!|l3b3lS=(h;!hUV+hAgzO4R@ky?lqlld&9x{1w1VTZJWea&_ zShkbBIPqY;s*%|vUG3zb)P{QNF|>%WM7_A&5m5=(q1B4ipL;~2XE7reSu_`7|2rqK z(vKNo4Dk*!iCLr00Bw{c5r`B49xj}ghO-9Go9I=R&nQyoGUcnDP)OM$T+cUPtTyL` zIK~KEU5J!gOc-8q>v_#UB?EAZSB+9s1#-U?@NImOX-2Us8J2cW%s^024W=7n42uEi zkDi2Z?}#z1${Jvb)RRY9QN@ejKp(7?Nd)dUOoHd7E2)L#aqKBc7_UTF?ea(qu$--c z$={^H@whsBOp^zzcgSEsSQ>?EIY7s+jXRGsL57UD`m}}5)t5G|8SrKxj;~$bO_mVA$V4q%~Fg3RXKn?W*x)U$^=cmBgdc~BF4(aVN-TwQp|%~?_y#!dvUKv z$D_u)Jzk&r5X0QL+_M?4$-<1=Po+GMh*Eo6ct^q3%=Cp%m#_9(1 zEsEZ~8f7CjvxXTO^)h!R5N1P-4uB*86Al1HvghC)a>yVks#!?GUbY-hGAgNfdD8Qa z3{ags=1uHXz;3}YG9iONRf9(XaT_z|362gtuyFoVu|!P2D!@%R0CE?WJifSeKR0Cn z|1nH7&JIPjiLVj&AW3EGc&%`E0XQqyJ76IEO+g8SKT;VU6s>9*RLJt_AyAKL(Wo)t zken(M@FQnEXFn9s^cH>$I3IZtG4OL77nYsb^$tC*_z(3ye;t^?ZjoMR(LmIt-ahVS z$h!{oil8&VvC}!q&v6izWS8z>uMk+rT&%f7L*q{k28hmOqQx|v=slneaBL~wIK9LY zbJ6kI;O#P4D=4z3Mo$S=8q|u0LKJERpwVz;KvjK6#bv+h+TXcm1je`zA;Sc2vcltn z5OWJL`BUQ2v2Yv*m4NF%5EOSNoG(F~HU1+@C}J=Q{ZTH;w<4j**Ukhn04_VQvK2%C zu$xh?{|JZ29R@f6ADRQ#un#%#31okq#Y&PH?A}3f4;Pjtm9YsUA=VLSEyvR@9Pt!u ztZWb?%Ct9?sy%3z=>Xl-_=W=7TM-o}b2w8J8#0A&L zX&dAWc%2y0iw3!Eb#|ztQDuu*om7z^As44ALLiA!uMGkN`tlrvEFxlX_^fvrLTpA3 zd!5b5jd~)$42ZcyXzhVYNKDc|A^XdizS#LIUqMIfS@6W)EvIQdk>Y{+4xZ;lihD#r z7cT~!F~=o?Eg4&V({xlj)0ptcuN*wBq3g>xy1@%nuTqu)o=T%DV!!u%KB@NNl4lVzpA|{*pT+ zd16Y3-V6WQuzMJl?|lMzuQ2B*)Fa4tn0JFd+k*u)PGtfDFk>=Qq4WG_QlC1b1PaP=rMwB9W#g4X%o>bsi!Nrr)cW%9jIc{_Zfiw&P_6hhMar!a8K*DZhZgv; zZN;VEL@|J9CT91#ON7~AuD7tEVE)C>QZNJ8Q86+=cL*^$#n`UiF&D@dq)!+OE$Ss9 z22_>`x?2SLWJHH;!B`1w1-WJ*uu=OiFv2e2bt%FvijX>d{u?!jz;XFv(G~9T zONeNVMlfytn5|IZi9N34Q zleL)5R+Tz!aT~+yWhe_2_mCL;DBF5mGoTr;GvJ52mmPELdvKZnTmOt@M$QORKG+T* zIRQg?r6-Q!=rlmb8`mD~E*RAp5u>-7)DM_W*AWWzgOveIrGo+>9kpz!vDu}+?WF5oTb9l>$Q* zQosZVaL&3g0(DSDGF1TlZ*=hfK#jN#_5*+pR3NCID*sug&SZesL#3OL-B4?FDMe>M z!5WnzVqPirU1t1$w0lt==hfv!kJEf$DPvZrTBpO(d@)M29=TFY}I8;yy9SDve@L}A7l^u3# z{~J-fB6#fhJl`V$c&-fY0X~U|d@GfD0)o#s_ z$)U`G4~qZ@vX|H?2^F}_|G!!YBn#A)vf6RIYFi3N*$_Yz#pD0l!87leXao(mG{DRT#>HbT6S zM=8mPBoaEVdmf=%MQweH0K|E~4Fbm&$2ID}By6Ps$!EweA|r&L2t!6VrT%YiQNQO3 zKnRrsWO4u}7%sK&S_h1a&J;xE$O@KSj-(ds>%%eI$LSdK|E@3}{*OdQ5X%5Vzl$~I zHWDf)Vb0uu*ybwQlZwb}Krvp)qM*2!vh@OGIx0>WIBPIc5?-QA>|sFqZzTiLu5DTp z;x$Oo^TdN>^C&h%oFfNk4MEJn<_J9NrI-wmee+=&^;#xY_+DtR<~d23$Sgyyi2MNk zXDyPz6{Uy@KcMG1@bwc*jX_!jnv5V3qhsOAfT*5hio#D??@ij+$j%rOEBaA0Ae{kq z)k|83(y0^zKp|Fm93%>b2cOPB2M14=42wK8Eh7$J_=F)_#i|+zdaOELpE3Ta0Ld>A zID#R}mx_tO2~*)UVI_c5i(!y3`%o+vgoPfC9m+6?+r9N2#-4c8fHeZNjk#v@t}m`hk`8zdEslA0#{(Sc;1R|bpu8} zn4!StED{(*>yH@2euyLpuSv(0G$3j`1Tm3CB8Zt`9H7L*=+&>=aa)UD1G|YjiDxZWl~c!74eCOp5artQ?_rtKiK;T$0hW_+UoT zHA8Y?D5zJyT4CecbWS?JB(?2s&A|cv5a6dVJ&3s4Nh$^eIRpi&bL_|=P$C>{`)5$n z@#Y_bq##vOn*=>a3Tp&T743<(J2@^&+;s#5#raZOB6zbQ@W#g)3gLm_NMv}BbDQ&b zSzrmy6PHqEqe{l*=(3ph835txT8^&tE zhZavF2aXlkoOjNIiNXa6aXAxjaB;5aRKMtdt{wFCiZWv1(xk)*vMml22Er5C2;vg2 zLcc+S)-$jSAnxt3PE@qUD{w?rIZ_Z=OWVw7_c@}`gG3By?uqD9Xt5&KHH8>a9}2o% zqi48BMef>qxr+3E?nt!*3I^ipdy0Y|(-wtWYGsjHE6)U7^+(?VSC-gF+g4Gv5pXi- zLmfz{aW;&!R+UH_W1h@0n4j4gb=+-hAo^5uWQoI|jd%_o{AiN1+WkmL1!kGV2yF_? zb)kQH@lrX8Vr!mJtGulHW}fko28mN80OPXHuQb3Dw8XwKw;F4F4RjUu|IMvQPS9GFtMsPjHNgw4^1;@0Qrc>^PV-haR zCtw^VWRB6m0OVywF;(#jldIjh6%P+qWeO6*x46zkc}(Ev#1e|bfo=XF8{evwmA-+> zC#edBZxNLgEs#0|@7|Fvg5V-DPl{BpXJVkD#qnYvk!&W!db1xGAZZ}H)Mw(!;UqWB z+8;^4L+-WP@_r|vj*J3t*Dx#ip$cs8QRrd=qXEUOuyg}Hp%GNQ834apGkfuV4q9cZ z!mBs)imssSXnxR<)&@V#@%i0=A6{)4hk}n?U8eAGJFg0B z)p?fNsHD=s%N=)41VtZ>!w>fiGOa~pBimQ8nr%0YN4 zNc`j4M6UCe5RVX%aA+n3eNfUsM(#`r|Mwl zg@l3&;w_Oz5qYSL#{P~IB+VWiyLyd@0pb*!_WI$G>fXk3M2I->Mfk8LmxU3lC4as7 z;>R&!N`*y5Rei+a=neVeI|66YJpvMUY5Da0i98YU6@IC*cqQg?3s;E-h2}^L~n)(7SXVoDwb(8OjyiT5WKM2 zm7n;y_ zL|1xvPeRGa8>iHI@`k5Z@MXXsCx8|yUlkbE#dRF+p+p@7TQ}afkth>^dy@PF()D1gM!LEL6y9ithGn4k$z#4Vw1Jxi!&RBirXvV%Cpl z1EfrxWKjiYwKWOmxKus5C~&!m&>XREpUs5>*I_7#c4bh3ETGlyid;_ym4VoM1V;V9 zsmNToL8TZOmz73d-65_iMsgq56Bs3~K0l79b8I_91md_GvDUA{=oqb?N4@xX^+Hpm zKCc0GhS)3+o+@Z5no{EOXx?vrcG;q*z!_kZ3^*K{K|)E|cI#I&1LC@)=ZVdy-1%5F zQ>dT)&VUazLRzPu)#mq{8TdozfZ8V&Gs)OT9f>{y&KWc5VFOazf|w*>>nqF&`mfp@ zhonBxF#^p%ew9lVDG(pTW|*@HsC#1Y*c)6ys$Fi_F-lzsS%HA+A?>nI&OpIlNTtPu%w#FI$nY?pt-Dg7bmqCMr< z3;~jW7yLlxB?!u6hZh_r^&=OQK2hcJ?|<(g%}C&3?_0%7IB5K?PJy!WJ5`kUN99wV z@s$4PxA_GpeS}hX>Qal+uiO!1T&SrrF&#UYmjj{V(TyOgRPU$Ye~V#i)SD{c6+(+5 zjGekogbh^w1(6K}7%8OLVyAg!>awQpPQyC|&7lxcb>o8+`1%NN1&>M!jV7u0Fp8Pl zuimH`Krler;}F&a40|_nAM{xeo_MA%Y@4!Uxwhg^@ZMQup$J|9VLn(;DZ1B4ae$*^ zAe~P8M7xLdYl0dB-_J;DShkJcr>yr1 zi%CaIt( zU@;Y}V2vQ%nLD+eV`RfJ#bMur`(9}Rz!I)UG3&UVb_K|PLjxi>x9lRekPah7LTX6p zdSpEaMd9fT_d_CTNI=G;&M=;2%EKo~#$Zw2l1dK6Q3=;T$%{uG%}kzU)R=D$ZWB<` zBL&}z%Sl{?QgCPAf;gpEgp4q0_blWcu7?SM>t83DXxiMdlp3v1r+v(Ai>56jWaEy- z++f^SI>o8tm~#}&${^2(GY}Oi!@|VOAeNW%MU<}a2@gBK-+>QaujJ1F+ntvhAER6_ zQF|_9gi#rOxCW@%KyD;&m1a|jW6Zpy;u(g9Si2HNSqiFe9t|J5(PC$4BCCQ#P#lkii9LD zC>y>Rm)-3762-GeP-zPhE>D~YmwFhR0geqpCmuTsak$X)N7y*z#64uM<4iN^&L8Hi zrV*1UjAa~87Yy$TEO8*fzn?qyV65+Q>ZbgXf$WUfR}iEHYz?tnz;@-d5Ia(*&3I#S z5*)(D&aF3gTmr44$73$Mie+!u2J>IsK$#uo6a7c&0JxKFf?cMS5kP)^%O*wp4+H~F zT?0(-5r&j#kP$S@5AXyrLy9{CoUhx;Db`hU)CY^JRd`KtVwJNNuzBDV9=3nTg-!

hr@rx0g~n^tOqV3 zhmB)JhrKvb0a#|aEuyPGJUCpkf#*7CtGs`&lo7Ea8kP}yT>8XrZlc*yvpm392zYoX zo?_ce>QNUa7iswBNw3fvHy$lqy+Qyg*$uJOOBlZVo zn9B!@6~ctX2197TZ8=MUV;?4N0EEo(DBBp(35mqPNks}lLNx2!{OsV>*F#9vBmh3erUKB^*S*O#FlpKNCb(kDXT718{9}3P!GsTXLrCGIy*vH7TjA6ZAx53FDp>L z`gNr;$Ok`T!d!V{w2ix61q`@YgS(F3xf;*qho@~p4B>$ZuEgmh<5wK?m{i=U6~-wm zz7(J+(E7WDSatN2(SGa@Lk#8y_b-L#OyGKQ zY~jRjas_D!!%e@~v|#MwwIpz&=S38iUCf?~Xx|MOnFy~52OEZnUn|^A|y=E*^?Pw?CD0DL6ZBLSd$fCKf+e9W!D6Fd*cT=0SxD7(5^nOw$w! zfqBGIuCZ?S1Y?nO?-fRQ!8au~_Nto{0vf!M;a$Kk!gz60akTIU!W zgf0=#Z>83_Z1yexTtpBP0Bv7@EQHG3!x>Px2%OEPW~vr z(gOcvDPpCgbtsL<{Sf3v{k1alcg zFoXeET?+j=N0m80vOvDFtc*#1LviQc&dqP}LM#wPC za_ZYR)K8dfVGs&~7&NSmqp^m2L39N8iMs<}yxN?)42by_(~1RDfS8mK0H!`?ynOKL z&|ZG!%@ZGDKu|pjlmQmuDGn4*#0)oWMm%tT2tWwjIihf&X{8`dsV#~h50c<9^lGf^ zy4nxL?79HUh0`MO=ZJFmEGJsX#+2=OJZmL@c_hm^$!b2uOM<}fYgLe&$8{crV(2(o z8@#f;jh<~k7b9yr=~<)QsQ}v|>Xw!sy-yGWbd5|)M^Jj=-3{`pj+}t`&jJY@C{`Dq z4cmWbP`rsNm|e?LnDLfo9j|5}t{4#40jx)&JINMQbk|IG2MW$wVy45#{Te}%L%I;S z?BW4Cn5qnBR9r*_VU*}r$&wqqy?Co20 zW7X;%?E{a50^9ex)q401eY%>|KvdCT%R>L6$r1`fJ!k0rs)to0LNK6y{ULx$8jj@) zuVqSt%jTA@^~F-ehJxd69v?a!;o_J!ovk>_^%^?^PW4Pz2zJZ*osa=@D&&@`83|z! zYAlx&fY~i7gZA^n{a)!lE0=6gYy~k*2SuRVp!s;Wi-M0AIacj?405cv4BI)u-G^Qm zQ$FaWkr^Ll!rWOS_}w397;(3#XXB6oj8gSlf?sFOpEH$8#y1n(G=bhZf=iQvk(o26 zert9vL?vi76A1uFcNG8<{b~YOOMayR7dL(1qPV}}f(raPde#F^4S0Y~S6P*tgcDu} zQWT0=At)@&Dtx0N3#;1U+ zj~HGzo>-t#M#{tUlr?W?j?D9RN^KusQPeIX8#y!MA9a%xL8)pX=Z_F45fQOGkn@h= z4K#fFnQMQ?z{9x2xr!+D$B0F5*9?#`Fc70)TnpB9k_?MpEt7%YCuF1*mzEZm6b%_u zGIq#>q7kJN3WwZSG%e$d{QLnI4e5}b)3sf8*LEFqvU0LbYlls>tiAfet1h>%{nNN% z6UI*}Dr+}pLe}J>2@^|7$7bcU&&h6|lhtbYq>|BvS;M;K6^`sWGB2-VhvA)zMt1Jp xv14I&=j_gTMV+!o Date: Mon, 27 Dec 2021 18:53:26 +0530 Subject: [PATCH 24/26] Remove debug logs --- .../parquet/ParquetPageSourceFactory.java | 20 +----- .../plugin/hudi/HudiPageSourceProvider.java | 24 +------ .../trino/plugin/hudi/HudiSplitManager.java | 13 +--- .../io/trino/plugin/hudi/HudiSplitSource.java | 68 +++++-------------- 4 files changed, 23 insertions(+), 102 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java index dc607cdfc25..d57d4a3909c 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java @@ -16,7 +16,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import io.airlift.log.Logger; import io.trino.parquet.Field; import io.trino.parquet.ParquetCorruptionException; import io.trino.parquet.ParquetDataSource; @@ -105,7 +104,6 @@ public class ParquetPageSourceFactory implements HivePageSourceFactory { - private static final Logger log = Logger.get(ParquetPageSourceFactory.class); /** * If this object is passed as one of the columns for {@code createPageSource}, * it will be populated as an additional column containing the index of each @@ -201,7 +199,6 @@ public static ReaderPageSource createPageSource( { // Ignore predicates on partial columns for now. effectivePredicate = effectivePredicate.filter((column, domain) -> column.isBaseColumn()); - log.warn(">>> Creating Parquet Page Source with columns: %s, predicate: %s", columns, effectivePredicate); MessageType fileSchema; MessageType requestedSchema; @@ -216,7 +213,6 @@ public static ReaderPageSource createPageSource( ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource); FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); fileSchema = fileMetaData.getSchema(); - log.warn(">>> File Schema: " + fileSchema.toString()); Optional message = projectSufficientColumns(columns) .map(projection -> projection.get().stream() @@ -231,14 +227,12 @@ public static ReaderPageSource createPageSource( .reduce(MessageType::union); requestedSchema = message.orElse(new MessageType(fileSchema.getName(), ImmutableList.of())); - log.warn(">>> Requested Schema: " + requestedSchema); messageColumn = getColumnIO(fileSchema, requestedSchema); - log.warn(">>> Ignore stats: " + options.isIgnoreStatistics()); + Map, RichColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, requestedSchema); TupleDomain parquetTupleDomain = options.isIgnoreStatistics() ? TupleDomain.all() : getParquetTupleDomain(descriptorsByPath, effectivePredicate, fileSchema, useColumnNames); - log.warn(">>> ParquetTupleDomain: %s", parquetTupleDomain.toString()); Predicate parquetPredicate = buildPredicate(requestedSchema, parquetTupleDomain, descriptorsByPath, timeZone); @@ -257,7 +251,6 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq } nextStart += block.getRowCount(); } - log.warn("Message Column: %s, Predicate: %s, ColumnIndexes: %s", messageColumn, parquetPredicate.toString(), columnIndexes.build()); parquetReader = new ParquetReader( Optional.ofNullable(fileMetaData.getCreatedBy()), messageColumn, @@ -301,8 +294,6 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq .map(HiveColumnHandle.class::cast) .collect(toUnmodifiableList())) .orElse(columns); - log.warn("Reader columns: %s", readerProjections.orElse(null)); - log.warn("Base columns: %s", baseColumns); for (HiveColumnHandle column : baseColumns) { checkArgument(column == PARQUET_ROW_INDEX_COLUMN || column.getColumnType() == REGULAR, "column type must be REGULAR: %s", column); @@ -326,10 +317,6 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq } } - log.warn("Trino types: %s", trinoTypes.build()); - log.warn("Internal fields: %s", internalFields.build()); - log.warn("Row index: %s", rowIndexColumns.build()); - ConnectorPageSource parquetPageSource = new ParquetPageSource( parquetReader, trinoTypes.build(), @@ -425,8 +412,6 @@ public static TupleDomain getParquetTupleDomain( return TupleDomain.none(); } - descriptorsByPath.forEach((key, value) -> log.warn(">>> descriptorsByPath KEY: %s, VALUE: %s", key, value.toString())); - ImmutableMap.Builder predicate = ImmutableMap.builder(); for (Entry entry : effectivePredicate.getDomains().get().entrySet()) { HiveColumnHandle columnHandle = entry.getKey(); @@ -438,14 +423,12 @@ public static TupleDomain getParquetTupleDomain( RichColumnDescriptor descriptor; if (useColumnNames) { descriptor = descriptorsByPath.get(ImmutableList.of(columnHandle.getName())); - log.warn(">>> Descriptor: %s, Column Handle: %s", descriptor, columnHandle); } else { org.apache.parquet.schema.Type parquetField = getParquetType(columnHandle, fileSchema, false); if (parquetField == null || !parquetField.isPrimitive()) { // Parquet file has fewer column than partition // Or the field is a complex type - log.warn(String.valueOf(">>> Parquet file has fewer column than partition: " + parquetField == null)); continue; } descriptor = descriptorsByPath.get(ImmutableList.of(parquetField.getName())); @@ -454,7 +437,6 @@ public static TupleDomain getParquetTupleDomain( predicate.put(descriptor, entry.getValue()); } } - log.warn(">>> Predicates: " + predicate.build()); return TupleDomain.withColumnDomains(predicate.build()); } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java index 19d5f4b00ca..57cf5b2eff2 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java @@ -15,7 +15,6 @@ package io.trino.plugin.hudi; import com.google.common.collect.ImmutableList; -import io.airlift.log.Logger; import io.trino.parquet.Field; import io.trino.parquet.ParquetCorruptionException; import io.trino.parquet.ParquetDataSource; @@ -23,7 +22,6 @@ import io.trino.parquet.ParquetReaderOptions; import io.trino.parquet.RichColumnDescriptor; import io.trino.parquet.predicate.Predicate; -import io.trino.parquet.reader.MetadataReader; import io.trino.parquet.reader.ParquetReader; import io.trino.plugin.hive.FileFormatDataSourceStats; import io.trino.plugin.hive.HdfsEnvironment; @@ -78,6 +76,7 @@ import static io.trino.parquet.ParquetTypeUtils.lookupColumnByName; import static io.trino.parquet.predicate.PredicateUtils.buildPredicate; import static io.trino.parquet.predicate.PredicateUtils.predicateMatches; +import static io.trino.parquet.reader.MetadataReader.readFooter; import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.REGULAR; import static io.trino.plugin.hive.HivePageSourceProvider.projectBaseColumns; import static io.trino.plugin.hive.HivePageSourceProvider.projectSufficientColumns; @@ -100,8 +99,6 @@ public class HudiPageSourceProvider implements ConnectorPageSourceProvider { - private static final Logger log = Logger.get(HudiPageSourceProvider.class); - private final HdfsEnvironment hdfsEnvironment; private final FileFormatDataSourceStats fileFormatDataSourceStats; private final ParquetReaderOptions parquetReaderOptions; @@ -182,19 +179,15 @@ private static ConnectorPageSource createParquetPageSource( ConnectorIdentity identity, List partitionKeys) { - log.debug(">>> Creating Parquet Page Source with partition keys: %s, columns: %s, predicate: %s", partitionKeys, columns, effectivePredicate); - ParquetDataSource dataSource = null; - // TODO: Reuse some elements of ParquetPageSourceFactory and extract the try block to a new HudiParquetReader class. try { FileSystem fileSystem = hdfsEnvironment.getFileSystem(identity, path, configuration); FSDataInputStream inputStream = hdfsEnvironment.doAs(identity, () -> fileSystem.open(path)); dataSource = new HdfsParquetDataSource(new ParquetDataSourceId(path.toString()), estimatedFileSize, inputStream, stats, options); - ParquetDataSource theDataSource = dataSource; // extra variable required for lambda below - ParquetMetadata parquetMetadata = hdfsEnvironment.doAs(identity, () -> MetadataReader.readFooter(theDataSource)); + ParquetDataSource parquetDataSource = dataSource; + ParquetMetadata parquetMetadata = hdfsEnvironment.doAs(identity, () -> readFooter(parquetDataSource)); FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); MessageType fileSchema = fileMetaData.getSchema(); - log.debug(">>> File Schema: " + fileSchema.toString()); Optional message = projectSufficientColumns(columns) .map(projection -> projection.get().stream() @@ -209,15 +202,12 @@ private static ConnectorPageSource createParquetPageSource( .reduce(MessageType::union); MessageType requestedSchema = message.orElse(new MessageType(fileSchema.getName(), ImmutableList.of())); - log.debug(">>> Requested Schema: " + requestedSchema); MessageColumnIO messageColumn = getColumnIO(fileSchema, requestedSchema); Map, RichColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, requestedSchema); - log.debug(">>> Ignore stats: " + options.isIgnoreStatistics()); TupleDomain parquetTupleDomain = options.isIgnoreStatistics() ? TupleDomain.all() : getParquetTupleDomain(descriptorsByPath, effectivePredicate, fileSchema, useParquetColumnNames); - log.debug(">>> ParquetTupleDomain: %s", parquetTupleDomain.toString()); Predicate parquetPredicate = buildPredicate(requestedSchema, parquetTupleDomain, descriptorsByPath, timeZone); @@ -237,7 +227,6 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq nextStart += block.getRowCount(); } - log.debug("Message Column: %s, Predicate: %s, ColumnIndexes: %s", messageColumn, parquetPredicate.toString(), columnIndexes.build()); ParquetReader parquetReader = new ParquetReader( Optional.ofNullable(fileMetaData.getCreatedBy()), messageColumn, @@ -250,13 +239,11 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq parquetPredicate, columnIndexes.build()); Optional readerProjections = projectBaseColumns(columns); - log.debug("Reade columns: %s", readerProjections.orElse(null)); List baseColumns = readerProjections.map(projection -> projection.get().stream() .map(HiveColumnHandle.class::cast) .collect(toUnmodifiableList())) .orElse(columns); - log.debug("Base columns: %s", baseColumns); for (HiveColumnHandle column : baseColumns) { checkArgument(column == PARQUET_ROW_INDEX_COLUMN || column.getColumnType() == REGULAR, "column type must be REGULAR: %s", column); @@ -280,10 +267,6 @@ && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parq } } - log.debug("Trino types: %s", trinoTypes.build()); - log.debug("Internal fields: %s", internalFields.build()); - log.debug("Row index: %s", rowIndexColumns.build()); - return new ParquetPageSource( parquetReader, trinoTypes.build(), @@ -318,7 +301,6 @@ private Map convertPartitionValues( List allColumns, List partitionKeys) { - log.debug(">>> Converting partition values for columns: %s, partition key: %s", allColumns, partitionKeys); return allColumns.stream() .filter(HiveColumnHandle::isPartitionKey) .collect(toMap( diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java index 3420b39a110..47f4298de60 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitManager.java @@ -14,7 +14,6 @@ package io.trino.plugin.hudi; -import io.airlift.log.Logger; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitSource; import io.trino.plugin.hive.HdfsEnvironment; import io.trino.plugin.hive.authentication.HiveIdentity; @@ -49,7 +48,6 @@ public class HudiSplitManager implements ConnectorSplitManager { public static final Pattern HOODIE_CONSUME_MODE_PATTERN_STRING = Pattern.compile("hoodie\\.(.*)\\.consume\\.mode"); - private static final Logger log = Logger.get(HudiSplitManager.class); private final HudiTransactionManager transactionManager; private final HdfsEnvironment hdfsEnvironment; @@ -70,7 +68,6 @@ public ConnectorSplitSource getSplits( DynamicFilter dynamicFilter, Constraint constraint) { - log.debug(" >>>> Getting Splits <<<< "); HiveIdentity identity = new HiveIdentity(session); HudiTableHandle hudiTable = (HudiTableHandle) tableHandle; SchemaTableName tableName = hudiTable.getSchemaTableName(); @@ -82,17 +79,8 @@ public ConnectorSplitSource getSplits( Configuration conf = hdfsEnvironment.getConfiguration(context, new Path(tablePath)); // TODO: Do we need below? Map valByRegex = conf.getValByRegex(HOODIE_CONSUME_MODE_PATTERN_STRING.pattern()); - log.debug("Hoodie consume mode: " + valByRegex); HoodieTableMetaClient metaClient = hudiTable.getMetaClient().orElseGet(() -> getMetaClient(conf, hudiTable.getBasePath())); - log.debug("HudiSplitManager ref: " + this.toString()); - log.debug("Table ref: " + table.toString()); - log.debug("HoodieTableMetaClient ref: " + metaClient.toString()); - log.debug("HoodieTableMetaClient base path: " + metaClient.getBasePath()); - hudiTable.getPartitions().ifPresent(p -> p.forEach(p1 -> log.debug("Partitions from TableHandle: " + p1))); - TupleDomain effectivePredicate = constraint.getSummary(); - log.debug("effectivePredicate: " + effectivePredicate); - HudiSplitSource splitSource = new HudiSplitSource( identity, metastore, @@ -101,6 +89,7 @@ public ConnectorSplitSource getSplits( isHudiMetadataEnabled(session), shouldSkipMetaStoreForPartition(session), dynamicFilter); + return new ClassLoaderSafeConnectorSplitSource(splitSource, Thread.currentThread().getContextClassLoader()); } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java index 0243de021e1..e240bd80ede 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiSplitSource.java @@ -16,7 +16,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; -import io.airlift.log.Logger; import io.trino.plugin.hive.HivePartition; import io.trino.plugin.hive.HivePartitionKey; import io.trino.plugin.hive.authentication.HiveIdentity; @@ -38,7 +37,6 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.engine.HoodieLocalEngineContext; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.view.FileSystemViewManager; @@ -71,12 +69,11 @@ import static java.util.Objects.nonNull; import static java.util.Objects.requireNonNull; import static java.util.concurrent.CompletableFuture.completedFuture; +import static org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath; public class HudiSplitSource implements ConnectorSplitSource { - private static final Logger log = Logger.get(HudiSplitSource.class); - private final HiveIdentity identity; private final HiveMetastore metastore; private final SchemaTableName tableName; @@ -87,7 +84,7 @@ public class HudiSplitSource private final HoodieTableMetaClient metaClient; private final Map> partitionMap; private final boolean metadataEnabled; - private final boolean shouldSkipMetaStoreForPartition; + private final boolean shouldSkipMetastoreForPartition; private final Map baseFileToPartitionMap; private final ArrayDeque baseFiles = new ArrayDeque<>(); private final DynamicFilter dynamicFilter; @@ -104,7 +101,7 @@ public HudiSplitSource( HudiTableHandle tableHandle, Configuration conf, boolean metadataEnabled, - boolean shouldSkipMetaStoreForPartition, + boolean shouldSkipMetastoreForPartition, DynamicFilter dynamicFilter) { this.identity = identity; @@ -116,7 +113,7 @@ public HudiSplitSource( this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); this.partitionMap = new HashMap<>(); this.metadataEnabled = metadataEnabled; - this.shouldSkipMetaStoreForPartition = shouldSkipMetaStoreForPartition; + this.shouldSkipMetastoreForPartition = shouldSkipMetastoreForPartition; this.metaClient = tableHandle.getMetaClient().orElseGet(() -> getMetaClient(conf, tableHandle.getBasePath())); this.fileSystem = metaClient.getFs(); this.baseFileToPartitionMap = new HashMap<>(); @@ -126,8 +123,6 @@ public HudiSplitSource( @Override public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) { - log.debug("Dynamic filter: " + dynamicFilter.getColumnsCovered()); - log.debug("Getting next batch with partitionKeys: " + partitionMap.keySet()); try { List connectorSplits = getSplitsForSnapshotMode(maxSize); return completedFuture(new ConnectorSplitBatch(connectorSplits, isFinished())); @@ -148,7 +143,7 @@ public void close() @Override public boolean isFinished() { - if (shouldSkipMetaStoreForPartition) { + if (shouldSkipMetastoreForPartition) { return !tableHandlePartitions.hasNext() && baseFiles.isEmpty(); } return !metastorePartitions.hasNext() && baseFiles.isEmpty(); @@ -158,7 +153,6 @@ private List getSplitsForSnapshotMode(int maxSize) throws IOException { if (isNull(this.fileSystemView)) { - HoodieTimer timer = new HoodieTimer().startTimer(); // First time calling this // Load the timeline and file status only once HoodieEngineContext engineContext = new HoodieLocalEngineContext(conf); @@ -166,18 +160,13 @@ private List getSplitsForSnapshotMode(int maxSize) .enable(metadataEnabled) .build(); // Scan the file system to load the instants from timeline - log.debug("Loading file system view for " + metaClient.getBasePath()); this.fileSystemView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, metaClient, metadataConfig); - log.debug(String.format("Finish in %d ms to load table view", timer.endTimer())); - - timer = new HoodieTimer().startTimer(); partitionColumns = table.getPartitionColumns(); if (!partitionColumns.isEmpty()) { - if (shouldSkipMetaStoreForPartition) { + if (shouldSkipMetastoreForPartition) { List partitions = tableHandle.getPartitions().orElseGet(ImmutableList::of).stream() .map(HivePartition::getPartitionId).collect(Collectors.toList()); - log.debug(">>> Partitions tableHandle: %s", partitions); tableHandlePartitions = partitions.iterator(); } else { @@ -198,51 +187,38 @@ private List getSplitsForSnapshotMode(int maxSize) } } else { - // no partitions, so data dir is same as table path - if (shouldSkipMetaStoreForPartition) { + if (shouldSkipMetastoreForPartition) { tableHandlePartitions = Collections.singletonList("").iterator(); } else { metastorePartitions = Collections.singletonList(Collections.singletonList("")).iterator(); } } - - log.debug(String.format("Finish in %d ms to fetch partition names", timer.endTimer())); } List batchHudiSplits = new ArrayList<>(); int remaining = maxSize; - log.debug("Target number of splits: " + maxSize); - - // Only process one batch now while (remaining > 0 && !isFinished()) { if (baseFiles.isEmpty()) { - HoodieTimer timer1 = new HoodieTimer().startTimer(); - Map> batchKeyMap; - if (shouldSkipMetaStoreForPartition) { + if (shouldSkipMetastoreForPartition) { List batchTableHandlePartitions = new ArrayList<>(); Iterators.limit(tableHandlePartitions, partitionBatchNum).forEachRemaining(batchTableHandlePartitions::add); batchKeyMap = batchTableHandlePartitions.stream().parallel() .map(p -> Pair.of(p, buildPartitionKeys(partitionColumns, buildPartitionValues(p)))) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); - log.debug(">>> shouldSkipMetaStoreForPartition batchKeyMap: %s", batchKeyMap); } else { List> batchMetastorePartitions = new ArrayList<>(); Iterators.limit(metastorePartitions, partitionBatchNum).forEachRemaining(batchMetastorePartitions::add); batchKeyMap = batchMetastorePartitions.stream().parallel() - .map(partitionNames -> getPartitionPathToKey( + .map(partitionNames -> getPartitionPathToKeyUsingMetastore( identity, metastore, table, table.getStorage().getLocation(), partitionNames)) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); - log.debug(">>> batchKeyMap: %s", batchKeyMap); } - partitionMap.putAll(batchKeyMap); - - log.debug(String.format("Finish in %d ms to get partition keys: %s", timer1.endTimer(), batchKeyMap)); - timer1 = new HoodieTimer().startTimer(); + partitionMap.putAll(batchKeyMap); List> baseFilesToAdd = batchKeyMap.keySet().stream().parallel() .flatMap(relativePartitionPath -> fileSystemView.getLatestBaseFiles(relativePartitionPath) .map(baseFile -> new ImmutablePair<>(baseFile, relativePartitionPath)) @@ -252,7 +228,6 @@ private List getSplitsForSnapshotMode(int maxSize) baseFilesToAdd.forEach(e -> baseFileToPartitionMap.put(e.getKey(), e.getValue())); // TODO: skip partitions that are filtered out based on the predicate baseFiles.addAll(baseFilesToAdd.stream().map(Pair::getKey).collect(Collectors.toList())); - log.debug(String.format("Finish in %d ms to get base files", timer1.endTimer())); } HoodieTimer timer = new HoodieTimer().startTimer(); @@ -298,38 +273,31 @@ private List getSplitsForSnapshotMode(int maxSize) .collect(Collectors.toList()); batchHudiSplits.addAll(hudiSplitsToAdd); remaining -= hudiSplitsToAdd.size(); - log.debug(String.format("Finish in %d ms to get batch splits", timer.endTimer())); if (remaining < hudiSplitsToAdd.size()) { break; } } - log.info("Number of Hudi splits generated in the batch: " + batchHudiSplits.size()); - return batchHudiSplits; } - private Pair> getPartitionPathToKey( + private Pair> getPartitionPathToKeyUsingMetastore( HiveIdentity identity, HiveMetastore metastore, Table table, String tablePath, List partitionValues) { - log.debug(">>> Inside getPartitionPathToKey: %s", partitionValues); - Optional partition1; + Optional partition; String relativePartitionPath; List partitionKeys; - partition1 = metastore.getPartition(identity, table, partitionValues); - partition1.ifPresent(p -> log.debug("Partition from metastore: %s", p)); - String dataDir1 = partition1.isPresent() - ? partition1.get().getStorage().getLocation() + partition = metastore.getPartition(identity, table, partitionValues); + String dataDir1 = partition.isPresent() + ? partition.get().getStorage().getLocation() : tablePath; - log.debug(">>> basePath: %s, dataDir1: %s", tablePath, dataDir1); - relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(tablePath), new Path(dataDir1)); - log.debug(">>> relativePartitionPath: %s", relativePartitionPath); - partitionKeys = getPartitionKeys(table, partition1); - log.debug(">>> Partition keys: %s", partitionKeys); + relativePartitionPath = getRelativePartitionPath(new Path(tablePath), new Path(dataDir1)); + partitionKeys = getPartitionKeys(table, partition); + return new ImmutablePair<>(relativePartitionPath, partitionKeys); } } From 62769155c6dc5a519d5416ce9d23ffb9e6cf6c16 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Tue, 28 Dec 2021 12:20:04 +0530 Subject: [PATCH 25/26] Fix dependency issue after rebase on master --- plugin/trino-hudi/pom.xml | 34 ++++++++++++++++++- .../etc/catalog/hive.properties | 3 +- .../etc/catalog/hudi.properties | 2 +- .../trino-server-dev/etc/config.properties | 4 +-- .../etc/event-listener.properties | 2 +- testing/trino-server-dev/etc/log.properties | 3 -- 6 files changed, 38 insertions(+), 10 deletions(-) diff --git a/plugin/trino-hudi/pom.xml b/plugin/trino-hudi/pom.xml index c47eb8f4a97..c6be6ec7d14 100644 --- a/plugin/trino-hudi/pom.xml +++ b/plugin/trino-hudi/pom.xml @@ -7,7 +7,7 @@ trino-root io.trino - 366-SNAPSHOT + 368-SNAPSHOT ../../pom.xml @@ -126,6 +126,22 @@ com.esotericsoftware kryo-shaded + + org.apache.hadoop + hadoop-client + + + org.apache.httpcomponents + httpcore + + + org.apache.hive + hive-exec + + + org.apache.hive + hive-jdbc + @@ -165,6 +181,22 @@ com.esotericsoftware kryo-shaded + + org.apache.hadoop + hadoop-client + + + org.apache.httpcomponents + httpcore + + + org.apache.hive + hive-exec + + + org.apache.hive + hive-jdbc + diff --git a/testing/trino-server-dev/etc/catalog/hive.properties b/testing/trino-server-dev/etc/catalog/hive.properties index 3e12a2f6c25..3f2d401d4ce 100644 --- a/testing/trino-server-dev/etc/catalog/hive.properties +++ b/testing/trino-server-dev/etc/catalog/hive.properties @@ -10,9 +10,8 @@ connector.name=hive # Configuration appropriate for Hive as started by product test environment, e.g. # trino-product-tests-launcher/bin/run-launcher env up --environment singlenode --without-trino # On Mac, this additionally requires that you add " hadoop-master" to /etc/hosts -hive.metastore.uri=thrift://hivemetastore:9083 +hive.metastore.uri=thrift://localhost:9083 hive.hdfs.socks-proxy=localhost:1180 -hive.config.resources=/Users/ethan/Work/data/conf/core-site.xml,/Users/ethan/Work/data/conf/hdfs-site.xml # Fail-fast in development hive.metastore.thrift.client.max-retry-time=1s diff --git a/testing/trino-server-dev/etc/catalog/hudi.properties b/testing/trino-server-dev/etc/catalog/hudi.properties index 33ba981c9bc..23bc4e0573f 100644 --- a/testing/trino-server-dev/etc/catalog/hudi.properties +++ b/testing/trino-server-dev/etc/catalog/hudi.properties @@ -14,4 +14,4 @@ connector.name=hudi hive.metastore.uri=thrift://localhost:9083 -#hive.config.resources=/Users/sagars/core-site.xml,/Users/sagars/hdfs-site.xml +#hive.config.resources=/path/to/core-site.xml,/path/to/hdfs-site.xml diff --git a/testing/trino-server-dev/etc/config.properties b/testing/trino-server-dev/etc/config.properties index 36eaa0a681b..fd2beb31dba 100644 --- a/testing/trino-server-dev/etc/config.properties +++ b/testing/trino-server-dev/etc/config.properties @@ -10,9 +10,9 @@ node.id=ffffffff-ffff-ffff-ffff-ffffffffffff node.environment=test node.internal-address=localhost experimental.concurrent-startup=true -http-server.http.port=9090 +http-server.http.port=8080 -discovery.uri=http://localhost:9090 +discovery.uri=http://localhost:8080 exchange.http-client.max-connections=1000 exchange.http-client.max-connections-per-server=1000 diff --git a/testing/trino-server-dev/etc/event-listener.properties b/testing/trino-server-dev/etc/event-listener.properties index 21e08c10eef..74fdeb5c470 100644 --- a/testing/trino-server-dev/etc/event-listener.properties +++ b/testing/trino-server-dev/etc/event-listener.properties @@ -1,3 +1,3 @@ event-listener.name=http http-event-listener.log-created=true -http-event-listener.connect-ingest-uri=http://localhost:9091 +http-event-listener.connect-ingest-uri=http://localhost:8090 diff --git a/testing/trino-server-dev/etc/log.properties b/testing/trino-server-dev/etc/log.properties index 18b148d8f5f..b615d661c74 100644 --- a/testing/trino-server-dev/etc/log.properties +++ b/testing/trino-server-dev/etc/log.properties @@ -12,6 +12,3 @@ io.trino.server.PluginManager=DEBUG # Maven plugin loading code com.ning.http.client=WARN -io.trino.plugin.hudi=DEBUG -io.trino.plugin.hive=DEBUG -org.apache.hudi=DEBUG From a74d809598b7bf62a45b96c5b2094c7a05d59312 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Wed, 22 Dec 2021 14:31:48 -0800 Subject: [PATCH 26/26] Use hudi-trino-bundle --- plugin/trino-hudi/pom.xml | 70 ++++++++------------------------------- 1 file changed, 13 insertions(+), 57 deletions(-) diff --git a/plugin/trino-hudi/pom.xml b/plugin/trino-hudi/pom.xml index c6be6ec7d14..ce7faa83a19 100644 --- a/plugin/trino-hudi/pom.xml +++ b/plugin/trino-hudi/pom.xml @@ -17,7 +17,7 @@ ${project.parent.basedir} - 0.10.0 + 0.11.0-SNAPSHOT @@ -89,64 +89,11 @@ joda-time joda-time + + org.apache.hudi - hudi-common - ${dep.hudi.version} - - - org.apache.hbase - hbase-server - - - org.apache.orc - orc-core - - - com.fasterxml.jackson.core - jackson-annotations - - - com.fasterxml.jackson.core - jackson-databind - - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - fluent-hc - - - org.rocksdb - rocksdbjni - - - com.esotericsoftware - kryo-shaded - - - org.apache.hadoop - hadoop-client - - - org.apache.httpcomponents - httpcore - - - org.apache.hive - hive-exec - - - org.apache.hive - hive-jdbc - - - - - org.apache.hudi - hudi-hadoop-mr + hudi-trino-bundle ${dep.hudi.version} @@ -199,10 +146,19 @@ + org.weakref jmxutils + + + + io.airlift + log-manager + runtime + + io.trino