From 3f8d31ac35b892058f1ea818c966f6f056d8225e Mon Sep 17 00:00:00 2001 From: Prabhu Joseph Date: Thu, 9 Nov 2023 08:12:43 +0530 Subject: [PATCH 1/4] [HUDI-6993] Support Flink 1.18 (#9949) * Address build failures in older Flink Versions * Remove unnecessary dependency on flink-connector-hive * Fix Flink 1.18 Validate-bundles --------- Signed-off-by: Prabhu Joseph Co-authored-by: Prabhu Joseph Co-authored-by: root --- .github/workflows/bot.yml | 12 +- README.md | 7 +- azure-pipelines-20230430.yml | 7 +- hudi-flink-datasource/hudi-flink/pom.xml | 1 + .../hudi/table/catalog/HoodieHiveCatalog.java | 36 +- .../hudi/adapter/HiveCatalogConstants.java | 51 ++ .../hudi/adapter/HiveCatalogConstants.java | 52 ++ .../hudi/adapter/HiveCatalogConstants.java | 52 ++ .../hudi/adapter/HiveCatalogConstants.java | 52 ++ .../hudi/adapter/HiveCatalogConstants.java | 52 ++ .../hudi-flink1.18.x/pom.xml | 168 +++++ .../AbstractStreamOperatorAdapter.java | 27 + .../AbstractStreamOperatorFactoryAdapter.java | 33 + .../DataStreamScanProviderAdapter.java | 34 + .../DataStreamSinkProviderAdapter.java | 37 ++ .../hudi/adapter/HiveCatalogConstants.java | 49 ++ .../hudi/adapter/MailboxExecutorAdapter.java | 37 ++ .../hudi/adapter/MaskingOutputAdapter.java | 67 ++ .../adapter/OperatorCoordinatorAdapter.java | 50 ++ .../hudi/adapter/RateLimiterAdapter.java | 40 ++ .../adapter/SortCodeGeneratorAdapter.java | 33 + .../SupportsRowLevelDeleteAdapter.java | 42 ++ .../SupportsRowLevelUpdateAdapter.java | 45 ++ .../java/org/apache/hudi/adapter/Utils.java | 91 +++ .../format/cow/ParquetSplitReaderUtil.java | 579 ++++++++++++++++++ .../format/cow/vector/HeapArrayVector.java | 70 +++ .../cow/vector/HeapMapColumnVector.java | 79 +++ .../cow/vector/HeapRowColumnVector.java | 54 ++ .../cow/vector/ParquetDecimalVector.java | 54 ++ .../vector/reader/AbstractColumnReader.java | 325 ++++++++++ .../cow/vector/reader/ArrayColumnReader.java | 473 ++++++++++++++ .../reader/BaseVectorizedColumnReader.java | 313 ++++++++++ .../cow/vector/reader/EmptyColumnReader.java | 42 ++ .../reader/FixedLenBytesColumnReader.java | 84 +++ .../reader/Int64TimestampColumnReader.java | 119 ++++ .../cow/vector/reader/MapColumnReader.java | 76 +++ .../reader/ParquetColumnarRowSplitReader.java | 390 ++++++++++++ .../reader/ParquetDataColumnReader.java | 199 ++++++ .../ParquetDataColumnReaderFactory.java | 304 +++++++++ .../cow/vector/reader/RowColumnReader.java | 63 ++ .../cow/vector/reader/RunLengthDecoder.java | 304 +++++++++ .../apache/hudi/adapter/OutputAdapter.java | 32 + .../StateInitializationContextAdapter.java | 31 + .../StreamingRuntimeContextAdapter.java | 43 ++ .../hudi/adapter/TestStreamConfigs.java | 35 ++ .../apache/hudi/adapter/TestTableEnvs.java | 52 ++ hudi-flink-datasource/pom.xml | 1 + ...2.sh => build_flink1180hive313spark332.sh} | 6 +- ...0.sh => build_flink1180hive313spark340.sh} | 6 +- packaging/bundle-validation/ci_run.sh | 2 + pom.xml | 37 +- scripts/release/deploy_staging_jars.sh | 1 + scripts/release/validate_staged_bundles.sh | 2 +- 53 files changed, 4812 insertions(+), 39 deletions(-) create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java create mode 100644 hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java create mode 100644 hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java create mode 100644 hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java create mode 100644 hudi-flink-datasource/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/pom.xml create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/DataStreamScanProviderAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/DataStreamSinkProviderAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/OperatorCoordinatorAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/SortCodeGeneratorAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/SupportsRowLevelDeleteAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/SupportsRowLevelUpdateAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/Utils.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/ParquetDecimalVector.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/AbstractColumnReader.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/BaseVectorizedColumnReader.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/EmptyColumnReader.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/FixedLenBytesColumnReader.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReader.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReaderFactory.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RowColumnReader.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RunLengthDecoder.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/TestStreamConfigs.java create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java rename packaging/bundle-validation/base/{build_flink1170hive313spark332.sh => build_flink1180hive313spark332.sh} (81%) rename packaging/bundle-validation/base/{build_flink1170hive313spark340.sh => build_flink1180hive313spark340.sh} (81%) diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml index 35de0b9087ed..fd3cc67976a1 100644 --- a/.github/workflows/bot.yml +++ b/.github/workflows/bot.yml @@ -119,7 +119,7 @@ jobs: include: - scalaProfile: "scala-2.12" sparkProfile: "spark3.2" - flinkProfile: "flink1.17" + flinkProfile: "flink1.18" steps: - uses: actions/checkout@v3 @@ -210,6 +210,7 @@ jobs: - flinkProfile: "flink1.15" - flinkProfile: "flink1.16" - flinkProfile: "flink1.17" + - flinkProfile: "flink1.18" steps: - uses: actions/checkout@v3 - name: Set up JDK 8 @@ -234,7 +235,7 @@ jobs: env: SCALA_PROFILE: 'scala-2.12' FLINK_PROFILE: ${{ matrix.flinkProfile }} - if: ${{ endsWith(env.FLINK_PROFILE, '1.17') }} + if: ${{ endsWith(env.FLINK_PROFILE, '1.18') }} run: | mvn clean install -Pintegration-tests -D"$SCALA_PROFILE" -D"$FLINK_PROFILE" -pl hudi-flink-datasource/hudi-flink -am -Davro.version=1.10.0 -DskipTests=true $MVN_ARGS mvn verify -Pintegration-tests -D"$SCALA_PROFILE" -D"$FLINK_PROFILE" -pl hudi-flink-datasource/hudi-flink $MVN_ARGS @@ -244,7 +245,7 @@ jobs: strategy: matrix: include: - - flinkProfile: 'flink1.17' + - flinkProfile: 'flink1.18' sparkProfile: 'spark3.4' sparkRuntime: 'spark3.4.0' @@ -272,9 +273,12 @@ jobs: strategy: matrix: include: - - flinkProfile: 'flink1.17' + - flinkProfile: 'flink1.18' sparkProfile: 'spark3.4' sparkRuntime: 'spark3.4.0' + - flinkProfile: 'flink1.18' + sparkProfile: 'spark3.3' + sparkRuntime: 'spark3.3.2' - flinkProfile: 'flink1.17' sparkProfile: 'spark3.3' sparkRuntime: 'spark3.3.2' diff --git a/README.md b/README.md index ff2b95ec5473..20016f689ad3 100644 --- a/README.md +++ b/README.md @@ -118,14 +118,15 @@ Starting from versions 0.11, Hudi no longer requires `spark-avro` to be specifie ### Build with different Flink versions -The default Flink version supported is 1.17. The default Flink 1.17.x version, corresponding to `flink1.17` profile is 1.17.0. +The default Flink version supported is 1.18. The default Flink 1.18.x version, corresponding to `flink1.18` profile is 1.18.0. Flink is Scala-free since 1.15.x, there is no need to specify the Scala version for Flink 1.15.x and above versions. Refer to the table below for building with different Flink and Scala versions. | Maven build options | Expected Flink bundle jar name | Notes | |:---------------------------|:-------------------------------|:---------------------------------| -| (empty) | hudi-flink1.17-bundle | For Flink 1.17 (default options) | -| `-Dflink1.17` | hudi-flink1.17-bundle | For Flink 1.17 (same as default) | +| (empty) | hudi-flink1.18-bundle | For Flink 1.18 (default options) | +| `-Dflink1.18` | hudi-flink1.18-bundle | For Flink 1.18 (same as default) | +| `-Dflink1.17` | hudi-flink1.17-bundle | For Flink 1.17 | | `-Dflink1.16` | hudi-flink1.16-bundle | For Flink 1.16 | | `-Dflink1.15` | hudi-flink1.15-bundle | For Flink 1.15 | | `-Dflink1.14` | hudi-flink1.14-bundle | For Flink 1.14 and Scala 2.12 | diff --git a/azure-pipelines-20230430.yml b/azure-pipelines-20230430.yml index ee5c016693a5..85d185fbc2c5 100644 --- a/azure-pipelines-20230430.yml +++ b/azure-pipelines-20230430.yml @@ -14,7 +14,7 @@ # limitations under the License. # NOTE: -# This config file defines how Azure CI runs tests with Spark 2.4 and Flink 1.17 profiles. +# This config file defines how Azure CI runs tests with Spark 2.4 and Flink 1.18 profiles. # PRs will need to keep in sync with master's version to trigger the CI runs. trigger: @@ -37,6 +37,7 @@ parameters: - 'hudi-flink-datasource/hudi-flink1.15.x' - 'hudi-flink-datasource/hudi-flink1.16.x' - 'hudi-flink-datasource/hudi-flink1.17.x' + - 'hudi-flink-datasource/hudi-flink1.18.x' - name: job2Modules type: object default: @@ -69,6 +70,7 @@ parameters: - '!hudi-flink-datasource/hudi-flink1.15.x' - '!hudi-flink-datasource/hudi-flink1.16.x' - '!hudi-flink-datasource/hudi-flink1.17.x' + - '!hudi-flink-datasource/hudi-flink1.18.x' - '!hudi-spark-datasource' - '!hudi-spark-datasource/hudi-spark' - '!hudi-spark-datasource/hudi-spark3.2.x' @@ -92,9 +94,10 @@ parameters: - '!hudi-flink-datasource/hudi-flink1.15.x' - '!hudi-flink-datasource/hudi-flink1.16.x' - '!hudi-flink-datasource/hudi-flink1.17.x' + - '!hudi-flink-datasource/hudi-flink1.18.x' variables: - BUILD_PROFILES: '-Dscala-2.12 -Dspark3.2 -Dflink1.17' + BUILD_PROFILES: '-Dscala-2.12 -Dspark3.2 -Dflink1.18' PLUGIN_OPTS: '-Dcheckstyle.skip=true -Drat.skip=true -Djacoco.skip=true -ntp -B -V -Pwarn-log -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.shade=warn -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.dependency=warn' MVN_OPTS_INSTALL: '-Phudi-platform-service -DskipTests $(BUILD_PROFILES) $(PLUGIN_OPTS) -Dmaven.wagon.httpconnectionManager.ttlSeconds=25 -Dmaven.wagon.http.retryHandler.count=5' MVN_OPTS_TEST: '-fae -Pwarn-log $(BUILD_PROFILES) $(PLUGIN_OPTS)' diff --git a/hudi-flink-datasource/hudi-flink/pom.xml b/hudi-flink-datasource/hudi-flink/pom.xml index 3a479358478d..c1247c16c08f 100644 --- a/hudi-flink-datasource/hudi-flink/pom.xml +++ b/hudi-flink-datasource/hudi-flink/pom.xml @@ -181,6 +181,7 @@ org.apache.flink ${flink.connector.kafka.artifactId} + ${flink.connector.kafka.version} compile diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java index 23a7a1fcca71..5ea7a585a0d2 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java @@ -18,6 +18,7 @@ package org.apache.hudi.table.catalog; +import org.apache.hudi.adapter.HiveCatalogConstants.AlterHiveDatabaseOp; import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.common.fs.FSUtils; @@ -47,9 +48,6 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; -import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabase; -import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner; -import org.apache.flink.sql.parser.hive.ddl.SqlCreateHiveDatabase; import org.apache.flink.table.catalog.AbstractCatalog; import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.CatalogDatabase; @@ -107,17 +105,20 @@ import java.util.List; import java.util.Map; -import static org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabase.ALTER_DATABASE_OP; -import static org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner.DATABASE_OWNER_NAME; -import static org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner.DATABASE_OWNER_TYPE; -import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; +import static org.apache.hudi.adapter.HiveCatalogConstants.ALTER_DATABASE_OP; +import static org.apache.hudi.adapter.HiveCatalogConstants.DATABASE_LOCATION_URI; +import static org.apache.hudi.adapter.HiveCatalogConstants.DATABASE_OWNER_NAME; +import static org.apache.hudi.adapter.HiveCatalogConstants.DATABASE_OWNER_TYPE; +import static org.apache.hudi.adapter.HiveCatalogConstants.ROLE_OWNER; +import static org.apache.hudi.adapter.HiveCatalogConstants.USER_OWNER; import static org.apache.hudi.configuration.FlinkOptions.PATH; import static org.apache.hudi.table.catalog.TableOptionProperties.COMMENT; import static org.apache.hudi.table.catalog.TableOptionProperties.PK_CONSTRAINT_NAME; import static org.apache.hudi.table.catalog.TableOptionProperties.SPARK_SOURCE_PROVIDER; +import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; /** * A catalog implementation for Hoodie based on MetaStore. @@ -219,7 +220,7 @@ public CatalogDatabase getDatabase(String databaseName) Map properties = new HashMap<>(hiveDatabase.getParameters()); - properties.put(SqlCreateHiveDatabase.DATABASE_LOCATION_URI, hiveDatabase.getLocationUri()); + properties.put(DATABASE_LOCATION_URI, hiveDatabase.getLocationUri()); return new CatalogDatabaseImpl(properties, hiveDatabase.getDescription()); } @@ -248,7 +249,7 @@ public void createDatabase( Map properties = database.getProperties(); - String dbLocationUri = properties.remove(SqlCreateHiveDatabase.DATABASE_LOCATION_URI); + String dbLocationUri = properties.remove(DATABASE_LOCATION_URI); if (dbLocationUri == null && this.catalogPath != null) { // infer default location uri dbLocationUri = new Path(this.catalogPath, databaseName).toString(); @@ -318,11 +319,10 @@ private static Database alterDatabase(Database hiveDB, CatalogDatabase newDataba String opStr = newParams.remove(ALTER_DATABASE_OP); if (opStr == null) { // by default is to alter db properties - opStr = SqlAlterHiveDatabase.AlterHiveDatabaseOp.CHANGE_PROPS.name(); + opStr = AlterHiveDatabaseOp.CHANGE_PROPS.name(); } - String newLocation = newParams.remove(SqlCreateHiveDatabase.DATABASE_LOCATION_URI); - SqlAlterHiveDatabase.AlterHiveDatabaseOp op = - SqlAlterHiveDatabase.AlterHiveDatabaseOp.valueOf(opStr); + String newLocation = newParams.remove(DATABASE_LOCATION_URI); + AlterHiveDatabaseOp op = AlterHiveDatabaseOp.valueOf(opStr); switch (op) { case CHANGE_PROPS: hiveDB.setParameters(newParams); @@ -335,10 +335,10 @@ private static Database alterDatabase(Database hiveDB, CatalogDatabase newDataba String ownerType = newParams.remove(DATABASE_OWNER_TYPE); hiveDB.setOwnerName(ownerName); switch (ownerType) { - case SqlAlterHiveDatabaseOwner.ROLE_OWNER: + case ROLE_OWNER: hiveDB.setOwnerType(PrincipalType.ROLE); break; - case SqlAlterHiveDatabaseOwner.USER_OWNER: + case USER_OWNER: hiveDB.setOwnerType(PrincipalType.USER); break; default: diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java new file mode 100644 index 000000000000..94ed3b538879 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabase; +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner; +import org.apache.flink.sql.parser.hive.ddl.SqlCreateHiveDatabase; + +/** + * Constants for Hive Catalog. + */ +public class HiveCatalogConstants { + + // ----------------------------------------------------------------------------------- + // Constants for ALTER DATABASE + // ----------------------------------------------------------------------------------- + public static final String ALTER_DATABASE_OP = SqlAlterHiveDatabase.ALTER_DATABASE_OP; + + public static final String DATABASE_LOCATION_URI = SqlCreateHiveDatabase.DATABASE_LOCATION_URI; + + public static final String DATABASE_OWNER_NAME = SqlAlterHiveDatabaseOwner.DATABASE_OWNER_NAME; + + public static final String DATABASE_OWNER_TYPE = SqlAlterHiveDatabaseOwner.DATABASE_OWNER_TYPE; + + public static final String ROLE_OWNER = SqlAlterHiveDatabaseOwner.ROLE_OWNER; + + public static final String USER_OWNER = SqlAlterHiveDatabaseOwner.USER_OWNER; + + /** Type of ALTER DATABASE operation. */ + public enum AlterHiveDatabaseOp { + CHANGE_PROPS, + CHANGE_LOCATION, + CHANGE_OWNER + } +} diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java new file mode 100644 index 000000000000..5d40e7ed1d87 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabase; +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner; +import org.apache.flink.sql.parser.hive.ddl.SqlCreateHiveDatabase; + +/** + * Constants for Hive Catalog. + */ +public class HiveCatalogConstants { + + // ----------------------------------------------------------------------------------- + // Constants for ALTER DATABASE + // ----------------------------------------------------------------------------------- + public static final String ALTER_DATABASE_OP = SqlAlterHiveDatabase.ALTER_DATABASE_OP; + + public static final String DATABASE_LOCATION_URI = SqlCreateHiveDatabase.DATABASE_LOCATION_URI; + + public static final String DATABASE_OWNER_NAME = SqlAlterHiveDatabaseOwner.DATABASE_OWNER_NAME; + + public static final String DATABASE_OWNER_TYPE = SqlAlterHiveDatabaseOwner.DATABASE_OWNER_TYPE; + + public static final String ROLE_OWNER = SqlAlterHiveDatabaseOwner.ROLE_OWNER; + + public static final String USER_OWNER = SqlAlterHiveDatabaseOwner.USER_OWNER; + + /** Type of ALTER DATABASE operation. */ + public enum AlterHiveDatabaseOp { + CHANGE_PROPS, + CHANGE_LOCATION, + CHANGE_OWNER + } +} + diff --git a/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java b/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java new file mode 100644 index 000000000000..5d40e7ed1d87 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabase; +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner; +import org.apache.flink.sql.parser.hive.ddl.SqlCreateHiveDatabase; + +/** + * Constants for Hive Catalog. + */ +public class HiveCatalogConstants { + + // ----------------------------------------------------------------------------------- + // Constants for ALTER DATABASE + // ----------------------------------------------------------------------------------- + public static final String ALTER_DATABASE_OP = SqlAlterHiveDatabase.ALTER_DATABASE_OP; + + public static final String DATABASE_LOCATION_URI = SqlCreateHiveDatabase.DATABASE_LOCATION_URI; + + public static final String DATABASE_OWNER_NAME = SqlAlterHiveDatabaseOwner.DATABASE_OWNER_NAME; + + public static final String DATABASE_OWNER_TYPE = SqlAlterHiveDatabaseOwner.DATABASE_OWNER_TYPE; + + public static final String ROLE_OWNER = SqlAlterHiveDatabaseOwner.ROLE_OWNER; + + public static final String USER_OWNER = SqlAlterHiveDatabaseOwner.USER_OWNER; + + /** Type of ALTER DATABASE operation. */ + public enum AlterHiveDatabaseOp { + CHANGE_PROPS, + CHANGE_LOCATION, + CHANGE_OWNER + } +} + diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java new file mode 100644 index 000000000000..5d40e7ed1d87 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabase; +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner; +import org.apache.flink.sql.parser.hive.ddl.SqlCreateHiveDatabase; + +/** + * Constants for Hive Catalog. + */ +public class HiveCatalogConstants { + + // ----------------------------------------------------------------------------------- + // Constants for ALTER DATABASE + // ----------------------------------------------------------------------------------- + public static final String ALTER_DATABASE_OP = SqlAlterHiveDatabase.ALTER_DATABASE_OP; + + public static final String DATABASE_LOCATION_URI = SqlCreateHiveDatabase.DATABASE_LOCATION_URI; + + public static final String DATABASE_OWNER_NAME = SqlAlterHiveDatabaseOwner.DATABASE_OWNER_NAME; + + public static final String DATABASE_OWNER_TYPE = SqlAlterHiveDatabaseOwner.DATABASE_OWNER_TYPE; + + public static final String ROLE_OWNER = SqlAlterHiveDatabaseOwner.ROLE_OWNER; + + public static final String USER_OWNER = SqlAlterHiveDatabaseOwner.USER_OWNER; + + /** Type of ALTER DATABASE operation. */ + public enum AlterHiveDatabaseOp { + CHANGE_PROPS, + CHANGE_LOCATION, + CHANGE_OWNER + } +} + diff --git a/hudi-flink-datasource/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java b/hudi-flink-datasource/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java new file mode 100644 index 000000000000..5d40e7ed1d87 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabase; +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner; +import org.apache.flink.sql.parser.hive.ddl.SqlCreateHiveDatabase; + +/** + * Constants for Hive Catalog. + */ +public class HiveCatalogConstants { + + // ----------------------------------------------------------------------------------- + // Constants for ALTER DATABASE + // ----------------------------------------------------------------------------------- + public static final String ALTER_DATABASE_OP = SqlAlterHiveDatabase.ALTER_DATABASE_OP; + + public static final String DATABASE_LOCATION_URI = SqlCreateHiveDatabase.DATABASE_LOCATION_URI; + + public static final String DATABASE_OWNER_NAME = SqlAlterHiveDatabaseOwner.DATABASE_OWNER_NAME; + + public static final String DATABASE_OWNER_TYPE = SqlAlterHiveDatabaseOwner.DATABASE_OWNER_TYPE; + + public static final String ROLE_OWNER = SqlAlterHiveDatabaseOwner.ROLE_OWNER; + + public static final String USER_OWNER = SqlAlterHiveDatabaseOwner.USER_OWNER; + + /** Type of ALTER DATABASE operation. */ + public enum AlterHiveDatabaseOp { + CHANGE_PROPS, + CHANGE_LOCATION, + CHANGE_OWNER + } +} + diff --git a/hudi-flink-datasource/hudi-flink1.18.x/pom.xml b/hudi-flink-datasource/hudi-flink1.18.x/pom.xml new file mode 100644 index 000000000000..d391701cd033 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/pom.xml @@ -0,0 +1,168 @@ + + + + + hudi-flink-datasource + org.apache.hudi + 0.14.2-rc1 + + 4.0.0 + + hudi-flink1.18.x + 0.14.2-rc1 + jar + + + ${project.parent.parent.basedir} + + + + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + + org.slf4j + slf4j-api + + + + + org.apache.hudi + hudi-common + ${project.version} + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + provided + + + + + org.apache.flink + flink-connector-hive_2.12 + ${flink1.18.version} + provided + + + org.apache.flink + flink-table-api-java + ${flink1.18.version} + provided + + + org.apache.flink + flink-table-api-java-bridge + ${flink1.18.version} + provided + + + org.apache.flink + flink-shaded-guava + 30.1.1-jre-14.0 + provided + + + org.apache.flink + flink-core + ${flink1.18.version} + provided + + + org.apache.flink + flink-streaming-java + ${flink1.18.version} + provided + + + org.apache.flink + flink-table-runtime + ${flink1.18.version} + provided + + + org.apache.flink + flink-parquet + ${flink1.18.version} + provided + + + org.apache.flink + flink-json + ${flink1.18.version} + provided + + + org.apache.flink + flink-table-planner_2.12 + ${flink1.18.version} + provided + + + + + org.apache.flink + flink-runtime + ${flink1.18.version} + test + test-jar + + + org.apache.hudi + hudi-tests-common + ${project.version} + test + + + + + + + org.jacoco + jacoco-maven-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + test-compile + + + + false + + + + org.apache.rat + apache-rat-plugin + + + + diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java new file mode 100644 index 000000000000..d4c6bc3a8f4d --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; + +/** + * Adapter clazz for {@code AbstractStreamOperator}. + */ +public abstract class AbstractStreamOperatorAdapter extends AbstractStreamOperator { +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java new file mode 100644 index 000000000000..6dcfe71ccfd9 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; + +/** + * Adapter clazz for {@link AbstractStreamOperatorFactory}. + */ +public abstract class AbstractStreamOperatorFactoryAdapter + extends AbstractStreamOperatorFactory implements YieldingOperatorFactory { + + public MailboxExecutorAdapter getMailboxExecutorAdapter() { + return new MailboxExecutorAdapter(getMailboxExecutor()); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/DataStreamScanProviderAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/DataStreamScanProviderAdapter.java new file mode 100644 index 000000000000..a6b5439ea1ff --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/DataStreamScanProviderAdapter.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.source.DataStreamScanProvider; +import org.apache.flink.table.data.RowData; + +/** + * Adapter clazz for {@code DataStreamScanProvider}. + */ +public interface DataStreamScanProviderAdapter extends DataStreamScanProvider { + default DataStream produceDataStream(ProviderContext providerContext, StreamExecutionEnvironment streamExecutionEnvironment) { + return produceDataStream(streamExecutionEnvironment); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/DataStreamSinkProviderAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/DataStreamSinkProviderAdapter.java new file mode 100644 index 000000000000..349f60f30acf --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/DataStreamSinkProviderAdapter.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.sink.DataStreamSinkProvider; +import org.apache.flink.table.data.RowData; + +/** + * Adapter clazz for {@code DataStreamSinkProvider}. + */ +public interface DataStreamSinkProviderAdapter extends DataStreamSinkProvider { + DataStreamSink consumeDataStream(DataStream dataStream); + + @Override + default DataStreamSink consumeDataStream(ProviderContext providerContext, DataStream dataStream) { + return consumeDataStream(dataStream); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java new file mode 100644 index 000000000000..7c1649301607 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.table.catalog.hive.util.Constants; + +/** + * Constants for Hive Catalog. + */ +public class HiveCatalogConstants { + + // ----------------------------------------------------------------------------------- + // Constants for ALTER DATABASE + // ----------------------------------------------------------------------------------- + public static final String ALTER_DATABASE_OP = Constants.ALTER_DATABASE_OP; + + public static final String DATABASE_LOCATION_URI = Constants.DATABASE_LOCATION_URI; + + public static final String DATABASE_OWNER_NAME = Constants.DATABASE_OWNER_NAME; + + public static final String DATABASE_OWNER_TYPE = Constants.DATABASE_OWNER_TYPE; + + public static final String ROLE_OWNER = Constants.ROLE_OWNER; + + public static final String USER_OWNER = Constants.USER_OWNER; + + /** Type of ALTER DATABASE operation. */ + public enum AlterHiveDatabaseOp { + CHANGE_PROPS, + CHANGE_LOCATION, + CHANGE_OWNER + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java new file mode 100644 index 000000000000..0c836f3db391 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.util.function.ThrowingRunnable; + +/** + * Adapter clazz for {@link MailboxExecutor}. + */ +public class MailboxExecutorAdapter { + private final MailboxExecutor executor; + + public MailboxExecutorAdapter(MailboxExecutor executor) { + this.executor = executor; + } + + public void execute(ThrowingRunnable command, String description) { + this.executor.execute(command, description); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java new file mode 100644 index 000000000000..e84da0d6ec30 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; +import org.apache.flink.util.OutputTag; + +/** Adapter class for {@code Output} to handle async compaction/clustering service thread safe issues */ +public class MaskingOutputAdapter implements Output> { + + private final Output> output; + + public MaskingOutputAdapter(Output> output) { + this.output = output; + } + + @Override + public void emitWatermark(Watermark watermark) { + // For thread safe, not to propagate the watermark + } + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) { + // For thread safe, not to propagate latency marker + } + + @Override + public void emitWatermarkStatus(WatermarkStatus watermarkStatus) { + // For thread safe, not to propagate watermark status + } + + @Override + public void collect(OutputTag outputTag, StreamRecord streamRecord) { + this.output.collect(outputTag, streamRecord); + } + + @Override + public void collect(StreamRecord outStreamRecord) { + this.output.collect(outStreamRecord); + } + + @Override + public void close() { + this.output.close(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/OperatorCoordinatorAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/OperatorCoordinatorAdapter.java new file mode 100644 index 000000000000..9c37de17bd1f --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/OperatorCoordinatorAdapter.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; + +import javax.annotation.Nullable; + +/** + * Adapter clazz for {@code OperatorCoordinator}. + */ +public interface OperatorCoordinatorAdapter extends OperatorCoordinator { + void handleEventFromOperator(int i, OperatorEvent operatorEvent) throws Exception; + + @Override + default void handleEventFromOperator(int i, int attemptNumber, OperatorEvent operatorEvent) throws Exception { + handleEventFromOperator(i, operatorEvent); + } + + void subtaskReady(int i, SubtaskGateway subtaskGateway); + + @Override + default void executionAttemptReady(int i, int attemptNumber, SubtaskGateway subtaskGateway) { + subtaskReady(i, subtaskGateway); + } + + @Override + default void executionAttemptFailed(int i, int attemptNumber, Throwable throwable) { + subtaskReady(i, null); + } + + void subtaskFailed(int i, @Nullable Throwable throwable); +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java new file mode 100644 index 000000000000..865c0c81d4d9 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.shaded.guava30.com.google.common.util.concurrent.RateLimiter; + +/** + * Bridge class for shaded guava clazz {@code RateLimiter}. + */ +public class RateLimiterAdapter { + private final RateLimiter rateLimiter; + + private RateLimiterAdapter(double permitsPerSecond) { + this.rateLimiter = RateLimiter.create(permitsPerSecond); + } + + public static RateLimiterAdapter create(double permitsPerSecond) { + return new RateLimiterAdapter(permitsPerSecond); + } + + public void acquire() { + this.rateLimiter.acquire(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/SortCodeGeneratorAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/SortCodeGeneratorAdapter.java new file mode 100644 index 000000000000..e38a58a0ccfb --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/SortCodeGeneratorAdapter.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator; +import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec; +import org.apache.flink.table.types.logical.RowType; + +/** + * Adapter clazz for {@code SortCodeGenerator}. + */ +public class SortCodeGeneratorAdapter extends SortCodeGenerator { + public SortCodeGeneratorAdapter(ReadableConfig tableConfig, RowType input, SortSpec sortSpec) { + super(tableConfig, Thread.currentThread().getContextClassLoader(), input, sortSpec); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/SupportsRowLevelDeleteAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/SupportsRowLevelDeleteAdapter.java new file mode 100644 index 000000000000..de0019d41bd9 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/SupportsRowLevelDeleteAdapter.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.table.connector.RowLevelModificationScanContext; +import org.apache.flink.table.connector.sink.abilities.SupportsRowLevelDelete; + +import javax.annotation.Nullable; + +/** + * Adapter clazz for {@link org.apache.flink.table.connector.sink.abilities.SupportsRowLevelDelete}. + */ +public interface SupportsRowLevelDeleteAdapter extends SupportsRowLevelDelete { + @Override + default RowLevelDeleteInfo applyRowLevelDelete(@Nullable RowLevelModificationScanContext context) { + return applyRowLevelDelete(); + } + + RowLevelDeleteInfoAdapter applyRowLevelDelete(); + + /** + * Adapter clazz for {@link SupportsRowLevelDelete.RowLevelDeleteInfo}. + */ + interface RowLevelDeleteInfoAdapter extends RowLevelDeleteInfo { + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/SupportsRowLevelUpdateAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/SupportsRowLevelUpdateAdapter.java new file mode 100644 index 000000000000..17c785d48455 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/SupportsRowLevelUpdateAdapter.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.connector.RowLevelModificationScanContext; +import org.apache.flink.table.connector.sink.abilities.SupportsRowLevelUpdate; + +import javax.annotation.Nullable; + +import java.util.List; + +/** + * Adapter clazz for {@link org.apache.flink.table.connector.sink.abilities.SupportsRowLevelUpdate}. + */ +public interface SupportsRowLevelUpdateAdapter extends SupportsRowLevelUpdate { + @Override + default RowLevelUpdateInfo applyRowLevelUpdate(List updatedColumns, @Nullable RowLevelModificationScanContext context) { + return applyRowLevelUpdate(updatedColumns); + } + + RowLevelUpdateInfoAdapter applyRowLevelUpdate(List updatedColumns); + + /** + * Adapter clazz for {@link SupportsRowLevelUpdate.RowLevelUpdateInfo}. + */ + interface RowLevelUpdateInfoAdapter extends RowLevelUpdateInfo { + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/Utils.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/Utils.java new file mode 100644 index 000000000000..659c65973674 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/Utils.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamSourceContexts; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.runtime.generated.NormalizedKeyComputer; +import org.apache.flink.table.runtime.generated.RecordComparator; +import org.apache.flink.table.runtime.operators.sort.BinaryExternalSorter; +import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer; +import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer; + +import java.util.Collections; + +/** + * Adapter utils. + */ +public class Utils { + public static SourceFunction.SourceContext getSourceContext( + TimeCharacteristic timeCharacteristic, + ProcessingTimeService processingTimeService, + StreamTask streamTask, + Output> output, + long watermarkInterval) { + return StreamSourceContexts.getSourceContext( + timeCharacteristic, + processingTimeService, + new Object(), // no actual locking needed + output, + watermarkInterval, + -1, + true); + } + + public static FactoryUtil.DefaultDynamicTableContext getTableContext( + ObjectIdentifier tablePath, + ResolvedCatalogTable catalogTable, + ReadableConfig conf) { + return new FactoryUtil.DefaultDynamicTableContext(tablePath, catalogTable, + Collections.emptyMap(), conf, Thread.currentThread().getContextClassLoader(), false); + } + + public static BinaryExternalSorter getBinaryExternalSorter( + final Object owner, + MemoryManager memoryManager, + long reservedMemorySize, + IOManager ioManager, + AbstractRowDataSerializer inputSerializer, + BinaryRowDataSerializer serializer, + NormalizedKeyComputer normalizedKeyComputer, + RecordComparator comparator, + Configuration conf) { + return new BinaryExternalSorter(owner, memoryManager, reservedMemorySize, + ioManager, inputSerializer, serializer, normalizedKeyComputer, comparator, + conf.get(ExecutionConfigOptions.TABLE_EXEC_SORT_MAX_NUM_FILE_HANDLES), + conf.get(ExecutionConfigOptions.TABLE_EXEC_SPILL_COMPRESSION_ENABLED), + (int) conf.get( + ExecutionConfigOptions.TABLE_EXEC_SPILL_COMPRESSION_BLOCK_SIZE).getBytes(), + conf.get(ExecutionConfigOptions.TABLE_EXEC_SORT_ASYNC_MERGE_ENABLED)); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java new file mode 100644 index 000000000000..9bf5390ee26c --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java @@ -0,0 +1,579 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow; + +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.table.format.cow.vector.HeapArrayVector; +import org.apache.hudi.table.format.cow.vector.HeapMapColumnVector; +import org.apache.hudi.table.format.cow.vector.HeapRowColumnVector; +import org.apache.hudi.table.format.cow.vector.ParquetDecimalVector; +import org.apache.hudi.table.format.cow.vector.reader.ArrayColumnReader; +import org.apache.hudi.table.format.cow.vector.reader.EmptyColumnReader; +import org.apache.hudi.table.format.cow.vector.reader.FixedLenBytesColumnReader; +import org.apache.hudi.table.format.cow.vector.reader.Int64TimestampColumnReader; +import org.apache.hudi.table.format.cow.vector.reader.MapColumnReader; +import org.apache.hudi.table.format.cow.vector.reader.ParquetColumnarRowSplitReader; +import org.apache.hudi.table.format.cow.vector.reader.RowColumnReader; + +import org.apache.flink.core.fs.Path; +import org.apache.flink.formats.parquet.vector.reader.BooleanColumnReader; +import org.apache.flink.formats.parquet.vector.reader.ByteColumnReader; +import org.apache.flink.formats.parquet.vector.reader.BytesColumnReader; +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.formats.parquet.vector.reader.DoubleColumnReader; +import org.apache.flink.formats.parquet.vector.reader.FloatColumnReader; +import org.apache.flink.formats.parquet.vector.reader.IntColumnReader; +import org.apache.flink.formats.parquet.vector.reader.LongColumnReader; +import org.apache.flink.formats.parquet.vector.reader.ShortColumnReader; +import org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.data.columnar.vector.ColumnVector; +import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch; +import org.apache.flink.table.data.columnar.vector.heap.HeapBooleanVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapByteVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapBytesVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapDoubleVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapFloatVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapIntVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapLongVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapShortVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapTimestampVector; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.util.Preconditions; +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.ParquetRuntimeException; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.filter.UnboundRecordFilter; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.InvalidSchemaException; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; + +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Date; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.flink.table.utils.DateTimeUtils.toInternal; +import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; +import static org.apache.parquet.Preconditions.checkArgument; + +/** + * Util for generating {@link ParquetColumnarRowSplitReader}. + * + *

NOTE: reference from Flink release 1.11.2 {@code ParquetSplitReaderUtil}, modify to support INT64 + * based TIMESTAMP_MILLIS as ConvertedType, should remove when Flink supports that. + */ +public class ParquetSplitReaderUtil { + + /** + * Util for generating partitioned {@link ParquetColumnarRowSplitReader}. + */ + public static ParquetColumnarRowSplitReader genPartColumnarRowReader( + boolean utcTimestamp, + boolean caseSensitive, + Configuration conf, + String[] fullFieldNames, + DataType[] fullFieldTypes, + Map partitionSpec, + int[] selectedFields, + int batchSize, + Path path, + long splitStart, + long splitLength, + FilterPredicate filterPredicate, + UnboundRecordFilter recordFilter) throws IOException { + List selNonPartNames = Arrays.stream(selectedFields) + .mapToObj(i -> fullFieldNames[i]) + .filter(n -> !partitionSpec.containsKey(n)) + .collect(Collectors.toList()); + + int[] selParquetFields = Arrays.stream(selectedFields) + .filter(i -> !partitionSpec.containsKey(fullFieldNames[i])) + .toArray(); + + ParquetColumnarRowSplitReader.ColumnBatchGenerator gen = readVectors -> { + // create and initialize the row batch + ColumnVector[] vectors = new ColumnVector[selectedFields.length]; + for (int i = 0; i < vectors.length; i++) { + String name = fullFieldNames[selectedFields[i]]; + LogicalType type = fullFieldTypes[selectedFields[i]].getLogicalType(); + vectors[i] = createVector(readVectors, selNonPartNames, name, type, partitionSpec, batchSize); + } + return new VectorizedColumnBatch(vectors); + }; + + return new ParquetColumnarRowSplitReader( + utcTimestamp, + caseSensitive, + conf, + Arrays.stream(selParquetFields) + .mapToObj(i -> fullFieldTypes[i].getLogicalType()) + .toArray(LogicalType[]::new), + selNonPartNames.toArray(new String[0]), + gen, + batchSize, + new org.apache.hadoop.fs.Path(path.toUri()), + splitStart, + splitLength, + filterPredicate, + recordFilter); + } + + private static ColumnVector createVector( + ColumnVector[] readVectors, + List selNonPartNames, + String name, + LogicalType type, + Map partitionSpec, + int batchSize) { + if (partitionSpec.containsKey(name)) { + return createVectorFromConstant(type, partitionSpec.get(name), batchSize); + } + ColumnVector readVector = readVectors[selNonPartNames.indexOf(name)]; + if (readVector == null) { + // when the read vector is null, use a constant null vector instead + readVector = createVectorFromConstant(type, null, batchSize); + } + return readVector; + } + + private static ColumnVector createVectorFromConstant( + LogicalType type, + Object value, + int batchSize) { + switch (type.getTypeRoot()) { + case CHAR: + case VARCHAR: + case BINARY: + case VARBINARY: + HeapBytesVector bsv = new HeapBytesVector(batchSize); + if (value == null) { + bsv.fillWithNulls(); + } else { + bsv.fill(value instanceof byte[] + ? (byte[]) value + : getUTF8Bytes(value.toString())); + } + return bsv; + case BOOLEAN: + HeapBooleanVector bv = new HeapBooleanVector(batchSize); + if (value == null) { + bv.fillWithNulls(); + } else { + bv.fill((boolean) value); + } + return bv; + case TINYINT: + HeapByteVector byteVector = new HeapByteVector(batchSize); + if (value == null) { + byteVector.fillWithNulls(); + } else { + byteVector.fill(((Number) value).byteValue()); + } + return byteVector; + case SMALLINT: + HeapShortVector sv = new HeapShortVector(batchSize); + if (value == null) { + sv.fillWithNulls(); + } else { + sv.fill(((Number) value).shortValue()); + } + return sv; + case INTEGER: + HeapIntVector iv = new HeapIntVector(batchSize); + if (value == null) { + iv.fillWithNulls(); + } else { + iv.fill(((Number) value).intValue()); + } + return iv; + case BIGINT: + HeapLongVector lv = new HeapLongVector(batchSize); + if (value == null) { + lv.fillWithNulls(); + } else { + lv.fill(((Number) value).longValue()); + } + return lv; + case DECIMAL: + DecimalType decimalType = (DecimalType) type; + int precision = decimalType.getPrecision(); + int scale = decimalType.getScale(); + DecimalData decimal = value == null + ? null + : Preconditions.checkNotNull(DecimalData.fromBigDecimal((BigDecimal) value, precision, scale)); + ColumnVector internalVector = createVectorFromConstant( + new VarBinaryType(), + decimal == null ? null : decimal.toUnscaledBytes(), + batchSize); + return new ParquetDecimalVector(internalVector); + case FLOAT: + HeapFloatVector fv = new HeapFloatVector(batchSize); + if (value == null) { + fv.fillWithNulls(); + } else { + fv.fill(((Number) value).floatValue()); + } + return fv; + case DOUBLE: + HeapDoubleVector dv = new HeapDoubleVector(batchSize); + if (value == null) { + dv.fillWithNulls(); + } else { + dv.fill(((Number) value).doubleValue()); + } + return dv; + case DATE: + if (value instanceof LocalDate) { + value = Date.valueOf((LocalDate) value); + } + return createVectorFromConstant( + new IntType(), + value == null ? null : toInternal((Date) value), + batchSize); + case TIMESTAMP_WITHOUT_TIME_ZONE: + HeapTimestampVector tv = new HeapTimestampVector(batchSize); + if (value == null) { + tv.fillWithNulls(); + } else { + tv.fill(TimestampData.fromLocalDateTime((LocalDateTime) value)); + } + return tv; + case ARRAY: + HeapArrayVector arrayVector = new HeapArrayVector(batchSize); + if (value == null) { + arrayVector.fillWithNulls(); + return arrayVector; + } else { + throw new UnsupportedOperationException("Unsupported create array with default value."); + } + case MAP: + HeapMapColumnVector mapVector = new HeapMapColumnVector(batchSize, null, null); + if (value == null) { + mapVector.fillWithNulls(); + return mapVector; + } else { + throw new UnsupportedOperationException("Unsupported create map with default value."); + } + case ROW: + HeapRowColumnVector rowVector = new HeapRowColumnVector(batchSize); + if (value == null) { + rowVector.fillWithNulls(); + return rowVector; + } else { + throw new UnsupportedOperationException("Unsupported create row with default value."); + } + default: + throw new UnsupportedOperationException("Unsupported type: " + type); + } + } + + private static List filterDescriptors(int depth, Type type, List columns) throws ParquetRuntimeException { + List filtered = new ArrayList<>(); + for (ColumnDescriptor descriptor : columns) { + if (depth >= descriptor.getPath().length) { + throw new InvalidSchemaException("Expect depth " + depth + " for schema: " + descriptor); + } + if (type.getName().equals(descriptor.getPath()[depth])) { + filtered.add(descriptor); + } + } + ValidationUtils.checkState(filtered.size() > 0, "Corrupted Parquet schema"); + return filtered; + } + + public static ColumnReader createColumnReader( + boolean utcTimestamp, + LogicalType fieldType, + Type physicalType, + List descriptors, + PageReadStore pages) throws IOException { + return createColumnReader(utcTimestamp, fieldType, physicalType, descriptors, + pages, 0); + } + + private static ColumnReader createColumnReader( + boolean utcTimestamp, + LogicalType fieldType, + Type physicalType, + List columns, + PageReadStore pages, + int depth) throws IOException { + List descriptors = filterDescriptors(depth, physicalType, columns); + ColumnDescriptor descriptor = descriptors.get(0); + PageReader pageReader = pages.getPageReader(descriptor); + switch (fieldType.getTypeRoot()) { + case BOOLEAN: + return new BooleanColumnReader(descriptor, pageReader); + case TINYINT: + return new ByteColumnReader(descriptor, pageReader); + case DOUBLE: + return new DoubleColumnReader(descriptor, pageReader); + case FLOAT: + return new FloatColumnReader(descriptor, pageReader); + case INTEGER: + case DATE: + case TIME_WITHOUT_TIME_ZONE: + return new IntColumnReader(descriptor, pageReader); + case BIGINT: + return new LongColumnReader(descriptor, pageReader); + case SMALLINT: + return new ShortColumnReader(descriptor, pageReader); + case CHAR: + case VARCHAR: + case BINARY: + case VARBINARY: + return new BytesColumnReader(descriptor, pageReader); + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + switch (descriptor.getPrimitiveType().getPrimitiveTypeName()) { + case INT64: + int precision = fieldType instanceof TimestampType + ? ((TimestampType) fieldType).getPrecision() + : ((LocalZonedTimestampType) fieldType).getPrecision(); + return new Int64TimestampColumnReader(utcTimestamp, descriptor, pageReader, precision); + case INT96: + return new TimestampColumnReader(utcTimestamp, descriptor, pageReader); + default: + throw new AssertionError(); + } + case DECIMAL: + switch (descriptor.getPrimitiveType().getPrimitiveTypeName()) { + case INT32: + return new IntColumnReader(descriptor, pageReader); + case INT64: + return new LongColumnReader(descriptor, pageReader); + case BINARY: + return new BytesColumnReader(descriptor, pageReader); + case FIXED_LEN_BYTE_ARRAY: + return new FixedLenBytesColumnReader( + descriptor, pageReader); + default: + throw new AssertionError(); + } + case ARRAY: + return new ArrayColumnReader( + descriptor, + pageReader, + utcTimestamp, + descriptor.getPrimitiveType(), + fieldType); + case MAP: + MapType mapType = (MapType) fieldType; + ArrayColumnReader keyReader = + new ArrayColumnReader( + descriptor, + pageReader, + utcTimestamp, + descriptor.getPrimitiveType(), + new ArrayType(mapType.getKeyType())); + ArrayColumnReader valueReader = + new ArrayColumnReader( + descriptors.get(1), + pages.getPageReader(descriptors.get(1)), + utcTimestamp, + descriptors.get(1).getPrimitiveType(), + new ArrayType(mapType.getValueType())); + return new MapColumnReader(keyReader, valueReader, fieldType); + case ROW: + RowType rowType = (RowType) fieldType; + GroupType groupType = physicalType.asGroupType(); + List fieldReaders = new ArrayList<>(); + for (int i = 0; i < rowType.getFieldCount(); i++) { + // schema evolution: read the parquet file with a new extended field name. + int fieldIndex = getFieldIndexInPhysicalType(rowType.getFields().get(i).getName(), groupType); + if (fieldIndex < 0) { + fieldReaders.add(new EmptyColumnReader()); + } else { + fieldReaders.add( + createColumnReader( + utcTimestamp, + rowType.getTypeAt(i), + groupType.getType(fieldIndex), + descriptors, + pages, + depth + 1)); + } + } + return new RowColumnReader(fieldReaders); + default: + throw new UnsupportedOperationException(fieldType + " is not supported now."); + } + } + + public static WritableColumnVector createWritableColumnVector( + int batchSize, + LogicalType fieldType, + Type physicalType, + List descriptors) { + return createWritableColumnVector(batchSize, fieldType, physicalType, descriptors, 0); + } + + private static WritableColumnVector createWritableColumnVector( + int batchSize, + LogicalType fieldType, + Type physicalType, + List columns, + int depth) { + List descriptors = filterDescriptors(depth, physicalType, columns); + PrimitiveType primitiveType = descriptors.get(0).getPrimitiveType(); + PrimitiveType.PrimitiveTypeName typeName = primitiveType.getPrimitiveTypeName(); + switch (fieldType.getTypeRoot()) { + case BOOLEAN: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.BOOLEAN, + "Unexpected type: %s", typeName); + return new HeapBooleanVector(batchSize); + case TINYINT: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.INT32, + "Unexpected type: %s", typeName); + return new HeapByteVector(batchSize); + case DOUBLE: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.DOUBLE, + "Unexpected type: %s", typeName); + return new HeapDoubleVector(batchSize); + case FLOAT: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.FLOAT, + "Unexpected type: %s", typeName); + return new HeapFloatVector(batchSize); + case INTEGER: + case DATE: + case TIME_WITHOUT_TIME_ZONE: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.INT32, + "Unexpected type: %s", typeName); + return new HeapIntVector(batchSize); + case BIGINT: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.INT64, + "Unexpected type: %s", typeName); + return new HeapLongVector(batchSize); + case SMALLINT: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.INT32, + "Unexpected type: %s", typeName); + return new HeapShortVector(batchSize); + case CHAR: + case VARCHAR: + case BINARY: + case VARBINARY: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.BINARY, + "Unexpected type: %s", typeName); + return new HeapBytesVector(batchSize); + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + checkArgument(primitiveType.getOriginalType() != OriginalType.TIME_MICROS, + "TIME_MICROS original type is not "); + return new HeapTimestampVector(batchSize); + case DECIMAL: + checkArgument( + (typeName == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY + || typeName == PrimitiveType.PrimitiveTypeName.BINARY) + && primitiveType.getOriginalType() == OriginalType.DECIMAL, + "Unexpected type: %s", typeName); + return new HeapBytesVector(batchSize); + case ARRAY: + ArrayType arrayType = (ArrayType) fieldType; + return new HeapArrayVector( + batchSize, + createWritableColumnVector( + batchSize, + arrayType.getElementType(), + physicalType, + descriptors, + depth)); + case MAP: + MapType mapType = (MapType) fieldType; + GroupType repeatedType = physicalType.asGroupType().getType(0).asGroupType(); + // the map column has three level paths. + return new HeapMapColumnVector( + batchSize, + createWritableColumnVector( + batchSize, + mapType.getKeyType(), + repeatedType.getType(0), + descriptors, + depth + 2), + createWritableColumnVector( + batchSize, + mapType.getValueType(), + repeatedType.getType(1), + descriptors, + depth + 2)); + case ROW: + RowType rowType = (RowType) fieldType; + GroupType groupType = physicalType.asGroupType(); + WritableColumnVector[] columnVectors = new WritableColumnVector[rowType.getFieldCount()]; + for (int i = 0; i < columnVectors.length; i++) { + // schema evolution: read the file with a new extended field name. + int fieldIndex = getFieldIndexInPhysicalType(rowType.getFields().get(i).getName(), groupType); + if (fieldIndex < 0) { + columnVectors[i] = (WritableColumnVector) createVectorFromConstant(rowType.getTypeAt(i), null, batchSize); + } else { + columnVectors[i] = + createWritableColumnVector( + batchSize, + rowType.getTypeAt(i), + groupType.getType(fieldIndex), + descriptors, + depth + 1); + } + } + return new HeapRowColumnVector(batchSize, columnVectors); + default: + throw new UnsupportedOperationException(fieldType + " is not supported now."); + } + } + + /** + * Returns the field index with given physical row type {@code groupType} and field name {@code fieldName}. + * + * @return The physical field index or -1 if the field does not exist + */ + private static int getFieldIndexInPhysicalType(String fieldName, GroupType groupType) { + // get index from fileSchema type, else, return -1 + return groupType.containsField(fieldName) ? groupType.getFieldIndex(fieldName) : -1; + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java new file mode 100644 index 000000000000..7db66d23d6fc --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow.vector; + +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.columnar.ColumnarArrayData; +import org.apache.flink.table.data.columnar.vector.ArrayColumnVector; +import org.apache.flink.table.data.columnar.vector.ColumnVector; +import org.apache.flink.table.data.columnar.vector.heap.AbstractHeapVector; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; + +/** + * This class represents a nullable heap array column vector. + */ +public class HeapArrayVector extends AbstractHeapVector + implements WritableColumnVector, ArrayColumnVector { + + public long[] offsets; + public long[] lengths; + public ColumnVector child; + private int size; + + public HeapArrayVector(int len) { + super(len); + offsets = new long[len]; + lengths = new long[len]; + } + + public HeapArrayVector(int len, ColumnVector vector) { + super(len); + offsets = new long[len]; + lengths = new long[len]; + this.child = vector; + } + + public int getSize() { + return size; + } + + public void setSize(int size) { + this.size = size; + } + + public int getLen() { + return this.isNull.length; + } + + @Override + public ArrayData getArray(int i) { + long offset = offsets[i]; + long length = lengths[i]; + return new ColumnarArrayData(child, (int) offset, (int) length); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java new file mode 100644 index 000000000000..a37973716950 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow.vector; + +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.columnar.ColumnarMapData; +import org.apache.flink.table.data.columnar.vector.ColumnVector; +import org.apache.flink.table.data.columnar.vector.MapColumnVector; +import org.apache.flink.table.data.columnar.vector.heap.AbstractHeapVector; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; + +/** + * This class represents a nullable heap map column vector. + */ +public class HeapMapColumnVector extends AbstractHeapVector + implements WritableColumnVector, MapColumnVector { + + private long[] offsets; + private long[] lengths; + private int size; + private ColumnVector keys; + private ColumnVector values; + + public HeapMapColumnVector(int len, ColumnVector keys, ColumnVector values) { + super(len); + size = 0; + offsets = new long[len]; + lengths = new long[len]; + this.keys = keys; + this.values = values; + } + + public void setOffsets(long[] offsets) { + this.offsets = offsets; + } + + public void setLengths(long[] lengths) { + this.lengths = lengths; + } + + public void setKeys(ColumnVector keys) { + this.keys = keys; + } + + public void setValues(ColumnVector values) { + this.values = values; + } + + public int getSize() { + return size; + } + + public void setSize(int size) { + this.size = size; + } + + @Override + public MapData getMap(int i) { + long offset = offsets[i]; + long length = lengths[i]; + return new ColumnarMapData(keys, values, (int) offset, (int) length); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java new file mode 100644 index 000000000000..ae194e4e6ab0 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow.vector; + +import org.apache.flink.table.data.columnar.ColumnarRowData; +import org.apache.flink.table.data.columnar.vector.RowColumnVector; +import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch; +import org.apache.flink.table.data.columnar.vector.heap.AbstractHeapVector; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; + +/** + * This class represents a nullable heap row column vector. + */ +public class HeapRowColumnVector extends AbstractHeapVector + implements WritableColumnVector, RowColumnVector { + + public WritableColumnVector[] vectors; + + public HeapRowColumnVector(int len, WritableColumnVector... vectors) { + super(len); + this.vectors = vectors; + } + + @Override + public ColumnarRowData getRow(int i) { + ColumnarRowData columnarRowData = new ColumnarRowData(new VectorizedColumnBatch(vectors)); + columnarRowData.setRowId(i); + return columnarRowData; + } + + @Override + public void reset() { + super.reset(); + for (WritableColumnVector vector : vectors) { + vector.reset(); + } + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/ParquetDecimalVector.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/ParquetDecimalVector.java new file mode 100644 index 000000000000..98b5e6105089 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/ParquetDecimalVector.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow.vector; + +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.columnar.vector.BytesColumnVector; +import org.apache.flink.table.data.columnar.vector.ColumnVector; +import org.apache.flink.table.data.columnar.vector.DecimalColumnVector; + +/** + * Parquet write decimal as int32 and int64 and binary, this class wrap the real vector to + * provide {@link DecimalColumnVector} interface. + * + *

Reference Flink release 1.11.2 {@link org.apache.flink.formats.parquet.vector.ParquetDecimalVector} + * because it is not public. + */ +public class ParquetDecimalVector implements DecimalColumnVector { + + public final ColumnVector vector; + + public ParquetDecimalVector(ColumnVector vector) { + this.vector = vector; + } + + @Override + public DecimalData getDecimal(int i, int precision, int scale) { + return DecimalData.fromUnscaledBytes( + ((BytesColumnVector) vector).getBytes(i).getBytes(), + precision, + scale); + } + + @Override + public boolean isNullAt(int i) { + return vector.isNullAt(i); + } +} + diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/AbstractColumnReader.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/AbstractColumnReader.java new file mode 100644 index 000000000000..a8b733de636a --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/AbstractColumnReader.java @@ -0,0 +1,325 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.flink.formats.parquet.vector.ParquetDictionary; +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; +import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector; +import org.apache.parquet.Preconditions; +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.Dictionary; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.page.DataPage; +import org.apache.parquet.column.page.DataPageV1; +import org.apache.parquet.column.page.DataPageV2; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.values.ValuesReader; +import org.apache.parquet.io.ParquetDecodingException; +import org.apache.parquet.schema.PrimitiveType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL; + +/** + * Abstract {@link ColumnReader}. + * See {@link org.apache.parquet.column.impl.ColumnReaderImpl}, + * part of the code is referred from Apache Spark and Apache Parquet. + * + *

Note: Reference Flink release 1.11.2 {@link org.apache.flink.formats.parquet.vector.reader.AbstractColumnReader} + * because some of the package scope methods. + */ +public abstract class AbstractColumnReader + implements ColumnReader { + + private static final Logger LOG = LoggerFactory.getLogger(org.apache.flink.formats.parquet.vector.reader.AbstractColumnReader.class); + + private final PageReader pageReader; + + /** + * The dictionary, if this column has dictionary encoding. + */ + protected final Dictionary dictionary; + + /** + * Maximum definition level for this column. + */ + protected final int maxDefLevel; + + protected final ColumnDescriptor descriptor; + + /** + * Total number of values read. + */ + private long valuesRead; + + /** + * value that indicates the end of the current page. That is, if valuesRead == + * endOfPageValueCount, we are at the end of the page. + */ + private long endOfPageValueCount; + + /** + * If true, the current page is dictionary encoded. + */ + private boolean isCurrentPageDictionaryEncoded; + + /** + * Total values in the current page. + */ + private int pageValueCount; + + /* + * Input streams: + * 1.Run length encoder to encode every data, so we have run length stream to get + * run length information. + * 2.Data maybe is real data, maybe is dictionary ids which need be decode to real + * data from Dictionary. + * + * Run length stream ------> Data stream + * | + * ------> Dictionary ids stream + */ + + /** + * Run length decoder for data and dictionary. + */ + protected RunLengthDecoder runLenDecoder; + + /** + * Data input stream. + */ + ByteBufferInputStream dataInputStream; + + /** + * Dictionary decoder to wrap dictionary ids input stream. + */ + private RunLengthDecoder dictionaryIdsDecoder; + + public AbstractColumnReader( + ColumnDescriptor descriptor, + PageReader pageReader) throws IOException { + this.descriptor = descriptor; + this.pageReader = pageReader; + this.maxDefLevel = descriptor.getMaxDefinitionLevel(); + + DictionaryPage dictionaryPage = pageReader.readDictionaryPage(); + if (dictionaryPage != null) { + try { + this.dictionary = dictionaryPage.getEncoding().initDictionary(descriptor, dictionaryPage); + this.isCurrentPageDictionaryEncoded = true; + } catch (IOException e) { + throw new IOException("could not decode the dictionary for " + descriptor, e); + } + } else { + this.dictionary = null; + this.isCurrentPageDictionaryEncoded = false; + } + /* + * Total number of values in this column (in this row group). + */ + long totalValueCount = pageReader.getTotalValueCount(); + if (totalValueCount == 0) { + throw new IOException("totalValueCount == 0"); + } + } + + protected void checkTypeName(PrimitiveType.PrimitiveTypeName expectedName) { + PrimitiveType.PrimitiveTypeName actualName = descriptor.getPrimitiveType().getPrimitiveTypeName(); + Preconditions.checkArgument( + actualName == expectedName, + "Expected type name: %s, actual type name: %s", + expectedName, + actualName); + } + + /** + * Reads `total` values from this columnReader into column. + */ + @Override + public final void readToVector(int readNumber, V vector) throws IOException { + int rowId = 0; + WritableIntVector dictionaryIds = null; + if (dictionary != null) { + dictionaryIds = vector.reserveDictionaryIds(readNumber); + } + while (readNumber > 0) { + // Compute the number of values we want to read in this page. + int leftInPage = (int) (endOfPageValueCount - valuesRead); + if (leftInPage == 0) { + DataPage page = pageReader.readPage(); + if (page instanceof DataPageV1) { + readPageV1((DataPageV1) page); + } else if (page instanceof DataPageV2) { + readPageV2((DataPageV2) page); + } else { + throw new RuntimeException("Unsupported page type: " + page.getClass()); + } + leftInPage = (int) (endOfPageValueCount - valuesRead); + } + int num = Math.min(readNumber, leftInPage); + if (isCurrentPageDictionaryEncoded) { + // Read and decode dictionary ids. + runLenDecoder.readDictionaryIds( + num, dictionaryIds, vector, rowId, maxDefLevel, this.dictionaryIdsDecoder); + + if (vector.hasDictionary() || (rowId == 0 && supportLazyDecode())) { + // Column vector supports lazy decoding of dictionary values so just set the dictionary. + // We can't do this if rowId != 0 AND the column doesn't have a dictionary (i.e. some + // non-dictionary encoded values have already been added). + vector.setDictionary(new ParquetDictionary(dictionary)); + } else { + readBatchFromDictionaryIds(rowId, num, vector, dictionaryIds); + } + } else { + if (vector.hasDictionary() && rowId != 0) { + // This batch already has dictionary encoded values but this new page is not. The batch + // does not support a mix of dictionary and not so we will decode the dictionary. + readBatchFromDictionaryIds(0, rowId, vector, vector.getDictionaryIds()); + } + vector.setDictionary(null); + readBatch(rowId, num, vector); + } + + valuesRead += num; + rowId += num; + readNumber -= num; + } + } + + private void readPageV1(DataPageV1 page) throws IOException { + this.pageValueCount = page.getValueCount(); + ValuesReader rlReader = page.getRlEncoding().getValuesReader(descriptor, REPETITION_LEVEL); + + // Initialize the decoders. + if (page.getDlEncoding() != Encoding.RLE && descriptor.getMaxDefinitionLevel() != 0) { + throw new UnsupportedOperationException("Unsupported encoding: " + page.getDlEncoding()); + } + int bitWidth = BytesUtils.getWidthFromMaxInt(descriptor.getMaxDefinitionLevel()); + this.runLenDecoder = new RunLengthDecoder(bitWidth); + try { + BytesInput bytes = page.getBytes(); + ByteBufferInputStream in = bytes.toInputStream(); + rlReader.initFromPage(pageValueCount, in); + this.runLenDecoder.initFromStream(pageValueCount, in); + prepareNewPage(page.getValueEncoding(), in); + } catch (IOException e) { + throw new IOException("could not read page " + page + " in col " + descriptor, e); + } + } + + private void readPageV2(DataPageV2 page) throws IOException { + this.pageValueCount = page.getValueCount(); + + int bitWidth = BytesUtils.getWidthFromMaxInt(descriptor.getMaxDefinitionLevel()); + // do not read the length from the stream. v2 pages handle dividing the page bytes. + this.runLenDecoder = new RunLengthDecoder(bitWidth, false); + this.runLenDecoder.initFromStream( + this.pageValueCount, page.getDefinitionLevels().toInputStream()); + try { + prepareNewPage(page.getDataEncoding(), page.getData().toInputStream()); + } catch (IOException e) { + throw new IOException("could not read page " + page + " in col " + descriptor, e); + } + } + + private void prepareNewPage( + Encoding dataEncoding, + ByteBufferInputStream in) throws IOException { + this.endOfPageValueCount = valuesRead + pageValueCount; + if (dataEncoding.usesDictionary()) { + if (dictionary == null) { + throw new IOException("Could not read page in col " + + descriptor + + " as the dictionary was missing for encoding " + + dataEncoding); + } + @SuppressWarnings("deprecation") + Encoding plainDict = Encoding.PLAIN_DICTIONARY; // var to allow warning suppression + if (dataEncoding != plainDict && dataEncoding != Encoding.RLE_DICTIONARY) { + throw new UnsupportedOperationException("Unsupported encoding: " + dataEncoding); + } + this.dataInputStream = null; + this.dictionaryIdsDecoder = new RunLengthDecoder(); + try { + this.dictionaryIdsDecoder.initFromStream(pageValueCount, in); + } catch (IOException e) { + throw new IOException("could not read dictionary in col " + descriptor, e); + } + this.isCurrentPageDictionaryEncoded = true; + } else { + if (dataEncoding != Encoding.PLAIN) { + throw new UnsupportedOperationException("Unsupported encoding: " + dataEncoding); + } + this.dictionaryIdsDecoder = null; + LOG.debug("init from page at offset {} for length {}", in.position(), in.available()); + this.dataInputStream = in.remainingStream(); + this.isCurrentPageDictionaryEncoded = false; + } + + afterReadPage(); + } + + final ByteBuffer readDataBuffer(int length) { + try { + return dataInputStream.slice(length).order(ByteOrder.LITTLE_ENDIAN); + } catch (IOException e) { + throw new ParquetDecodingException("Failed to read " + length + " bytes", e); + } + } + + /** + * After read a page, we may need some initialization. + */ + protected void afterReadPage() { + } + + /** + * Support lazy dictionary ids decode. See more in {@link ParquetDictionary}. + * If return false, we will decode all the data first. + */ + protected boolean supportLazyDecode() { + return true; + } + + /** + * Read batch from {@link #runLenDecoder} and {@link #dataInputStream}. + */ + protected abstract void readBatch(int rowId, int num, V column); + + /** + * Decode dictionary ids to data. + * From {@link #runLenDecoder} and {@link #dictionaryIdsDecoder}. + */ + protected abstract void readBatchFromDictionaryIds( + int rowId, + int num, + V column, + WritableIntVector dictionaryIds); +} + diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java new file mode 100644 index 000000000000..6a8a01b74946 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java @@ -0,0 +1,473 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.hudi.table.format.cow.vector.HeapArrayVector; +import org.apache.hudi.table.format.cow.vector.ParquetDecimalVector; + +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch; +import org.apache.flink.table.data.columnar.vector.heap.HeapBooleanVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapByteVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapBytesVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapDoubleVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapFloatVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapIntVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapLongVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapShortVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapTimestampVector; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Array {@link ColumnReader}. + */ +public class ArrayColumnReader extends BaseVectorizedColumnReader { + + // The value read in last time + private Object lastValue; + + // flag to indicate if there is no data in parquet data page + private boolean eof = false; + + // flag to indicate if it's the first time to read parquet data page with this instance + boolean isFirstRow = true; + + public ArrayColumnReader( + ColumnDescriptor descriptor, + PageReader pageReader, + boolean isUtcTimestamp, + Type type, + LogicalType logicalType) + throws IOException { + super(descriptor, pageReader, isUtcTimestamp, type, logicalType); + } + + @Override + public void readToVector(int readNumber, WritableColumnVector vector) throws IOException { + HeapArrayVector lcv = (HeapArrayVector) vector; + // before readBatch, initial the size of offsets & lengths as the default value, + // the actual size will be assigned in setChildrenInfo() after reading complete. + lcv.offsets = new long[VectorizedColumnBatch.DEFAULT_SIZE]; + lcv.lengths = new long[VectorizedColumnBatch.DEFAULT_SIZE]; + // Because the length of ListColumnVector.child can't be known now, + // the valueList will save all data for ListColumnVector temporary. + List valueList = new ArrayList<>(); + + LogicalType category = ((ArrayType) logicalType).getElementType(); + + // read the first row in parquet data page, this will be only happened once for this + // instance + if (isFirstRow) { + if (!fetchNextValue(category)) { + return; + } + isFirstRow = false; + } + + int index = collectDataFromParquetPage(readNumber, lcv, valueList, category); + + // Convert valueList to array for the ListColumnVector.child + fillColumnVector(category, lcv, valueList, index); + } + + /** + * Reads a single value from parquet page, puts it into lastValue. Returns a boolean indicating + * if there is more values to read (true). + * + * @param category + * @return boolean + * @throws IOException + */ + private boolean fetchNextValue(LogicalType category) throws IOException { + int left = readPageIfNeed(); + if (left > 0) { + // get the values of repetition and definitionLevel + readRepetitionAndDefinitionLevels(); + // read the data if it isn't null + if (definitionLevel == maxDefLevel) { + if (isCurrentPageDictionaryEncoded) { + lastValue = dataColumn.readValueDictionaryId(); + } else { + lastValue = readPrimitiveTypedRow(category); + } + } else { + lastValue = null; + } + return true; + } else { + eof = true; + return false; + } + } + + private int readPageIfNeed() throws IOException { + // Compute the number of values we want to read in this page. + int leftInPage = (int) (endOfPageValueCount - valuesRead); + if (leftInPage == 0) { + // no data left in current page, load data from new page + readPage(); + leftInPage = (int) (endOfPageValueCount - valuesRead); + } + return leftInPage; + } + + // Need to be in consistent with that VectorizedPrimitiveColumnReader#readBatchHelper + // TODO Reduce the duplicated code + private Object readPrimitiveTypedRow(LogicalType category) { + switch (category.getTypeRoot()) { + case CHAR: + case VARCHAR: + case BINARY: + case VARBINARY: + return dataColumn.readString(); + case BOOLEAN: + return dataColumn.readBoolean(); + case TIME_WITHOUT_TIME_ZONE: + case DATE: + case INTEGER: + return dataColumn.readInteger(); + case TINYINT: + return dataColumn.readTinyInt(); + case SMALLINT: + return dataColumn.readSmallInt(); + case BIGINT: + return dataColumn.readLong(); + case FLOAT: + return dataColumn.readFloat(); + case DOUBLE: + return dataColumn.readDouble(); + case DECIMAL: + switch (descriptor.getPrimitiveType().getPrimitiveTypeName()) { + case INT32: + return dataColumn.readInteger(); + case INT64: + return dataColumn.readLong(); + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return dataColumn.readString(); + default: + throw new AssertionError(); + } + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return dataColumn.readTimestamp(); + default: + throw new RuntimeException("Unsupported type in the list: " + type); + } + } + + private Object dictionaryDecodeValue(LogicalType category, Integer dictionaryValue) { + if (dictionaryValue == null) { + return null; + } + + switch (category.getTypeRoot()) { + case CHAR: + case VARCHAR: + case BINARY: + case VARBINARY: + return dictionary.readString(dictionaryValue); + case DATE: + case TIME_WITHOUT_TIME_ZONE: + case INTEGER: + return dictionary.readInteger(dictionaryValue); + case BOOLEAN: + return dictionary.readBoolean(dictionaryValue) ? 1 : 0; + case DOUBLE: + return dictionary.readDouble(dictionaryValue); + case FLOAT: + return dictionary.readFloat(dictionaryValue); + case TINYINT: + return dictionary.readTinyInt(dictionaryValue); + case SMALLINT: + return dictionary.readSmallInt(dictionaryValue); + case BIGINT: + return dictionary.readLong(dictionaryValue); + case DECIMAL: + switch (descriptor.getPrimitiveType().getPrimitiveTypeName()) { + case INT32: + return dictionary.readInteger(dictionaryValue); + case INT64: + return dictionary.readLong(dictionaryValue); + case FIXED_LEN_BYTE_ARRAY: + case BINARY: + return dictionary.readString(dictionaryValue); + default: + throw new AssertionError(); + } + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return dictionary.readTimestamp(dictionaryValue); + default: + throw new RuntimeException("Unsupported type in the list: " + type); + } + } + + /** + * Collects data from a parquet page and returns the final row index where it stopped. The + * returned index can be equal to or less than total. + * + * @param total maximum number of rows to collect + * @param lcv column vector to do initial setup in data collection time + * @param valueList collection of values that will be fed into the vector later + * @param category + * @return int + * @throws IOException + */ + private int collectDataFromParquetPage( + int total, HeapArrayVector lcv, List valueList, LogicalType category) + throws IOException { + int index = 0; + /* + * Here is a nested loop for collecting all values from a parquet page. + * A column of array type can be considered as a list of lists, so the two loops are as below: + * 1. The outer loop iterates on rows (index is a row index, so points to a row in the batch), e.g.: + * [0, 2, 3] <- index: 0 + * [NULL, 3, 4] <- index: 1 + * + * 2. The inner loop iterates on values within a row (sets all data from parquet data page + * for an element in ListColumnVector), so fetchNextValue returns values one-by-one: + * 0, 2, 3, NULL, 3, 4 + * + * As described below, the repetition level (repetitionLevel != 0) + * can be used to decide when we'll start to read values for the next list. + */ + while (!eof && index < total) { + // add element to ListColumnVector one by one + lcv.offsets[index] = valueList.size(); + /* + * Let's collect all values for a single list. + * Repetition level = 0 means that a new list started there in the parquet page, + * in that case, let's exit from the loop, and start to collect value for a new list. + */ + do { + /* + * Definition level = 0 when a NULL value was returned instead of a list + * (this is not the same as a NULL value in of a list). + */ + if (definitionLevel == 0) { + lcv.setNullAt(index); + } + valueList.add( + isCurrentPageDictionaryEncoded + ? dictionaryDecodeValue(category, (Integer) lastValue) + : lastValue); + } while (fetchNextValue(category) && (repetitionLevel != 0)); + + lcv.lengths[index] = valueList.size() - lcv.offsets[index]; + index++; + } + return index; + } + + /** + * The lengths & offsets will be initialized as default size (1024), it should be set to the + * actual size according to the element number. + */ + private void setChildrenInfo(HeapArrayVector lcv, int itemNum, int elementNum) { + lcv.setSize(itemNum); + long[] lcvLength = new long[elementNum]; + long[] lcvOffset = new long[elementNum]; + System.arraycopy(lcv.lengths, 0, lcvLength, 0, elementNum); + System.arraycopy(lcv.offsets, 0, lcvOffset, 0, elementNum); + lcv.lengths = lcvLength; + lcv.offsets = lcvOffset; + } + + private void fillColumnVector( + LogicalType category, HeapArrayVector lcv, List valueList, int elementNum) { + int total = valueList.size(); + setChildrenInfo(lcv, total, elementNum); + switch (category.getTypeRoot()) { + case CHAR: + case VARCHAR: + case BINARY: + case VARBINARY: + lcv.child = new HeapBytesVector(total); + ((HeapBytesVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + byte[] src = ((List) valueList).get(i); + if (src == null) { + ((HeapBytesVector) lcv.child).setNullAt(i); + } else { + ((HeapBytesVector) lcv.child).appendBytes(i, src, 0, src.length); + } + } + break; + case BOOLEAN: + lcv.child = new HeapBooleanVector(total); + ((HeapBooleanVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapBooleanVector) lcv.child).setNullAt(i); + } else { + ((HeapBooleanVector) lcv.child).vector[i] = + ((List) valueList).get(i); + } + } + break; + case TINYINT: + lcv.child = new HeapByteVector(total); + ((HeapByteVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapByteVector) lcv.child).setNullAt(i); + } else { + ((HeapByteVector) lcv.child).vector[i] = + (byte) ((List) valueList).get(i).intValue(); + } + } + break; + case SMALLINT: + lcv.child = new HeapShortVector(total); + ((HeapShortVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapShortVector) lcv.child).setNullAt(i); + } else { + ((HeapShortVector) lcv.child).vector[i] = + (short) ((List) valueList).get(i).intValue(); + } + } + break; + case INTEGER: + case DATE: + case TIME_WITHOUT_TIME_ZONE: + lcv.child = new HeapIntVector(total); + ((HeapIntVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapIntVector) lcv.child).setNullAt(i); + } else { + ((HeapIntVector) lcv.child).vector[i] = ((List) valueList).get(i); + } + } + break; + case FLOAT: + lcv.child = new HeapFloatVector(total); + ((HeapFloatVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapFloatVector) lcv.child).setNullAt(i); + } else { + ((HeapFloatVector) lcv.child).vector[i] = ((List) valueList).get(i); + } + } + break; + case BIGINT: + lcv.child = new HeapLongVector(total); + ((HeapLongVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapLongVector) lcv.child).setNullAt(i); + } else { + ((HeapLongVector) lcv.child).vector[i] = ((List) valueList).get(i); + } + } + break; + case DOUBLE: + lcv.child = new HeapDoubleVector(total); + ((HeapDoubleVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapDoubleVector) lcv.child).setNullAt(i); + } else { + ((HeapDoubleVector) lcv.child).vector[i] = + ((List) valueList).get(i); + } + } + break; + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + lcv.child = new HeapTimestampVector(total); + ((HeapTimestampVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapTimestampVector) lcv.child).setNullAt(i); + } else { + ((HeapTimestampVector) lcv.child) + .setTimestamp(i, ((List) valueList).get(i)); + } + } + break; + case DECIMAL: + PrimitiveType.PrimitiveTypeName primitiveTypeName = + descriptor.getPrimitiveType().getPrimitiveTypeName(); + switch (primitiveTypeName) { + case INT32: + lcv.child = new ParquetDecimalVector(new HeapIntVector(total)); + ((HeapIntVector) ((ParquetDecimalVector) lcv.child).vector).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapIntVector) ((ParquetDecimalVector) lcv.child).vector) + .setNullAt(i); + } else { + ((HeapIntVector) ((ParquetDecimalVector) lcv.child).vector) + .vector[i] = + ((List) valueList).get(i); + } + } + break; + case INT64: + lcv.child = new ParquetDecimalVector(new HeapLongVector(total)); + ((HeapLongVector) ((ParquetDecimalVector) lcv.child).vector).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapLongVector) ((ParquetDecimalVector) lcv.child).vector) + .setNullAt(i); + } else { + ((HeapLongVector) ((ParquetDecimalVector) lcv.child).vector) + .vector[i] = + ((List) valueList).get(i); + } + } + break; + default: + lcv.child = new ParquetDecimalVector(new HeapBytesVector(total)); + ((HeapBytesVector) ((ParquetDecimalVector) lcv.child).vector).reset(); + for (int i = 0; i < valueList.size(); i++) { + byte[] src = ((List) valueList).get(i); + if (valueList.get(i) == null) { + ((HeapBytesVector) ((ParquetDecimalVector) lcv.child).vector) + .setNullAt(i); + } else { + ((HeapBytesVector) ((ParquetDecimalVector) lcv.child).vector) + .appendBytes(i, src, 0, src.length); + } + } + break; + } + break; + default: + throw new RuntimeException("Unsupported type in the list: " + type); + } + } +} + diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/BaseVectorizedColumnReader.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/BaseVectorizedColumnReader.java new file mode 100644 index 000000000000..fea6dc47af50 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/BaseVectorizedColumnReader.java @@ -0,0 +1,313 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.page.DataPage; +import org.apache.parquet.column.page.DataPageV1; +import org.apache.parquet.column.page.DataPageV2; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.values.ValuesReader; +import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder; +import org.apache.parquet.io.ParquetDecodingException; +import org.apache.parquet.schema.Type; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.IOException; + +import static org.apache.parquet.column.ValuesType.DEFINITION_LEVEL; +import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL; +import static org.apache.parquet.column.ValuesType.VALUES; + +/** + * Abstract {@link ColumnReader}. part of the code is referred from Apache Hive and Apache Parquet. + */ +public abstract class BaseVectorizedColumnReader implements ColumnReader { + + private static final Logger LOG = LoggerFactory.getLogger(BaseVectorizedColumnReader.class); + + protected boolean isUtcTimestamp; + + /** + * Total number of values read. + */ + protected long valuesRead; + + /** + * value that indicates the end of the current page. That is, if valuesRead == + * endOfPageValueCount, we are at the end of the page. + */ + protected long endOfPageValueCount; + + /** + * The dictionary, if this column has dictionary encoding. + */ + protected final ParquetDataColumnReader dictionary; + + /** + * If true, the current page is dictionary encoded. + */ + protected boolean isCurrentPageDictionaryEncoded; + + /** + * Maximum definition level for this column. + */ + protected final int maxDefLevel; + + protected int definitionLevel; + protected int repetitionLevel; + + /** + * Repetition/Definition/Value readers. + */ + protected IntIterator repetitionLevelColumn; + + protected IntIterator definitionLevelColumn; + protected ParquetDataColumnReader dataColumn; + + /** + * Total values in the current page. + */ + protected int pageValueCount; + + protected final PageReader pageReader; + protected final ColumnDescriptor descriptor; + protected final Type type; + protected final LogicalType logicalType; + + public BaseVectorizedColumnReader( + ColumnDescriptor descriptor, + PageReader pageReader, + boolean isUtcTimestamp, + Type parquetType, + LogicalType logicalType) + throws IOException { + this.descriptor = descriptor; + this.type = parquetType; + this.pageReader = pageReader; + this.maxDefLevel = descriptor.getMaxDefinitionLevel(); + this.isUtcTimestamp = isUtcTimestamp; + this.logicalType = logicalType; + + DictionaryPage dictionaryPage = pageReader.readDictionaryPage(); + if (dictionaryPage != null) { + try { + this.dictionary = + ParquetDataColumnReaderFactory.getDataColumnReaderByTypeOnDictionary( + parquetType.asPrimitiveType(), + dictionaryPage + .getEncoding() + .initDictionary(descriptor, dictionaryPage), + isUtcTimestamp); + this.isCurrentPageDictionaryEncoded = true; + } catch (IOException e) { + throw new IOException("could not decode the dictionary for " + descriptor, e); + } + } else { + this.dictionary = null; + this.isCurrentPageDictionaryEncoded = false; + } + } + + protected void readRepetitionAndDefinitionLevels() { + repetitionLevel = repetitionLevelColumn.nextInt(); + definitionLevel = definitionLevelColumn.nextInt(); + valuesRead++; + } + + protected void readPage() throws IOException { + DataPage page = pageReader.readPage(); + + if (page == null) { + return; + } + + page.accept( + new DataPage.Visitor() { + @Override + public Void visit(DataPageV1 dataPageV1) { + readPageV1(dataPageV1); + return null; + } + + @Override + public Void visit(DataPageV2 dataPageV2) { + readPageV2(dataPageV2); + return null; + } + }); + } + + private void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount) + throws IOException { + this.pageValueCount = valueCount; + this.endOfPageValueCount = valuesRead + pageValueCount; + if (dataEncoding.usesDictionary()) { + this.dataColumn = null; + if (dictionary == null) { + throw new IOException( + "could not read page in col " + + descriptor + + " as the dictionary was missing for encoding " + + dataEncoding); + } + dataColumn = + ParquetDataColumnReaderFactory.getDataColumnReaderByType( + type.asPrimitiveType(), + dataEncoding.getDictionaryBasedValuesReader( + descriptor, VALUES, dictionary.getDictionary()), + isUtcTimestamp); + this.isCurrentPageDictionaryEncoded = true; + } else { + dataColumn = + ParquetDataColumnReaderFactory.getDataColumnReaderByType( + type.asPrimitiveType(), + dataEncoding.getValuesReader(descriptor, VALUES), + isUtcTimestamp); + this.isCurrentPageDictionaryEncoded = false; + } + + try { + dataColumn.initFromPage(pageValueCount, in); + } catch (IOException e) { + throw new IOException("could not read page in col " + descriptor, e); + } + } + + private void readPageV1(DataPageV1 page) { + ValuesReader rlReader = page.getRlEncoding().getValuesReader(descriptor, REPETITION_LEVEL); + ValuesReader dlReader = page.getDlEncoding().getValuesReader(descriptor, DEFINITION_LEVEL); + this.repetitionLevelColumn = new ValuesReaderIntIterator(rlReader); + this.definitionLevelColumn = new ValuesReaderIntIterator(dlReader); + try { + BytesInput bytes = page.getBytes(); + LOG.debug("page size " + bytes.size() + " bytes and " + pageValueCount + " records"); + ByteBufferInputStream in = bytes.toInputStream(); + LOG.debug("reading repetition levels at " + in.position()); + rlReader.initFromPage(pageValueCount, in); + LOG.debug("reading definition levels at " + in.position()); + dlReader.initFromPage(pageValueCount, in); + LOG.debug("reading data at " + in.position()); + initDataReader(page.getValueEncoding(), in, page.getValueCount()); + } catch (IOException e) { + throw new ParquetDecodingException( + "could not read page " + page + " in col " + descriptor, e); + } + } + + private void readPageV2(DataPageV2 page) { + this.pageValueCount = page.getValueCount(); + this.repetitionLevelColumn = + newRLEIterator(descriptor.getMaxRepetitionLevel(), page.getRepetitionLevels()); + this.definitionLevelColumn = + newRLEIterator(descriptor.getMaxDefinitionLevel(), page.getDefinitionLevels()); + try { + LOG.debug( + "page data size " + + page.getData().size() + + " bytes and " + + pageValueCount + + " records"); + initDataReader( + page.getDataEncoding(), page.getData().toInputStream(), page.getValueCount()); + } catch (IOException e) { + throw new ParquetDecodingException( + "could not read page " + page + " in col " + descriptor, e); + } + } + + private IntIterator newRLEIterator(int maxLevel, BytesInput bytes) { + try { + if (maxLevel == 0) { + return new NullIntIterator(); + } + return new RLEIntIterator( + new RunLengthBitPackingHybridDecoder( + BytesUtils.getWidthFromMaxInt(maxLevel), + new ByteArrayInputStream(bytes.toByteArray()))); + } catch (IOException e) { + throw new ParquetDecodingException( + "could not read levels in page for col " + descriptor, e); + } + } + + /** + * Utility classes to abstract over different way to read ints with different encodings. + */ + abstract static class IntIterator { + abstract int nextInt(); + } + + /** + * read ints from {@link ValuesReader}. + */ + protected static final class ValuesReaderIntIterator extends IntIterator { + ValuesReader delegate; + + public ValuesReaderIntIterator(ValuesReader delegate) { + this.delegate = delegate; + } + + @Override + int nextInt() { + return delegate.readInteger(); + } + } + + /** + * read ints from {@link RunLengthBitPackingHybridDecoder}. + */ + protected static final class RLEIntIterator extends IntIterator { + RunLengthBitPackingHybridDecoder delegate; + + public RLEIntIterator(RunLengthBitPackingHybridDecoder delegate) { + this.delegate = delegate; + } + + @Override + int nextInt() { + try { + return delegate.readInt(); + } catch (IOException e) { + throw new ParquetDecodingException(e); + } + } + } + + /** + * return zero. + */ + protected static final class NullIntIterator extends IntIterator { + @Override + int nextInt() { + return 0; + } + } +} + diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/EmptyColumnReader.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/EmptyColumnReader.java new file mode 100644 index 000000000000..6ea610bf2af2 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/EmptyColumnReader.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; + +import java.io.IOException; + +/** + * Empty {@link ColumnReader}. + *

+ * This reader is to handle parquet files that have not been updated to the latest Schema. + * When reading a parquet file with the latest schema, parquet file might not have the new field. + * The EmptyColumnReader is used to handle such scenarios. + */ +public class EmptyColumnReader implements ColumnReader { + + public EmptyColumnReader() {} + + @Override + public void readToVector(int readNumber, WritableColumnVector vector) throws IOException { + vector.fillWithNulls(); + } +} + diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/FixedLenBytesColumnReader.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/FixedLenBytesColumnReader.java new file mode 100644 index 000000000000..be50e6c6239d --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/FixedLenBytesColumnReader.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.flink.table.data.columnar.vector.writable.WritableBytesVector; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; +import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Fixed length bytes {@code ColumnReader}, just for decimal. + * + *

Note: Reference Flink release 1.13.2 + * {@code org.apache.flink.formats.parquet.vector.reader.FixedLenBytesColumnReader} + * to always write as legacy decimal format. + */ +public class FixedLenBytesColumnReader + extends AbstractColumnReader { + + public FixedLenBytesColumnReader( + ColumnDescriptor descriptor, PageReader pageReader) throws IOException { + super(descriptor, pageReader); + checkTypeName(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY); + } + + @Override + protected void readBatch(int rowId, int num, V column) { + int bytesLen = descriptor.getPrimitiveType().getTypeLength(); + WritableBytesVector bytesVector = (WritableBytesVector) column; + for (int i = 0; i < num; i++) { + if (runLenDecoder.readInteger() == maxDefLevel) { + byte[] bytes = readDataBinary(bytesLen).getBytes(); + bytesVector.appendBytes(rowId + i, bytes, 0, bytes.length); + } else { + bytesVector.setNullAt(rowId + i); + } + } + } + + @Override + protected void readBatchFromDictionaryIds( + int rowId, int num, V column, WritableIntVector dictionaryIds) { + WritableBytesVector bytesVector = (WritableBytesVector) column; + for (int i = rowId; i < rowId + num; ++i) { + if (!bytesVector.isNullAt(i)) { + byte[] v = dictionary.decodeToBinary(dictionaryIds.getInt(i)).getBytes(); + bytesVector.appendBytes(i, v, 0, v.length); + } + } + } + + private Binary readDataBinary(int len) { + ByteBuffer buffer = readDataBuffer(len); + if (buffer.hasArray()) { + return Binary.fromConstantByteArray( + buffer.array(), buffer.arrayOffset() + buffer.position(), len); + } else { + byte[] bytes = new byte[len]; + buffer.get(bytes); + return Binary.fromConstantByteArray(bytes); + } + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java new file mode 100644 index 000000000000..b44273b57ca2 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector; +import org.apache.flink.table.data.columnar.vector.writable.WritableTimestampVector; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.temporal.ChronoUnit; + +/** + * Timestamp {@link org.apache.flink.formats.parquet.vector.reader.ColumnReader} that supports INT64 8 bytes, + * TIMESTAMP_MILLIS is the deprecated ConvertedType counterpart of a TIMESTAMP logical type + * that is UTC normalized and has MILLIS precision. + * + *

See https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp + * TIMESTAMP_MILLIS and TIMESTAMP_MICROS are the deprecated ConvertedType. + */ +public class Int64TimestampColumnReader extends AbstractColumnReader { + + private final boolean utcTimestamp; + + private final ChronoUnit chronoUnit; + + public Int64TimestampColumnReader( + boolean utcTimestamp, + ColumnDescriptor descriptor, + PageReader pageReader, + int precision) throws IOException { + super(descriptor, pageReader); + this.utcTimestamp = utcTimestamp; + if (precision <= 3) { + this.chronoUnit = ChronoUnit.MILLIS; + } else if (precision <= 6) { + this.chronoUnit = ChronoUnit.MICROS; + } else { + throw new IllegalArgumentException( + "Avro does not support TIMESTAMP type with precision: " + + precision + + ", it only support precisions <= 6."); + } + checkTypeName(PrimitiveType.PrimitiveTypeName.INT64); + } + + @Override + protected boolean supportLazyDecode() { + return false; + } + + @Override + protected void readBatch(int rowId, int num, WritableTimestampVector column) { + for (int i = 0; i < num; i++) { + if (runLenDecoder.readInteger() == maxDefLevel) { + ByteBuffer buffer = readDataBuffer(8); + column.setTimestamp(rowId + i, int64ToTimestamp(utcTimestamp, buffer.getLong(), chronoUnit)); + } else { + column.setNullAt(rowId + i); + } + } + } + + @Override + protected void readBatchFromDictionaryIds( + int rowId, + int num, + WritableTimestampVector column, + WritableIntVector dictionaryIds) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.setTimestamp(i, decodeInt64ToTimestamp( + utcTimestamp, dictionary, dictionaryIds.getInt(i), chronoUnit)); + } + } + } + + public static TimestampData decodeInt64ToTimestamp( + boolean utcTimestamp, + org.apache.parquet.column.Dictionary dictionary, + int id, + ChronoUnit unit) { + long value = dictionary.decodeToLong(id); + return int64ToTimestamp(utcTimestamp, value, unit); + } + + private static TimestampData int64ToTimestamp( + boolean utcTimestamp, + long interval, + ChronoUnit unit) { + final Instant instant = Instant.EPOCH.plus(interval, unit); + if (utcTimestamp) { + return TimestampData.fromInstant(instant); + } else { + // this applies the local timezone + return TimestampData.fromTimestamp(Timestamp.from(instant)); + } + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java new file mode 100644 index 000000000000..a6762d2e175c --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.hudi.table.format.cow.vector.HeapArrayVector; +import org.apache.hudi.table.format.cow.vector.HeapMapColumnVector; + +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.data.columnar.vector.ColumnVector; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; + +import java.io.IOException; + +/** + * Map {@link ColumnReader}. + */ +public class MapColumnReader implements ColumnReader { + + private final LogicalType logicalType; + private final ArrayColumnReader keyReader; + private final ArrayColumnReader valueReader; + + public MapColumnReader( + ArrayColumnReader keyReader, ArrayColumnReader valueReader, LogicalType logicalType) { + this.keyReader = keyReader; + this.valueReader = valueReader; + this.logicalType = logicalType; + } + + public void readBatch(int total, ColumnVector column) throws IOException { + HeapMapColumnVector mapColumnVector = (HeapMapColumnVector) column; + MapType mapType = (MapType) logicalType; + // initialize 2 ListColumnVector for keys and values + HeapArrayVector keyArrayColumnVector = new HeapArrayVector(total); + HeapArrayVector valueArrayColumnVector = new HeapArrayVector(total); + // read the keys and values + keyReader.readToVector(total, keyArrayColumnVector); + valueReader.readToVector(total, valueArrayColumnVector); + + // set the related attributes according to the keys and values + mapColumnVector.setKeys(keyArrayColumnVector.child); + mapColumnVector.setValues(valueArrayColumnVector.child); + mapColumnVector.setOffsets(keyArrayColumnVector.offsets); + mapColumnVector.setLengths(keyArrayColumnVector.lengths); + mapColumnVector.setSize(keyArrayColumnVector.getSize()); + for (int i = 0; i < keyArrayColumnVector.getLen(); i++) { + if (keyArrayColumnVector.isNullAt(i)) { + mapColumnVector.setNullAt(i); + } + } + } + + @Override + public void readToVector(int readNumber, WritableColumnVector vector) throws IOException { + readBatch(readNumber, vector); + } +} + diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java new file mode 100644 index 000000000000..65912cef671b --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java @@ -0,0 +1,390 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.hudi.table.format.cow.vector.ParquetDecimalVector; + +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.columnar.ColumnarRowData; +import org.apache.flink.table.data.columnar.vector.ColumnVector; +import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.filter.UnboundRecordFilter; +import org.apache.parquet.filter2.compat.FilterCompat; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.stream.IntStream; + +import static org.apache.hudi.table.format.cow.ParquetSplitReaderUtil.createColumnReader; +import static org.apache.hudi.table.format.cow.ParquetSplitReaderUtil.createWritableColumnVector; +import static org.apache.parquet.filter2.compat.FilterCompat.get; +import static org.apache.parquet.filter2.compat.RowGroupFilter.filterRowGroups; +import static org.apache.parquet.format.converter.ParquetMetadataConverter.range; +import static org.apache.parquet.hadoop.ParquetFileReader.readFooter; + +/** + * This reader is used to read a {@link VectorizedColumnBatch} from input split. + * + *

Note: Reference Flink release 1.11.2 + * {@code org.apache.flink.formats.parquet.vector.ParquetColumnarRowSplitReader} + * because it is package scope. + */ +public class ParquetColumnarRowSplitReader implements Closeable { + + private final boolean utcTimestamp; + + private final MessageType fileSchema; + + private final LogicalType[] requestedTypes; + + private final MessageType requestedSchema; + + /** + * The total number of rows this RecordReader will eventually read. The sum of the rows of all + * the row groups. + */ + private final long totalRowCount; + + private final WritableColumnVector[] writableVectors; + + private final VectorizedColumnBatch columnarBatch; + + private final ColumnarRowData row; + + private final int batchSize; + + private ParquetFileReader reader; + + /** + * For each request column, the reader to read this column. This is NULL if this column is + * missing from the file, in which case we populate the attribute with NULL. + */ + private ColumnReader[] columnReaders; + + /** + * The number of rows that have been returned. + */ + private long rowsReturned; + + /** + * The number of rows that have been reading, including the current in flight row group. + */ + private long totalCountLoadedSoFar; + + // the index of the next row to return + private int nextRow; + + // the number of rows in the current batch + private int rowsInBatch; + + public ParquetColumnarRowSplitReader( + boolean utcTimestamp, + boolean caseSensitive, + Configuration conf, + LogicalType[] selectedTypes, + String[] selectedFieldNames, + ColumnBatchGenerator generator, + int batchSize, + Path path, + long splitStart, + long splitLength, + FilterPredicate filterPredicate, + UnboundRecordFilter recordFilter) throws IOException { + this.utcTimestamp = utcTimestamp; + this.batchSize = batchSize; + // then we need to apply the predicate push down filter + ParquetMetadata footer = readFooter(conf, path, range(splitStart, splitStart + splitLength)); + MessageType fileSchema = footer.getFileMetaData().getSchema(); + FilterCompat.Filter filter = get(filterPredicate, recordFilter); + List blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema); + + this.fileSchema = footer.getFileMetaData().getSchema(); + + Type[] types = clipParquetSchema(fileSchema, selectedFieldNames, caseSensitive); + int[] requestedIndices = IntStream.range(0, types.length).filter(i -> types[i] != null).toArray(); + Type[] readTypes = Arrays.stream(requestedIndices).mapToObj(i -> types[i]).toArray(Type[]::new); + + this.requestedTypes = Arrays.stream(requestedIndices).mapToObj(i -> selectedTypes[i]).toArray(LogicalType[]::new); + this.requestedSchema = Types.buildMessage().addFields(readTypes).named("flink-parquet"); + this.reader = new ParquetFileReader( + conf, footer.getFileMetaData(), path, blocks, requestedSchema.getColumns()); + + long totalRowCount = 0; + for (BlockMetaData block : blocks) { + totalRowCount += block.getRowCount(); + } + this.totalRowCount = totalRowCount; + this.nextRow = 0; + this.rowsInBatch = 0; + this.rowsReturned = 0; + + checkSchema(); + + this.writableVectors = createWritableVectors(); + ColumnVector[] columnVectors = patchedVector(selectedFieldNames.length, createReadableVectors(), requestedIndices); + this.columnarBatch = generator.generate(columnVectors); + this.row = new ColumnarRowData(columnarBatch); + } + + /** + * Patches the given vectors with nulls. + * The vector position that is not requested (or read from file) is patched as null. + * + * @param fields The total selected fields number + * @param vectors The readable vectors + * @param indices The requested indices from the selected fields + */ + private static ColumnVector[] patchedVector(int fields, ColumnVector[] vectors, int[] indices) { + ColumnVector[] patched = new ColumnVector[fields]; + for (int i = 0; i < indices.length; i++) { + patched[indices[i]] = vectors[i]; + } + return patched; + } + + /** + * Clips `parquetSchema` according to `fieldNames`. + */ + private static Type[] clipParquetSchema( + GroupType parquetSchema, String[] fieldNames, boolean caseSensitive) { + Type[] types = new Type[fieldNames.length]; + if (caseSensitive) { + for (int i = 0; i < fieldNames.length; ++i) { + String fieldName = fieldNames[i]; + types[i] = parquetSchema.containsField(fieldName) ? parquetSchema.getType(fieldName) : null; + } + } else { + Map caseInsensitiveFieldMap = new HashMap<>(); + for (Type type : parquetSchema.getFields()) { + caseInsensitiveFieldMap.compute(type.getName().toLowerCase(Locale.ROOT), + (key, previousType) -> { + if (previousType != null) { + throw new FlinkRuntimeException( + "Parquet with case insensitive mode should have no duplicate key: " + key); + } + return type; + }); + } + for (int i = 0; i < fieldNames.length; ++i) { + Type type = caseInsensitiveFieldMap.get(fieldNames[i].toLowerCase(Locale.ROOT)); + // TODO clip for array,map,row types. + types[i] = type; + } + } + + return types; + } + + private WritableColumnVector[] createWritableVectors() { + WritableColumnVector[] columns = new WritableColumnVector[requestedTypes.length]; + List types = requestedSchema.getFields(); + List descriptors = requestedSchema.getColumns(); + for (int i = 0; i < requestedTypes.length; i++) { + columns[i] = createWritableColumnVector( + batchSize, + requestedTypes[i], + types.get(i), + descriptors); + } + return columns; + } + + /** + * Create readable vectors from writable vectors. + * Especially for decimal, see {@link org.apache.flink.formats.parquet.vector.ParquetDecimalVector}. + */ + private ColumnVector[] createReadableVectors() { + ColumnVector[] vectors = new ColumnVector[writableVectors.length]; + for (int i = 0; i < writableVectors.length; i++) { + vectors[i] = requestedTypes[i].getTypeRoot() == LogicalTypeRoot.DECIMAL + ? new ParquetDecimalVector(writableVectors[i]) + : writableVectors[i]; + } + return vectors; + } + + private void checkSchema() throws IOException, UnsupportedOperationException { + /* + * Check that the requested schema is supported. + */ + for (int i = 0; i < requestedSchema.getFieldCount(); ++i) { + String[] colPath = requestedSchema.getPaths().get(i); + if (fileSchema.containsPath(colPath)) { + ColumnDescriptor fd = fileSchema.getColumnDescription(colPath); + if (!fd.equals(requestedSchema.getColumns().get(i))) { + throw new UnsupportedOperationException("Schema evolution not supported."); + } + } else { + if (requestedSchema.getColumns().get(i).getMaxDefinitionLevel() == 0) { + // Column is missing in data but the required data is non-nullable. This file is invalid. + throw new IOException("Required column is missing in data file. Col: " + Arrays.toString(colPath)); + } + } + } + } + + /** + * Method used to check if the end of the input is reached. + * + * @return True if the end is reached, otherwise false. + * @throws IOException Thrown, if an I/O error occurred. + */ + public boolean reachedEnd() throws IOException { + return !ensureBatch(); + } + + public RowData nextRecord() { + // return the next row + row.setRowId(this.nextRow++); + return row; + } + + /** + * Checks if there is at least one row left in the batch to return. If no more row are + * available, it reads another batch of rows. + * + * @return Returns true if there is one more row to return, false otherwise. + * @throws IOException throw if an exception happens while reading a batch. + */ + private boolean ensureBatch() throws IOException { + if (nextRow >= rowsInBatch) { + // No more rows available in the Rows array. + nextRow = 0; + // Try to read the next batch if rows from the file. + return nextBatch(); + } + // there is at least one Row left in the Rows array. + return true; + } + + /** + * Advances to the next batch of rows. Returns false if there are no more. + */ + private boolean nextBatch() throws IOException { + for (WritableColumnVector v : writableVectors) { + v.reset(); + } + columnarBatch.setNumRows(0); + if (rowsReturned >= totalRowCount) { + return false; + } + if (rowsReturned == totalCountLoadedSoFar) { + readNextRowGroup(); + } + + int num = (int) Math.min(batchSize, totalCountLoadedSoFar - rowsReturned); + for (int i = 0; i < columnReaders.length; ++i) { + //noinspection unchecked + columnReaders[i].readToVector(num, writableVectors[i]); + } + rowsReturned += num; + columnarBatch.setNumRows(num); + rowsInBatch = num; + return true; + } + + private void readNextRowGroup() throws IOException { + PageReadStore pages = reader.readNextRowGroup(); + if (pages == null) { + throw new IOException("expecting more rows but reached last block. Read " + + rowsReturned + " out of " + totalRowCount); + } + List types = requestedSchema.getFields(); + List columns = requestedSchema.getColumns(); + columnReaders = new ColumnReader[types.size()]; + for (int i = 0; i < types.size(); ++i) { + columnReaders[i] = createColumnReader( + utcTimestamp, + requestedTypes[i], + types.get(i), + columns, + pages); + } + totalCountLoadedSoFar += pages.getRowCount(); + } + + /** + * Seek to a particular row number. + */ + public void seekToRow(long rowCount) throws IOException { + if (totalCountLoadedSoFar != 0) { + throw new UnsupportedOperationException("Only support seek at first."); + } + + List blockMetaData = reader.getRowGroups(); + + for (BlockMetaData metaData : blockMetaData) { + if (metaData.getRowCount() > rowCount) { + break; + } else { + reader.skipNextRowGroup(); + rowsReturned += metaData.getRowCount(); + totalCountLoadedSoFar += metaData.getRowCount(); + rowsInBatch = (int) metaData.getRowCount(); + nextRow = (int) metaData.getRowCount(); + rowCount -= metaData.getRowCount(); + } + } + for (int i = 0; i < rowCount; i++) { + boolean end = reachedEnd(); + if (end) { + throw new RuntimeException("Seek to many rows."); + } + nextRecord(); + } + } + + @Override + public void close() throws IOException { + if (reader != null) { + reader.close(); + reader = null; + } + } + + /** + * Interface to gen {@link VectorizedColumnBatch}. + */ + public interface ColumnBatchGenerator { + VectorizedColumnBatch generate(ColumnVector[] readVectors); + } +} + diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReader.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReader.java new file mode 100644 index 000000000000..e96cf22d29ef --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReader.java @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.flink.table.data.TimestampData; +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.apache.parquet.column.Dictionary; + +import java.io.IOException; + +/** + * The interface to wrap the underlying Parquet dictionary and non dictionary encoded page reader. + */ +public interface ParquetDataColumnReader { + + /** + * Initialize the reader by page data. + * + * @param valueCount value count + * @param in page data + * @throws IOException + */ + void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException; + + /** + * @return the next Dictionary ID from the page + */ + int readValueDictionaryId(); + + /** + * @return the next Long from the page + */ + long readLong(); + + /** + * @return the next Integer from the page + */ + int readInteger(); + + /** + * @return the next SmallInt from the page + */ + int readSmallInt(); + + /** + * @return the next TinyInt from the page + */ + int readTinyInt(); + + /** + * @return the next Float from the page + */ + float readFloat(); + + /** + * @return the next Boolean from the page + */ + boolean readBoolean(); + + /** + * @return the next String from the page + */ + byte[] readString(); + + /** + * @return the next Varchar from the page + */ + byte[] readVarchar(); + + /** + * @return the next Char from the page + */ + byte[] readChar(); + + /** + * @return the next Bytes from the page + */ + byte[] readBytes(); + + /** + * @return the next Decimal from the page + */ + byte[] readDecimal(); + + /** + * @return the next Double from the page + */ + double readDouble(); + + /** + * @return the next TimestampData from the page + */ + TimestampData readTimestamp(); + + /** + * @return is data valid + */ + boolean isValid(); + + /** + * @return the underlying dictionary if current reader is dictionary encoded + */ + Dictionary getDictionary(); + + /** + * @param id in dictionary + * @return the Bytes from the dictionary by id + */ + byte[] readBytes(int id); + + /** + * @param id in dictionary + * @return the Float from the dictionary by id + */ + float readFloat(int id); + + /** + * @param id in dictionary + * @return the Double from the dictionary by id + */ + double readDouble(int id); + + /** + * @param id in dictionary + * @return the Integer from the dictionary by id + */ + int readInteger(int id); + + /** + * @param id in dictionary + * @return the Long from the dictionary by id + */ + long readLong(int id); + + /** + * @param id in dictionary + * @return the Small Int from the dictionary by id + */ + int readSmallInt(int id); + + /** + * @param id in dictionary + * @return the tiny int from the dictionary by id + */ + int readTinyInt(int id); + + /** + * @param id in dictionary + * @return the Boolean from the dictionary by id + */ + boolean readBoolean(int id); + + /** + * @param id in dictionary + * @return the Decimal from the dictionary by id + */ + byte[] readDecimal(int id); + + /** + * @param id in dictionary + * @return the TimestampData from the dictionary by id + */ + TimestampData readTimestamp(int id); + + /** + * @param id in dictionary + * @return the String from the dictionary by id + */ + byte[] readString(int id); + + /** + * @param id in dictionary + * @return the Varchar from the dictionary by id + */ + byte[] readVarchar(int id); + + /** + * @param id in dictionary + * @return the Char from the dictionary by id + */ + byte[] readChar(int id); +} + diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReaderFactory.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReaderFactory.java new file mode 100644 index 000000000000..861d5cb00bbe --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReaderFactory.java @@ -0,0 +1,304 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.flink.table.data.TimestampData; +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.apache.parquet.column.Dictionary; +import org.apache.parquet.column.values.ValuesReader; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.sql.Timestamp; + +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.JULIAN_EPOCH_OFFSET_DAYS; +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.MILLIS_IN_DAY; +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.NANOS_PER_MILLISECOND; +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.NANOS_PER_SECOND; + +/** + * Parquet file has self-describing schema which may differ from the user required schema (e.g. + * schema evolution). This factory is used to retrieve user required typed data via corresponding + * reader which reads the underlying data. + */ +public final class ParquetDataColumnReaderFactory { + + private ParquetDataColumnReaderFactory() { + } + + /** + * default reader for {@link ParquetDataColumnReader}. + */ + public static class DefaultParquetDataColumnReader implements ParquetDataColumnReader { + protected ValuesReader valuesReader; + protected Dictionary dict; + + // After the data is read in the parquet type, isValid will be set to true if the data can + // be returned in the type defined in HMS. Otherwise isValid is set to false. + boolean isValid = true; + + public DefaultParquetDataColumnReader(ValuesReader valuesReader) { + this.valuesReader = valuesReader; + } + + public DefaultParquetDataColumnReader(Dictionary dict) { + this.dict = dict; + } + + @Override + public void initFromPage(int i, ByteBufferInputStream in) throws IOException { + valuesReader.initFromPage(i, in); + } + + @Override + public boolean readBoolean() { + return valuesReader.readBoolean(); + } + + @Override + public boolean readBoolean(int id) { + return dict.decodeToBoolean(id); + } + + @Override + public byte[] readString(int id) { + return dict.decodeToBinary(id).getBytesUnsafe(); + } + + @Override + public byte[] readString() { + return valuesReader.readBytes().getBytesUnsafe(); + } + + @Override + public byte[] readVarchar() { + // we need to enforce the size here even the types are the same + return valuesReader.readBytes().getBytesUnsafe(); + } + + @Override + public byte[] readVarchar(int id) { + return dict.decodeToBinary(id).getBytesUnsafe(); + } + + @Override + public byte[] readChar() { + return valuesReader.readBytes().getBytesUnsafe(); + } + + @Override + public byte[] readChar(int id) { + return dict.decodeToBinary(id).getBytesUnsafe(); + } + + @Override + public byte[] readBytes() { + return valuesReader.readBytes().getBytesUnsafe(); + } + + @Override + public byte[] readBytes(int id) { + return dict.decodeToBinary(id).getBytesUnsafe(); + } + + @Override + public byte[] readDecimal() { + return valuesReader.readBytes().getBytesUnsafe(); + } + + @Override + public byte[] readDecimal(int id) { + return dict.decodeToBinary(id).getBytesUnsafe(); + } + + @Override + public float readFloat() { + return valuesReader.readFloat(); + } + + @Override + public float readFloat(int id) { + return dict.decodeToFloat(id); + } + + @Override + public double readDouble() { + return valuesReader.readDouble(); + } + + @Override + public double readDouble(int id) { + return dict.decodeToDouble(id); + } + + @Override + public TimestampData readTimestamp() { + throw new RuntimeException("Unsupported operation"); + } + + @Override + public TimestampData readTimestamp(int id) { + throw new RuntimeException("Unsupported operation"); + } + + @Override + public int readInteger() { + return valuesReader.readInteger(); + } + + @Override + public int readInteger(int id) { + return dict.decodeToInt(id); + } + + @Override + public boolean isValid() { + return isValid; + } + + @Override + public long readLong(int id) { + return dict.decodeToLong(id); + } + + @Override + public long readLong() { + return valuesReader.readLong(); + } + + @Override + public int readSmallInt() { + return valuesReader.readInteger(); + } + + @Override + public int readSmallInt(int id) { + return dict.decodeToInt(id); + } + + @Override + public int readTinyInt() { + return valuesReader.readInteger(); + } + + @Override + public int readTinyInt(int id) { + return dict.decodeToInt(id); + } + + @Override + public int readValueDictionaryId() { + return valuesReader.readValueDictionaryId(); + } + + public void skip() { + valuesReader.skip(); + } + + @Override + public Dictionary getDictionary() { + return dict; + } + } + + /** + * The reader who reads from the underlying Timestamp value value. + */ + public static class TypesFromInt96PageReader extends DefaultParquetDataColumnReader { + private final boolean isUtcTimestamp; + + public TypesFromInt96PageReader(ValuesReader realReader, boolean isUtcTimestamp) { + super(realReader); + this.isUtcTimestamp = isUtcTimestamp; + } + + public TypesFromInt96PageReader(Dictionary dict, boolean isUtcTimestamp) { + super(dict); + this.isUtcTimestamp = isUtcTimestamp; + } + + private TimestampData convert(Binary binary) { + ByteBuffer buf = binary.toByteBuffer(); + buf.order(ByteOrder.LITTLE_ENDIAN); + long timeOfDayNanos = buf.getLong(); + int julianDay = buf.getInt(); + return int96ToTimestamp(isUtcTimestamp, timeOfDayNanos, julianDay); + } + + @Override + public TimestampData readTimestamp(int id) { + return convert(dict.decodeToBinary(id)); + } + + @Override + public TimestampData readTimestamp() { + return convert(valuesReader.readBytes()); + } + } + + private static ParquetDataColumnReader getDataColumnReaderByTypeHelper( + boolean isDictionary, + PrimitiveType parquetType, + Dictionary dictionary, + ValuesReader valuesReader, + boolean isUtcTimestamp) { + if (parquetType.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.INT96) { + return isDictionary + ? new TypesFromInt96PageReader(dictionary, isUtcTimestamp) + : new TypesFromInt96PageReader(valuesReader, isUtcTimestamp); + } else { + return isDictionary + ? new DefaultParquetDataColumnReader(dictionary) + : new DefaultParquetDataColumnReader(valuesReader); + } + } + + public static ParquetDataColumnReader getDataColumnReaderByTypeOnDictionary( + PrimitiveType parquetType, Dictionary realReader, boolean isUtcTimestamp) { + return getDataColumnReaderByTypeHelper(true, parquetType, realReader, null, isUtcTimestamp); + } + + public static ParquetDataColumnReader getDataColumnReaderByType( + PrimitiveType parquetType, ValuesReader realReader, boolean isUtcTimestamp) { + return getDataColumnReaderByTypeHelper( + false, parquetType, null, realReader, isUtcTimestamp); + } + + private static TimestampData int96ToTimestamp( + boolean utcTimestamp, long nanosOfDay, int julianDay) { + long millisecond = julianDayToMillis(julianDay) + (nanosOfDay / NANOS_PER_MILLISECOND); + + if (utcTimestamp) { + int nanoOfMillisecond = (int) (nanosOfDay % NANOS_PER_MILLISECOND); + return TimestampData.fromEpochMillis(millisecond, nanoOfMillisecond); + } else { + Timestamp timestamp = new Timestamp(millisecond); + timestamp.setNanos((int) (nanosOfDay % NANOS_PER_SECOND)); + return TimestampData.fromTimestamp(timestamp); + } + } + + private static long julianDayToMillis(int julianDay) { + return (julianDay - JULIAN_EPOCH_OFFSET_DAYS) * MILLIS_IN_DAY; + } +} + diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RowColumnReader.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RowColumnReader.java new file mode 100644 index 000000000000..79b50487f13c --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RowColumnReader.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.hudi.table.format.cow.vector.HeapRowColumnVector; + +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; + +import java.io.IOException; +import java.util.List; + +/** + * Row {@link ColumnReader}. + */ +public class RowColumnReader implements ColumnReader { + + private final List fieldReaders; + + public RowColumnReader(List fieldReaders) { + this.fieldReaders = fieldReaders; + } + + @Override + public void readToVector(int readNumber, WritableColumnVector vector) throws IOException { + HeapRowColumnVector rowColumnVector = (HeapRowColumnVector) vector; + WritableColumnVector[] vectors = rowColumnVector.vectors; + // row vector null array + boolean[] isNulls = new boolean[readNumber]; + for (int i = 0; i < vectors.length; i++) { + fieldReaders.get(i).readToVector(readNumber, vectors[i]); + + for (int j = 0; j < readNumber; j++) { + if (i == 0) { + isNulls[j] = vectors[i].isNullAt(j); + } else { + isNulls[j] = isNulls[j] && vectors[i].isNullAt(j); + } + if (i == vectors.length - 1 && isNulls[j]) { + // rowColumnVector[j] is null only when all fields[j] of rowColumnVector[j] is + // null + rowColumnVector.setNullAt(j); + } + } + } + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RunLengthDecoder.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RunLengthDecoder.java new file mode 100644 index 000000000000..4371ec30ae4c --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RunLengthDecoder.java @@ -0,0 +1,304 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector; +import org.apache.flink.table.data.columnar.vector.writable.WritableIntVector; +import org.apache.parquet.Preconditions; +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.column.values.bitpacking.BytePacker; +import org.apache.parquet.column.values.bitpacking.Packer; +import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder; +import org.apache.parquet.io.ParquetDecodingException; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Run length decoder for data and dictionary ids. + * See https://github.com/apache/parquet-format/blob/master/Encodings.md + * See {@link RunLengthBitPackingHybridDecoder}. + * + *

Note: Reference Flink release 1.11.2 + * {@code org.apache.flink.formats.parquet.vector.reader.RunLengthDecoder} + * because it is package scope. + */ +final class RunLengthDecoder { + + /** + * If true, the bit width is fixed. This decoder is used in different places and this also + * controls if we need to read the bitwidth from the beginning of the data stream. + */ + private final boolean fixedWidth; + private final boolean readLength; + + // Encoded data. + private ByteBufferInputStream in; + + // bit/byte width of decoded data and utility to batch unpack them. + private int bitWidth; + private int bytesWidth; + private BytePacker packer; + + // Current decoding mode and values + MODE mode; + int currentCount; + int currentValue; + + // Buffer of decoded values if the values are PACKED. + int[] currentBuffer = new int[16]; + int currentBufferIdx = 0; + + RunLengthDecoder() { + this.fixedWidth = false; + this.readLength = false; + } + + RunLengthDecoder(int bitWidth) { + this.fixedWidth = true; + this.readLength = bitWidth != 0; + initWidthAndPacker(bitWidth); + } + + RunLengthDecoder(int bitWidth, boolean readLength) { + this.fixedWidth = true; + this.readLength = readLength; + initWidthAndPacker(bitWidth); + } + + /** + * Init from input stream. + */ + void initFromStream(int valueCount, ByteBufferInputStream in) throws IOException { + this.in = in; + if (fixedWidth) { + // initialize for repetition and definition levels + if (readLength) { + int length = readIntLittleEndian(); + this.in = in.sliceStream(length); + } + } else { + // initialize for values + if (in.available() > 0) { + initWidthAndPacker(in.read()); + } + } + if (bitWidth == 0) { + // 0 bit width, treat this as an RLE run of valueCount number of 0's. + this.mode = MODE.RLE; + this.currentCount = valueCount; + this.currentValue = 0; + } else { + this.currentCount = 0; + } + } + + /** + * Initializes the internal state for decoding ints of `bitWidth`. + */ + private void initWidthAndPacker(int bitWidth) { + Preconditions.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32"); + this.bitWidth = bitWidth; + this.bytesWidth = BytesUtils.paddedByteCountFromBits(bitWidth); + this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth); + } + + int readInteger() { + if (this.currentCount == 0) { + this.readNextGroup(); + } + + this.currentCount--; + switch (mode) { + case RLE: + return this.currentValue; + case PACKED: + return this.currentBuffer[currentBufferIdx++]; + default: + throw new AssertionError(); + } + } + + /** + * Decoding for dictionary ids. The IDs are populated into `values` and the nullability is + * populated into `nulls`. + */ + void readDictionaryIds( + int total, + WritableIntVector values, + WritableColumnVector nulls, + int rowId, + int level, + RunLengthDecoder data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) { + this.readNextGroup(); + } + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readDictionaryIdData(n, values, rowId); + } else { + nulls.setNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + values.setInt(rowId + i, data.readInteger()); + } else { + nulls.setNullAt(rowId + i); + } + } + break; + default: + throw new AssertionError(); + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + /** + * It is used to decode dictionary IDs. + */ + private void readDictionaryIdData(int total, WritableIntVector c, int rowId) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) { + this.readNextGroup(); + } + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + c.setInts(rowId, n, currentValue); + break; + case PACKED: + c.setInts(rowId, n, currentBuffer, currentBufferIdx); + currentBufferIdx += n; + break; + default: + throw new AssertionError(); + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + /** + * Reads the next varint encoded int. + */ + private int readUnsignedVarInt() throws IOException { + int value = 0; + int shift = 0; + int b; + do { + b = in.read(); + value |= (b & 0x7F) << shift; + shift += 7; + } while ((b & 0x80) != 0); + return value; + } + + /** + * Reads the next 4 byte little endian int. + */ + private int readIntLittleEndian() throws IOException { + int ch4 = in.read(); + int ch3 = in.read(); + int ch2 = in.read(); + int ch1 = in.read(); + return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + ch4); + } + + /** + * Reads the next byteWidth little endian int. + */ + private int readIntLittleEndianPaddedOnBitWidth() throws IOException { + switch (bytesWidth) { + case 0: + return 0; + case 1: + return in.read(); + case 2: { + int ch2 = in.read(); + int ch1 = in.read(); + return (ch1 << 8) + ch2; + } + case 3: { + int ch3 = in.read(); + int ch2 = in.read(); + int ch1 = in.read(); + return (ch1 << 16) + (ch2 << 8) + ch3; + } + case 4: { + return readIntLittleEndian(); + } + default: + throw new RuntimeException("Unreachable"); + } + } + + /** + * Reads the next group. + */ + void readNextGroup() { + try { + int header = readUnsignedVarInt(); + this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED; + switch (mode) { + case RLE: + this.currentCount = header >>> 1; + this.currentValue = readIntLittleEndianPaddedOnBitWidth(); + return; + case PACKED: + int numGroups = header >>> 1; + this.currentCount = numGroups * 8; + + if (this.currentBuffer.length < this.currentCount) { + this.currentBuffer = new int[this.currentCount]; + } + currentBufferIdx = 0; + int valueIndex = 0; + while (valueIndex < this.currentCount) { + // values are bit packed 8 at a time, so reading bitWidth will always work + ByteBuffer buffer = in.slice(bitWidth); + this.packer.unpack8Values(buffer, buffer.position(), this.currentBuffer, valueIndex); + valueIndex += 8; + } + return; + default: + throw new ParquetDecodingException("not a valid mode " + this.mode); + } + } catch (IOException e) { + throw new ParquetDecodingException("Failed to read from input stream", e); + } + } + + enum MODE { + RLE, + PACKED + } +} + diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java new file mode 100644 index 000000000000..c0d83e6096e3 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; + +/** + * Adapter clazz for {@link Output}. + */ +public interface OutputAdapter extends Output { + @Override + default void emitWatermarkStatus(WatermarkStatus watermarkStatus) { + // no operation + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java new file mode 100644 index 000000000000..c903ec2ed408 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.hudi.adapter; + +import org.apache.flink.runtime.state.StateInitializationContext; + +import java.util.OptionalLong; + +/** + * Adapter clazz for {@link StateInitializationContext}. + */ +public interface StateInitializationContextAdapter extends StateInitializationContext { + default OptionalLong getRestoredCheckpointId() { + return OptionalLong.empty(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java new file mode 100644 index 000000000000..4461c28943d3 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.hudi.adapter; + +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.metrics.groups.OperatorMetricGroup; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; + +import java.util.Map; + +/** + * Adapter clazz for {@link StreamingRuntimeContext}. + */ +public class StreamingRuntimeContextAdapter extends StreamingRuntimeContext { + + public StreamingRuntimeContextAdapter(AbstractStreamOperator operator, Environment env, + Map> accumulators) { + super(operator, env, accumulators); + } + + @Override + public OperatorMetricGroup getMetricGroup() { + return UnregisteredMetricsGroup.createOperatorMetricGroup(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/TestStreamConfigs.java b/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/TestStreamConfigs.java new file mode 100644 index 000000000000..a7a620b4ec13 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/TestStreamConfigs.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.streaming.api.graph.StreamConfig; + +/** + * StreamConfig for test goals. + */ +public class TestStreamConfigs { + + public static void setupNetworkInputs(StreamConfig streamConfig, TypeSerializer... inputSerializers) { + streamConfig.setupNetworkInputs(inputSerializers); + // Since Flink 1.16, need call serializeAllConfigs to serialize all object configs synchronously. + // See https://issues.apache.org/jira/browse/FLINK-26675. + streamConfig.serializeAllConfigs(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java b/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java new file mode 100644 index 000000000000..e65437609a21 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.adapter; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +/** + * TableEnv for test goals. + */ +public class TestTableEnvs { + + public static TableEnvironment getBatchTableEnv() { + Configuration conf = new Configuration(); + // for batch upsert use cases: current suggestion is to disable these 2 options, + // from 1.14, flink runtime execution mode has switched from streaming + // to batch for batch execution mode(before that, both streaming and batch use streaming execution mode), + // current batch execution mode has these limitations: + // + // 1. the keyed stream default to always sort the inputs by key; + // 2. the batch state-backend requires the inputs sort by state key + // + // For our hudi batch pipeline upsert case, we rely on the consuming sequence for index records and data records, + // the index records must be loaded first before data records for BucketAssignFunction to keep upsert semantics correct, + // so we suggest disabling these 2 options to use streaming state-backend for batch execution mode + // to keep the strategy before 1.14. + conf.setBoolean("execution.sorted-inputs.enabled", false); + conf.setBoolean("execution.batch-state-backend.enabled", false); + StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(conf); + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + return StreamTableEnvironment.create(execEnv, settings); + } +} diff --git a/hudi-flink-datasource/pom.xml b/hudi-flink-datasource/pom.xml index c2a973ee1fd1..77124415d138 100644 --- a/hudi-flink-datasource/pom.xml +++ b/hudi-flink-datasource/pom.xml @@ -38,6 +38,7 @@ hudi-flink1.15.x hudi-flink1.16.x hudi-flink1.17.x + hudi-flink1.18.x hudi-flink diff --git a/packaging/bundle-validation/base/build_flink1170hive313spark332.sh b/packaging/bundle-validation/base/build_flink1180hive313spark332.sh similarity index 81% rename from packaging/bundle-validation/base/build_flink1170hive313spark332.sh rename to packaging/bundle-validation/base/build_flink1180hive313spark332.sh index ae4858afcabb..dca096a8d9b8 100755 --- a/packaging/bundle-validation/base/build_flink1170hive313spark332.sh +++ b/packaging/bundle-validation/base/build_flink1180hive313spark332.sh @@ -19,9 +19,9 @@ docker build \ --build-arg HIVE_VERSION=3.1.3 \ - --build-arg FLINK_VERSION=1.17.0 \ + --build-arg FLINK_VERSION=1.18.0 \ --build-arg SPARK_VERSION=3.3.2 \ --build-arg SPARK_HADOOP_VERSION=3 \ --build-arg HADOOP_VERSION=3.3.5 \ - -t hudi-ci-bundle-validation-base:flink1170hive313spark332 . -docker image tag hudi-ci-bundle-validation-base:flink1170hive313spark332 apachehudi/hudi-ci-bundle-validation-base:flink1170hive313spark332 + -t hudi-ci-bundle-validation-base:flink1180hive313spark332 . +docker image tag hudi-ci-bundle-validation-base:flink1180hive313spark332 apachehudi/hudi-ci-bundle-validation-base:flink1180hive313spark332 diff --git a/packaging/bundle-validation/base/build_flink1170hive313spark340.sh b/packaging/bundle-validation/base/build_flink1180hive313spark340.sh similarity index 81% rename from packaging/bundle-validation/base/build_flink1170hive313spark340.sh rename to packaging/bundle-validation/base/build_flink1180hive313spark340.sh index e59ccea7766f..2ceb9a81c58c 100755 --- a/packaging/bundle-validation/base/build_flink1170hive313spark340.sh +++ b/packaging/bundle-validation/base/build_flink1180hive313spark340.sh @@ -19,9 +19,9 @@ docker build \ --build-arg HIVE_VERSION=3.1.3 \ - --build-arg FLINK_VERSION=1.17.0 \ + --build-arg FLINK_VERSION=1.18.0 \ --build-arg SPARK_VERSION=3.4.0 \ --build-arg SPARK_HADOOP_VERSION=3 \ --build-arg HADOOP_VERSION=3.3.5 \ - -t hudi-ci-bundle-validation-base:flink1170hive313spark340 . -docker image tag hudi-ci-bundle-validation-base:flink1170hive313spark340 apachehudi/hudi-ci-bundle-validation-base:flink1170hive313spark340 + -t hudi-ci-bundle-validation-base:flink1180hive313spark340 . +docker image tag hudi-ci-bundle-validation-base:flink1180hive313spark340 apachehudi/hudi-ci-bundle-validation-base:flink1180hive313spark340 diff --git a/packaging/bundle-validation/ci_run.sh b/packaging/bundle-validation/ci_run.sh index bfdf9a1f661b..505ee9c7c2d4 100755 --- a/packaging/bundle-validation/ci_run.sh +++ b/packaging/bundle-validation/ci_run.sh @@ -162,6 +162,8 @@ else HUDI_FLINK_BUNDLE_NAME=hudi-flink1.16-bundle elif [[ ${FLINK_PROFILE} == 'flink1.17' ]]; then HUDI_FLINK_BUNDLE_NAME=hudi-flink1.17-bundle + elif [[ ${FLINK_PROFILE} == 'flink1.18' ]]; then + HUDI_FLINK_BUNDLE_NAME=hudi-flink1.18-bundle fi echo "Downloading bundle jars from staging repo orgapachehudi-$STAGING_REPO_NUM ..." diff --git a/pom.xml b/pom.xml index fc6b7f9a2243..04ea7a507b69 100644 --- a/pom.xml +++ b/pom.xml @@ -136,17 +136,19 @@ 2.4.4 3.4.1 + 1.18.0 1.17.1 1.16.2 1.15.1 1.14.5 1.13.6 - ${flink1.17.version} - hudi-flink1.17.x - 1.17 + ${flink1.18.version} + hudi-flink1.18.x + 1.18 1.11.1 - 1.12.2 + 1.13.1 + 3.0.0-1.17 flink-runtime flink-table-runtime flink-table-planner_2.12 @@ -1066,7 +1068,7 @@ org.apache.flink ${flink.connector.kafka.artifactId} - ${flink.version} + ${flink.connector.kafka.version} provided @@ -2525,11 +2527,29 @@ + + flink1.18 + + 1.5.6 + 1.11.1 + 1.13.1 + + + + flink1.18 + + + flink1.17 + ${flink1.17.version} + hudi-flink1.17.x + 1.17 1.5.6 1.11.1 + 1.12.3 + ${flink1.17.version} @@ -2545,6 +2565,8 @@ 1.16 1.5.6 1.11.1 + 1.12.2 + ${flink1.16.version} @@ -2560,6 +2582,8 @@ 1.15 1.5.6 1.11.1 + 1.12.2 + ${flink1.15.version} @@ -2584,6 +2608,8 @@ flink-clients_${scala.binary.version} flink-connector-kafka_${scala.binary.version} flink-hadoop-compatibility_${scala.binary.version} + 1.11.1 + ${flink1.14.version} @@ -2609,6 +2635,7 @@ flink-clients_${scala.binary.version} flink-connector-kafka_${scala.binary.version} flink-hadoop-compatibility_${scala.binary.version} + ${flink1.13.version} true diff --git a/scripts/release/deploy_staging_jars.sh b/scripts/release/deploy_staging_jars.sh index 221c3ddfede7..146e3fbdfdea 100755 --- a/scripts/release/deploy_staging_jars.sh +++ b/scripts/release/deploy_staging_jars.sh @@ -84,6 +84,7 @@ declare -a ALL_VERSION_OPTS=( "-Dscala-2.12 -Dflink1.15 -Davro.version=1.10.0 -pl packaging/hudi-flink-bundle -am" "-Dscala-2.12 -Dflink1.16 -Davro.version=1.11.1 -pl packaging/hudi-flink-bundle -am" "-Dscala-2.12 -Dflink1.17 -Davro.version=1.11.1 -pl packaging/hudi-flink-bundle -am" +"-Dscala-2.12 -Dflink1.18 -Davro.version=1.11.1 -pl packaging/hudi-flink-bundle -am" ) printf -v joined "'%s'\n" "${ALL_VERSION_OPTS[@]}" diff --git a/scripts/release/validate_staged_bundles.sh b/scripts/release/validate_staged_bundles.sh index 19db3b2fb48d..866b8cee335b 100755 --- a/scripts/release/validate_staged_bundles.sh +++ b/scripts/release/validate_staged_bundles.sh @@ -33,7 +33,7 @@ declare -a extensions=("-javadoc.jar" "-javadoc.jar.asc" "-javadoc.jar.md5" "-ja ".pom.md5" ".pom.sha1") declare -a bundles=("hudi-aws-bundle" "hudi-cli-bundle_2.11" "hudi-cli-bundle_2.12" "hudi-datahub-sync-bundle" "hudi-flink1.13-bundle" "hudi-flink1.14-bundle" -"hudi-flink1.15-bundle" "hudi-flink1.16-bundle" "hudi-flink1.17-bundle" "hudi-gcp-bundle" "hudi-hadoop-mr-bundle" "hudi-hive-sync-bundle" "hudi-integ-test-bundle" +"hudi-flink1.15-bundle" "hudi-flink1.16-bundle" "hudi-flink1.17-bundle" "hudi-flink1.18-bundle" "hudi-gcp-bundle" "hudi-hadoop-mr-bundle" "hudi-hive-sync-bundle" "hudi-integ-test-bundle" "hudi-kafka-connect-bundle" "hudi-metaserver-server-bundle" "hudi-presto-bundle" "hudi-spark-bundle_2.11" "hudi-spark-bundle_2.12" "hudi-spark2.4-bundle_2.11" "hudi-spark2.4-bundle_2.12" "hudi-spark3-bundle_2.12" "hudi-spark3.0-bundle_2.12" "hudi-spark3.1-bundle_2.12" "hudi-spark3.2-bundle_2.12" "hudi-spark3.3-bundle_2.12" "hudi-spark3.4-bundle_2.12" "hudi-timeline-server-bundle" "hudi-trino-bundle" From 4d6fd611bd3833045c418354194efeea5f1bacfe Mon Sep 17 00:00:00 2001 From: voonhous Date: Thu, 7 Dec 2023 12:01:08 +0800 Subject: [PATCH 2/4] [HUDI-7173] Fix hudi-on-flink read issues involving schema evolution and decimal types (#10247) --- .../format/cow/ParquetSplitReaderUtil.java | 28 ++++++------- .../format/cow/vector/HeapDecimalVector.java | 39 +++++++++++++++++++ 2 files changed, 53 insertions(+), 14 deletions(-) create mode 100644 hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapDecimalVector.java diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java index 9bf5390ee26c..1b636c63b2f6 100644 --- a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java @@ -20,9 +20,9 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.table.format.cow.vector.HeapArrayVector; +import org.apache.hudi.table.format.cow.vector.HeapDecimalVector; import org.apache.hudi.table.format.cow.vector.HeapMapColumnVector; import org.apache.hudi.table.format.cow.vector.HeapRowColumnVector; -import org.apache.hudi.table.format.cow.vector.ParquetDecimalVector; import org.apache.hudi.table.format.cow.vector.reader.ArrayColumnReader; import org.apache.hudi.table.format.cow.vector.reader.EmptyColumnReader; import org.apache.hudi.table.format.cow.vector.reader.FixedLenBytesColumnReader; @@ -65,7 +65,6 @@ import org.apache.flink.table.types.logical.MapType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.TimestampType; -import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.util.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.parquet.ParquetRuntimeException; @@ -234,17 +233,18 @@ private static ColumnVector createVectorFromConstant( } return lv; case DECIMAL: - DecimalType decimalType = (DecimalType) type; - int precision = decimalType.getPrecision(); - int scale = decimalType.getScale(); - DecimalData decimal = value == null - ? null - : Preconditions.checkNotNull(DecimalData.fromBigDecimal((BigDecimal) value, precision, scale)); - ColumnVector internalVector = createVectorFromConstant( - new VarBinaryType(), - decimal == null ? null : decimal.toUnscaledBytes(), - batchSize); - return new ParquetDecimalVector(internalVector); + HeapDecimalVector decv = new HeapDecimalVector(batchSize); + if (value == null) { + decv.fillWithNulls(); + } else { + DecimalType decimalType = (DecimalType) type; + int precision = decimalType.getPrecision(); + int scale = decimalType.getScale(); + DecimalData decimal = Preconditions.checkNotNull( + DecimalData.fromBigDecimal((BigDecimal) value, precision, scale)); + decv.fill(decimal.toUnscaledBytes()); + } + return decv; case FLOAT: HeapFloatVector fv = new HeapFloatVector(batchSize); if (value == null) { @@ -513,7 +513,7 @@ private static WritableColumnVector createWritableColumnVector( || typeName == PrimitiveType.PrimitiveTypeName.BINARY) && primitiveType.getOriginalType() == OriginalType.DECIMAL, "Unexpected type: %s", typeName); - return new HeapBytesVector(batchSize); + return new HeapDecimalVector(batchSize); case ARRAY: ArrayType arrayType = (ArrayType) fieldType; return new HeapArrayVector( diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapDecimalVector.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapDecimalVector.java new file mode 100644 index 000000000000..fdc55ac18fc6 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapDecimalVector.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.table.format.cow.vector; + +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.columnar.vector.DecimalColumnVector; +import org.apache.flink.table.data.columnar.vector.heap.HeapBytesVector; + +/** + * This class represents a nullable heap map decimal vector. + */ +public class HeapDecimalVector extends HeapBytesVector implements DecimalColumnVector { + + public HeapDecimalVector(int len) { + super(len); + } + + @Override + public DecimalData getDecimal(int i, int precision, int scale) { + return DecimalData.fromUnscaledBytes( + this.getBytes(i).getBytes(), precision, scale); + } +} From 7a12483a356aaf41a0b77c5db0c7b904908dcdc3 Mon Sep 17 00:00:00 2001 From: Peter Huang Date: Thu, 29 Aug 2024 14:11:16 -0700 Subject: [PATCH 3/4] change back to 0.14.2-SNAPSHOT --- docker/hoodie/hadoop/base/pom.xml | 2 +- docker/hoodie/hadoop/base_java11/pom.xml | 2 +- docker/hoodie/hadoop/datanode/pom.xml | 2 +- docker/hoodie/hadoop/historyserver/pom.xml | 2 +- docker/hoodie/hadoop/hive_base/pom.xml | 2 +- docker/hoodie/hadoop/namenode/pom.xml | 2 +- docker/hoodie/hadoop/pom.xml | 2 +- docker/hoodie/hadoop/prestobase/pom.xml | 2 +- docker/hoodie/hadoop/spark_base/pom.xml | 2 +- docker/hoodie/hadoop/sparkadhoc/pom.xml | 2 +- docker/hoodie/hadoop/sparkmaster/pom.xml | 2 +- docker/hoodie/hadoop/sparkworker/pom.xml | 2 +- docker/hoodie/hadoop/trinobase/pom.xml | 2 +- docker/hoodie/hadoop/trinocoordinator/pom.xml | 2 +- docker/hoodie/hadoop/trinoworker/pom.xml | 2 +- hudi-aws/pom.xml | 4 ++-- hudi-cli/pom.xml | 2 +- hudi-client/hudi-client-common/pom.xml | 4 ++-- hudi-client/hudi-flink-client/pom.xml | 4 ++-- hudi-client/hudi-java-client/pom.xml | 4 ++-- hudi-client/hudi-spark-client/pom.xml | 4 ++-- hudi-client/pom.xml | 2 +- hudi-common/pom.xml | 2 +- hudi-examples/hudi-examples-common/pom.xml | 2 +- hudi-examples/hudi-examples-flink/pom.xml | 2 +- hudi-examples/hudi-examples-java/pom.xml | 2 +- hudi-examples/hudi-examples-spark/pom.xml | 2 +- hudi-examples/pom.xml | 2 +- hudi-flink-datasource/hudi-flink/pom.xml | 4 ++-- hudi-flink-datasource/hudi-flink1.13.x/pom.xml | 4 ++-- hudi-flink-datasource/hudi-flink1.14.x/pom.xml | 4 ++-- hudi-flink-datasource/hudi-flink1.15.x/pom.xml | 4 ++-- hudi-flink-datasource/hudi-flink1.16.x/pom.xml | 4 ++-- hudi-flink-datasource/hudi-flink1.17.x/pom.xml | 4 ++-- hudi-flink-datasource/hudi-flink1.18.x/pom.xml | 4 ++-- hudi-flink-datasource/pom.xml | 4 ++-- hudi-gcp/pom.xml | 2 +- hudi-hadoop-mr/pom.xml | 2 +- hudi-integ-test/pom.xml | 2 +- hudi-kafka-connect/pom.xml | 4 ++-- .../hudi-metaserver/hudi-metaserver-client/pom.xml | 2 +- .../hudi-metaserver/hudi-metaserver-server/pom.xml | 2 +- hudi-platform-service/hudi-metaserver/pom.xml | 4 ++-- hudi-platform-service/pom.xml | 2 +- hudi-spark-datasource/hudi-spark-common/pom.xml | 4 ++-- hudi-spark-datasource/hudi-spark/pom.xml | 4 ++-- hudi-spark-datasource/hudi-spark2-common/pom.xml | 2 +- hudi-spark-datasource/hudi-spark2/pom.xml | 4 ++-- hudi-spark-datasource/hudi-spark3-common/pom.xml | 2 +- hudi-spark-datasource/hudi-spark3.0.x/pom.xml | 4 ++-- hudi-spark-datasource/hudi-spark3.1.x/pom.xml | 4 ++-- hudi-spark-datasource/hudi-spark3.2.x/pom.xml | 4 ++-- hudi-spark-datasource/hudi-spark3.2plus-common/pom.xml | 2 +- hudi-spark-datasource/hudi-spark3.3.x/pom.xml | 4 ++-- hudi-spark-datasource/hudi-spark3.4.x/pom.xml | 4 ++-- hudi-spark-datasource/pom.xml | 2 +- hudi-sync/hudi-adb-sync/pom.xml | 2 +- hudi-sync/hudi-datahub-sync/pom.xml | 2 +- hudi-sync/hudi-hive-sync/pom.xml | 2 +- hudi-sync/hudi-sync-common/pom.xml | 2 +- hudi-sync/pom.xml | 2 +- hudi-tests-common/pom.xml | 2 +- hudi-timeline-service/pom.xml | 2 +- hudi-utilities/pom.xml | 2 +- packaging/hudi-aws-bundle/pom.xml | 2 +- packaging/hudi-cli-bundle/pom.xml | 2 +- packaging/hudi-datahub-sync-bundle/pom.xml | 2 +- packaging/hudi-flink-bundle/pom.xml | 2 +- packaging/hudi-gcp-bundle/pom.xml | 2 +- packaging/hudi-hadoop-mr-bundle/pom.xml | 2 +- packaging/hudi-hive-sync-bundle/pom.xml | 2 +- packaging/hudi-integ-test-bundle/pom.xml | 2 +- packaging/hudi-kafka-connect-bundle/pom.xml | 2 +- packaging/hudi-metaserver-server-bundle/pom.xml | 2 +- packaging/hudi-presto-bundle/pom.xml | 2 +- packaging/hudi-spark-bundle/pom.xml | 2 +- packaging/hudi-timeline-server-bundle/pom.xml | 2 +- packaging/hudi-trino-bundle/pom.xml | 2 +- packaging/hudi-utilities-bundle/pom.xml | 2 +- packaging/hudi-utilities-slim-bundle/pom.xml | 2 +- pom.xml | 2 +- 81 files changed, 104 insertions(+), 104 deletions(-) diff --git a/docker/hoodie/hadoop/base/pom.xml b/docker/hoodie/hadoop/base/pom.xml index 8aaa7ba371e6..9695f427bda3 100644 --- a/docker/hoodie/hadoop/base/pom.xml +++ b/docker/hoodie/hadoop/base/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/base_java11/pom.xml b/docker/hoodie/hadoop/base_java11/pom.xml index ac6e72c72aeb..71d0b8350d69 100644 --- a/docker/hoodie/hadoop/base_java11/pom.xml +++ b/docker/hoodie/hadoop/base_java11/pom.xml @@ -20,7 +20,7 @@ hudi-hadoop-docker org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/datanode/pom.xml b/docker/hoodie/hadoop/datanode/pom.xml index 52b1d050993a..5e33a75e9658 100644 --- a/docker/hoodie/hadoop/datanode/pom.xml +++ b/docker/hoodie/hadoop/datanode/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/historyserver/pom.xml b/docker/hoodie/hadoop/historyserver/pom.xml index 9ee4d731aa5c..2ee437944b06 100644 --- a/docker/hoodie/hadoop/historyserver/pom.xml +++ b/docker/hoodie/hadoop/historyserver/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/hive_base/pom.xml b/docker/hoodie/hadoop/hive_base/pom.xml index eb5045d65a13..a99220464484 100644 --- a/docker/hoodie/hadoop/hive_base/pom.xml +++ b/docker/hoodie/hadoop/hive_base/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/namenode/pom.xml b/docker/hoodie/hadoop/namenode/pom.xml index 292a3409d626..aae8059cd11a 100644 --- a/docker/hoodie/hadoop/namenode/pom.xml +++ b/docker/hoodie/hadoop/namenode/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/pom.xml b/docker/hoodie/hadoop/pom.xml index e47d4ec5ecae..eeaaba75b8b1 100644 --- a/docker/hoodie/hadoop/pom.xml +++ b/docker/hoodie/hadoop/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../../pom.xml 4.0.0 diff --git a/docker/hoodie/hadoop/prestobase/pom.xml b/docker/hoodie/hadoop/prestobase/pom.xml index 2d399195c7bb..68c9f98f5b60 100644 --- a/docker/hoodie/hadoop/prestobase/pom.xml +++ b/docker/hoodie/hadoop/prestobase/pom.xml @@ -20,7 +20,7 @@ hudi-hadoop-docker org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/spark_base/pom.xml b/docker/hoodie/hadoop/spark_base/pom.xml index 5514aaa8fd47..07118f1460ba 100644 --- a/docker/hoodie/hadoop/spark_base/pom.xml +++ b/docker/hoodie/hadoop/spark_base/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkadhoc/pom.xml b/docker/hoodie/hadoop/sparkadhoc/pom.xml index 976384c66bd0..aef228bab84a 100644 --- a/docker/hoodie/hadoop/sparkadhoc/pom.xml +++ b/docker/hoodie/hadoop/sparkadhoc/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkmaster/pom.xml b/docker/hoodie/hadoop/sparkmaster/pom.xml index 16ff5629c7ce..35c8fef9cec1 100644 --- a/docker/hoodie/hadoop/sparkmaster/pom.xml +++ b/docker/hoodie/hadoop/sparkmaster/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkworker/pom.xml b/docker/hoodie/hadoop/sparkworker/pom.xml index 95e2b156e4a3..3f7cce36efe1 100644 --- a/docker/hoodie/hadoop/sparkworker/pom.xml +++ b/docker/hoodie/hadoop/sparkworker/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/trinobase/pom.xml b/docker/hoodie/hadoop/trinobase/pom.xml index f858d99866a2..2df42f24a168 100644 --- a/docker/hoodie/hadoop/trinobase/pom.xml +++ b/docker/hoodie/hadoop/trinobase/pom.xml @@ -22,7 +22,7 @@ hudi-hadoop-docker org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/trinocoordinator/pom.xml b/docker/hoodie/hadoop/trinocoordinator/pom.xml index 7e22f0f5b3f0..f0f10507de1a 100644 --- a/docker/hoodie/hadoop/trinocoordinator/pom.xml +++ b/docker/hoodie/hadoop/trinocoordinator/pom.xml @@ -22,7 +22,7 @@ hudi-hadoop-docker org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/trinoworker/pom.xml b/docker/hoodie/hadoop/trinoworker/pom.xml index 80f3ba5b2e0b..6a514ca1d6f8 100644 --- a/docker/hoodie/hadoop/trinoworker/pom.xml +++ b/docker/hoodie/hadoop/trinoworker/pom.xml @@ -22,7 +22,7 @@ hudi-hadoop-docker org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 pom diff --git a/hudi-aws/pom.xml b/hudi-aws/pom.xml index 7e95d364a780..3093134d3a09 100644 --- a/hudi-aws/pom.xml +++ b/hudi-aws/pom.xml @@ -19,12 +19,12 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-aws - 0.14.2-rc1 + 0.14.2-SNAPSHOT hudi-aws jar diff --git a/hudi-cli/pom.xml b/hudi-cli/pom.xml index 0dcbb2bee5e6..fcc9a63a8779 100644 --- a/hudi-cli/pom.xml +++ b/hudi-cli/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml index f00c5f1c4bf2..7a10ae12c35d 100644 --- a/hudi-client/hudi-client-common/pom.xml +++ b/hudi-client/hudi-client-common/pom.xml @@ -20,12 +20,12 @@ hudi-client org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-client-common - 0.14.2-rc1 + 0.14.2-SNAPSHOT hudi-client-common jar diff --git a/hudi-client/hudi-flink-client/pom.xml b/hudi-client/hudi-flink-client/pom.xml index 4638050460a8..460e723496a2 100644 --- a/hudi-client/hudi-flink-client/pom.xml +++ b/hudi-client/hudi-flink-client/pom.xml @@ -20,12 +20,12 @@ hudi-client org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-flink-client - 0.14.2-rc1 + 0.14.2-SNAPSHOT hudi-flink-client jar diff --git a/hudi-client/hudi-java-client/pom.xml b/hudi-client/hudi-java-client/pom.xml index 6f686bd30373..f66592688669 100644 --- a/hudi-client/hudi-java-client/pom.xml +++ b/hudi-client/hudi-java-client/pom.xml @@ -19,12 +19,12 @@ hudi-client org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-java-client - 0.14.2-rc1 + 0.14.2-SNAPSHOT hudi-java-client jar diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml index af126137c998..fa437494fd9f 100644 --- a/hudi-client/hudi-spark-client/pom.xml +++ b/hudi-client/hudi-spark-client/pom.xml @@ -19,12 +19,12 @@ hudi-client org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-spark-client - 0.14.2-rc1 + 0.14.2-SNAPSHOT hudi-spark-client jar diff --git a/hudi-client/pom.xml b/hudi-client/pom.xml index a73a1fd0124a..6498ddbd8c71 100644 --- a/hudi-client/pom.xml +++ b/hudi-client/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index c8089ec4567f..1d5cda0f75e8 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -20,7 +20,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-examples/hudi-examples-common/pom.xml b/hudi-examples/hudi-examples-common/pom.xml index 703537629b81..406449f90bfd 100644 --- a/hudi-examples/hudi-examples-common/pom.xml +++ b/hudi-examples/hudi-examples-common/pom.xml @@ -21,7 +21,7 @@ hudi-examples org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-examples/hudi-examples-flink/pom.xml b/hudi-examples/hudi-examples-flink/pom.xml index 4a3cb93418a0..e66540201771 100644 --- a/hudi-examples/hudi-examples-flink/pom.xml +++ b/hudi-examples/hudi-examples-flink/pom.xml @@ -21,7 +21,7 @@ hudi-examples org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-examples/hudi-examples-java/pom.xml b/hudi-examples/hudi-examples-java/pom.xml index 698564812024..d5ea102f4571 100644 --- a/hudi-examples/hudi-examples-java/pom.xml +++ b/hudi-examples/hudi-examples-java/pom.xml @@ -21,7 +21,7 @@ hudi-examples org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-examples/hudi-examples-spark/pom.xml b/hudi-examples/hudi-examples-spark/pom.xml index 7b1b161f3432..c21ec28e0a93 100644 --- a/hudi-examples/hudi-examples-spark/pom.xml +++ b/hudi-examples/hudi-examples-spark/pom.xml @@ -21,7 +21,7 @@ hudi-examples org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-examples/pom.xml b/hudi-examples/pom.xml index b87bd506302b..d8553639b026 100644 --- a/hudi-examples/pom.xml +++ b/hudi-examples/pom.xml @@ -20,7 +20,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-flink-datasource/hudi-flink/pom.xml b/hudi-flink-datasource/hudi-flink/pom.xml index c1247c16c08f..2f03b55322d4 100644 --- a/hudi-flink-datasource/hudi-flink/pom.xml +++ b/hudi-flink-datasource/hudi-flink/pom.xml @@ -22,12 +22,12 @@ hudi-flink-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-flink - 0.14.2-rc1 + 0.14.2-SNAPSHOT jar diff --git a/hudi-flink-datasource/hudi-flink1.13.x/pom.xml b/hudi-flink-datasource/hudi-flink1.13.x/pom.xml index cef3a87e41fe..5581c7a2b40b 100644 --- a/hudi-flink-datasource/hudi-flink1.13.x/pom.xml +++ b/hudi-flink-datasource/hudi-flink1.13.x/pom.xml @@ -20,12 +20,12 @@ hudi-flink-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-flink1.13.x - 0.14.2-rc1 + 0.14.2-SNAPSHOT jar diff --git a/hudi-flink-datasource/hudi-flink1.14.x/pom.xml b/hudi-flink-datasource/hudi-flink1.14.x/pom.xml index fee71f633f3b..6aad1fcdce48 100644 --- a/hudi-flink-datasource/hudi-flink1.14.x/pom.xml +++ b/hudi-flink-datasource/hudi-flink1.14.x/pom.xml @@ -20,12 +20,12 @@ hudi-flink-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-flink1.14.x - 0.14.2-rc1 + 0.14.2-SNAPSHOT jar diff --git a/hudi-flink-datasource/hudi-flink1.15.x/pom.xml b/hudi-flink-datasource/hudi-flink1.15.x/pom.xml index 7b6b1b28bbc0..a4777eaf732a 100644 --- a/hudi-flink-datasource/hudi-flink1.15.x/pom.xml +++ b/hudi-flink-datasource/hudi-flink1.15.x/pom.xml @@ -20,12 +20,12 @@ hudi-flink-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-flink1.15.x - 0.14.2-rc1 + 0.14.2-SNAPSHOT jar diff --git a/hudi-flink-datasource/hudi-flink1.16.x/pom.xml b/hudi-flink-datasource/hudi-flink1.16.x/pom.xml index 724b01770826..dd18efb45ca5 100644 --- a/hudi-flink-datasource/hudi-flink1.16.x/pom.xml +++ b/hudi-flink-datasource/hudi-flink1.16.x/pom.xml @@ -20,12 +20,12 @@ hudi-flink-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-flink1.16.x - 0.14.2-rc1 + 0.14.2-SNAPSHOT jar diff --git a/hudi-flink-datasource/hudi-flink1.17.x/pom.xml b/hudi-flink-datasource/hudi-flink1.17.x/pom.xml index 95ccb5389f22..9a87908527aa 100644 --- a/hudi-flink-datasource/hudi-flink1.17.x/pom.xml +++ b/hudi-flink-datasource/hudi-flink1.17.x/pom.xml @@ -20,12 +20,12 @@ hudi-flink-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-flink1.17.x - 0.14.2-rc1 + 0.14.2-SNAPSHOT jar diff --git a/hudi-flink-datasource/hudi-flink1.18.x/pom.xml b/hudi-flink-datasource/hudi-flink1.18.x/pom.xml index d391701cd033..dab044bb0821 100644 --- a/hudi-flink-datasource/hudi-flink1.18.x/pom.xml +++ b/hudi-flink-datasource/hudi-flink1.18.x/pom.xml @@ -20,12 +20,12 @@ hudi-flink-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-flink1.18.x - 0.14.2-rc1 + 0.14.2-SNAPSHOT jar diff --git a/hudi-flink-datasource/pom.xml b/hudi-flink-datasource/pom.xml index 77124415d138..e77a9405dc7a 100644 --- a/hudi-flink-datasource/pom.xml +++ b/hudi-flink-datasource/pom.xml @@ -20,12 +20,12 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-flink-datasource - 0.14.2-rc1 + 0.14.2-SNAPSHOT pom diff --git a/hudi-gcp/pom.xml b/hudi-gcp/pom.xml index a2cd110073a3..190a74848082 100644 --- a/hudi-gcp/pom.xml +++ b/hudi-gcp/pom.xml @@ -20,7 +20,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../pom.xml diff --git a/hudi-hadoop-mr/pom.xml b/hudi-hadoop-mr/pom.xml index 1c8b13b7e7f4..c09707fba0c9 100644 --- a/hudi-hadoop-mr/pom.xml +++ b/hudi-hadoop-mr/pom.xml @@ -20,7 +20,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-integ-test/pom.xml b/hudi-integ-test/pom.xml index c81b565d3b95..3610fc16ea2f 100644 --- a/hudi-integ-test/pom.xml +++ b/hudi-integ-test/pom.xml @@ -21,7 +21,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../pom.xml hudi-integ-test diff --git a/hudi-kafka-connect/pom.xml b/hudi-kafka-connect/pom.xml index 54febb0672b6..d7f1e2e4adb1 100644 --- a/hudi-kafka-connect/pom.xml +++ b/hudi-kafka-connect/pom.xml @@ -19,13 +19,13 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-kafka-connect Kafka Connect Sink Connector for Hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT jar diff --git a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/pom.xml b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/pom.xml index a23edca3f59e..e34c250ee5df 100644 --- a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/pom.xml +++ b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/pom.xml @@ -21,7 +21,7 @@ hudi-metaserver org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/pom.xml b/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/pom.xml index 14dbe66d503b..c9b526adc7b9 100644 --- a/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/pom.xml +++ b/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/pom.xml @@ -21,7 +21,7 @@ hudi-metaserver org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-platform-service/hudi-metaserver/pom.xml b/hudi-platform-service/hudi-metaserver/pom.xml index cc7fd95a0d35..b576e2c44dba 100644 --- a/hudi-platform-service/hudi-metaserver/pom.xml +++ b/hudi-platform-service/hudi-metaserver/pom.xml @@ -20,12 +20,12 @@ hudi-platform-service org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-metaserver - 0.14.2-rc1 + 0.14.2-SNAPSHOT hudi-metaserver pom diff --git a/hudi-platform-service/pom.xml b/hudi-platform-service/pom.xml index ec04df08bf57..b88122c69c47 100644 --- a/hudi-platform-service/pom.xml +++ b/hudi-platform-service/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-spark-datasource/hudi-spark-common/pom.xml b/hudi-spark-datasource/hudi-spark-common/pom.xml index d6747c9a2ba0..2bd4f049b4e6 100644 --- a/hudi-spark-datasource/hudi-spark-common/pom.xml +++ b/hudi-spark-datasource/hudi-spark-common/pom.xml @@ -17,12 +17,12 @@ hudi-spark-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-spark-common_${scala.binary.version} - 0.14.2-rc1 + 0.14.2-SNAPSHOT hudi-spark-common_${scala.binary.version} jar diff --git a/hudi-spark-datasource/hudi-spark/pom.xml b/hudi-spark-datasource/hudi-spark/pom.xml index a7bd0f25b475..9a2a9232619a 100644 --- a/hudi-spark-datasource/hudi-spark/pom.xml +++ b/hudi-spark-datasource/hudi-spark/pom.xml @@ -19,12 +19,12 @@ hudi-spark-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-spark_${scala.binary.version} - 0.14.2-rc1 + 0.14.2-SNAPSHOT hudi-spark_${scala.binary.version} jar diff --git a/hudi-spark-datasource/hudi-spark2-common/pom.xml b/hudi-spark-datasource/hudi-spark2-common/pom.xml index 1d0fa03b8c53..8f62805e8cc3 100644 --- a/hudi-spark-datasource/hudi-spark2-common/pom.xml +++ b/hudi-spark-datasource/hudi-spark2-common/pom.xml @@ -21,7 +21,7 @@ hudi-spark-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-spark-datasource/hudi-spark2/pom.xml b/hudi-spark-datasource/hudi-spark2/pom.xml index d88bf6bfde99..579c1cf8b87a 100644 --- a/hudi-spark-datasource/hudi-spark2/pom.xml +++ b/hudi-spark-datasource/hudi-spark2/pom.xml @@ -17,12 +17,12 @@ hudi-spark-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-spark2_${scala.binary.version} - 0.14.2-rc1 + 0.14.2-SNAPSHOT hudi-spark2_${scala.binary.version} jar diff --git a/hudi-spark-datasource/hudi-spark3-common/pom.xml b/hudi-spark-datasource/hudi-spark3-common/pom.xml index c5946a23120e..f13d917beb37 100644 --- a/hudi-spark-datasource/hudi-spark3-common/pom.xml +++ b/hudi-spark-datasource/hudi-spark3-common/pom.xml @@ -21,7 +21,7 @@ hudi-spark-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-spark-datasource/hudi-spark3.0.x/pom.xml b/hudi-spark-datasource/hudi-spark3.0.x/pom.xml index e8876fb357b8..276f2183cf8f 100644 --- a/hudi-spark-datasource/hudi-spark3.0.x/pom.xml +++ b/hudi-spark-datasource/hudi-spark3.0.x/pom.xml @@ -17,12 +17,12 @@ hudi-spark-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-spark3.0.x_2.12 - 0.14.2-rc1 + 0.14.2-SNAPSHOT hudi-spark3.0.x_2.12 jar diff --git a/hudi-spark-datasource/hudi-spark3.1.x/pom.xml b/hudi-spark-datasource/hudi-spark3.1.x/pom.xml index 20436fd1ec7e..b05d3fdea62d 100644 --- a/hudi-spark-datasource/hudi-spark3.1.x/pom.xml +++ b/hudi-spark-datasource/hudi-spark3.1.x/pom.xml @@ -17,12 +17,12 @@ hudi-spark-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-spark3.1.x_2.12 - 0.14.2-rc1 + 0.14.2-SNAPSHOT hudi-spark3.1.x_2.12 jar diff --git a/hudi-spark-datasource/hudi-spark3.2.x/pom.xml b/hudi-spark-datasource/hudi-spark3.2.x/pom.xml index b4e1b161f4c2..33a035f2b22d 100644 --- a/hudi-spark-datasource/hudi-spark3.2.x/pom.xml +++ b/hudi-spark-datasource/hudi-spark3.2.x/pom.xml @@ -18,12 +18,12 @@ hudi-spark-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-spark3.2.x_2.12 - 0.14.2-rc1 + 0.14.2-SNAPSHOT hudi-spark3.2.x_2.12 jar diff --git a/hudi-spark-datasource/hudi-spark3.2plus-common/pom.xml b/hudi-spark-datasource/hudi-spark3.2plus-common/pom.xml index 8fbe31fbfdae..6915ed70bb21 100644 --- a/hudi-spark-datasource/hudi-spark3.2plus-common/pom.xml +++ b/hudi-spark-datasource/hudi-spark3.2plus-common/pom.xml @@ -21,7 +21,7 @@ hudi-spark-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-spark-datasource/hudi-spark3.3.x/pom.xml b/hudi-spark-datasource/hudi-spark3.3.x/pom.xml index f9fd218124ca..267f54f587e8 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/pom.xml +++ b/hudi-spark-datasource/hudi-spark3.3.x/pom.xml @@ -17,12 +17,12 @@ hudi-spark-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-spark3.3.x_2.12 - 0.14.2-rc1 + 0.14.2-SNAPSHOT hudi-spark3.3.x_2.12 jar diff --git a/hudi-spark-datasource/hudi-spark3.4.x/pom.xml b/hudi-spark-datasource/hudi-spark3.4.x/pom.xml index 0eddbcddf180..0da2c0aa5b40 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/pom.xml +++ b/hudi-spark-datasource/hudi-spark3.4.x/pom.xml @@ -17,12 +17,12 @@ hudi-spark-datasource org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 hudi-spark3.4.x_2.12 - 0.14.2-rc1 + 0.14.2-SNAPSHOT hudi-spark3.4.x_2.12 jar diff --git a/hudi-spark-datasource/pom.xml b/hudi-spark-datasource/pom.xml index fb3353217830..e854a6fda64e 100644 --- a/hudi-spark-datasource/pom.xml +++ b/hudi-spark-datasource/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-sync/hudi-adb-sync/pom.xml b/hudi-sync/hudi-adb-sync/pom.xml index 7b27e71e33d2..256234de2937 100644 --- a/hudi-sync/hudi-adb-sync/pom.xml +++ b/hudi-sync/hudi-adb-sync/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml diff --git a/hudi-sync/hudi-datahub-sync/pom.xml b/hudi-sync/hudi-datahub-sync/pom.xml index 95669d93bdbb..9aeb25ad5b7e 100644 --- a/hudi-sync/hudi-datahub-sync/pom.xml +++ b/hudi-sync/hudi-datahub-sync/pom.xml @@ -24,7 +24,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml diff --git a/hudi-sync/hudi-hive-sync/pom.xml b/hudi-sync/hudi-hive-sync/pom.xml index 01f0391a8627..7f33b092f291 100644 --- a/hudi-sync/hudi-hive-sync/pom.xml +++ b/hudi-sync/hudi-hive-sync/pom.xml @@ -20,7 +20,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml diff --git a/hudi-sync/hudi-sync-common/pom.xml b/hudi-sync/hudi-sync-common/pom.xml index e5d53aeb64be..97cb60398850 100644 --- a/hudi-sync/hudi-sync-common/pom.xml +++ b/hudi-sync/hudi-sync-common/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/hudi-sync/pom.xml b/hudi-sync/pom.xml index 3b5af17b9d69..ca43aed2bee3 100644 --- a/hudi-sync/pom.xml +++ b/hudi-sync/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-tests-common/pom.xml b/hudi-tests-common/pom.xml index 9b27a318de0f..9afdf3e9f9ce 100644 --- a/hudi-tests-common/pom.xml +++ b/hudi-tests-common/pom.xml @@ -18,7 +18,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-timeline-service/pom.xml b/hudi-timeline-service/pom.xml index 117feeba1a50..f771f29154eb 100644 --- a/hudi-timeline-service/pom.xml +++ b/hudi-timeline-service/pom.xml @@ -20,7 +20,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml index fd776ceaf44c..c61cf64968c2 100644 --- a/hudi-utilities/pom.xml +++ b/hudi-utilities/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT 4.0.0 diff --git a/packaging/hudi-aws-bundle/pom.xml b/packaging/hudi-aws-bundle/pom.xml index 3837e4a1411e..8634d77a7bd2 100644 --- a/packaging/hudi-aws-bundle/pom.xml +++ b/packaging/hudi-aws-bundle/pom.xml @@ -24,7 +24,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-cli-bundle/pom.xml b/packaging/hudi-cli-bundle/pom.xml index 84eacfe061f7..6b0383b8f528 100644 --- a/packaging/hudi-cli-bundle/pom.xml +++ b/packaging/hudi-cli-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-datahub-sync-bundle/pom.xml b/packaging/hudi-datahub-sync-bundle/pom.xml index 3ce4eaf33419..b0daa2a10326 100644 --- a/packaging/hudi-datahub-sync-bundle/pom.xml +++ b/packaging/hudi-datahub-sync-bundle/pom.xml @@ -21,7 +21,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index 2f2330f71dfe..62e68daa9cd3 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -21,7 +21,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-gcp-bundle/pom.xml b/packaging/hudi-gcp-bundle/pom.xml index 6847d0310a36..7612b54ebba4 100644 --- a/packaging/hudi-gcp-bundle/pom.xml +++ b/packaging/hudi-gcp-bundle/pom.xml @@ -22,7 +22,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml index 3bf5d1cd37d3..45e60609fd3a 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-hive-sync-bundle/pom.xml b/packaging/hudi-hive-sync-bundle/pom.xml index ca70364c0906..0a9d54b5e83f 100644 --- a/packaging/hudi-hive-sync-bundle/pom.xml +++ b/packaging/hudi-hive-sync-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index 43341f6a30e2..ddd7f1aa346f 100644 --- a/packaging/hudi-integ-test-bundle/pom.xml +++ b/packaging/hudi-integ-test-bundle/pom.xml @@ -17,7 +17,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-kafka-connect-bundle/pom.xml b/packaging/hudi-kafka-connect-bundle/pom.xml index ecb71c969143..ce7983266fd6 100644 --- a/packaging/hudi-kafka-connect-bundle/pom.xml +++ b/packaging/hudi-kafka-connect-bundle/pom.xml @@ -20,7 +20,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-metaserver-server-bundle/pom.xml b/packaging/hudi-metaserver-server-bundle/pom.xml index 1ac5f3834d8a..05236261f45c 100644 --- a/packaging/hudi-metaserver-server-bundle/pom.xml +++ b/packaging/hudi-metaserver-server-bundle/pom.xml @@ -21,7 +21,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml index 2ceac795ba0e..eb78b7a1db35 100644 --- a/packaging/hudi-presto-bundle/pom.xml +++ b/packaging/hudi-presto-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index a784b1cec86d..e485bc03de2a 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-timeline-server-bundle/pom.xml b/packaging/hudi-timeline-server-bundle/pom.xml index 3cf71fd97cb9..dfdcab5f56e1 100644 --- a/packaging/hudi-timeline-server-bundle/pom.xml +++ b/packaging/hudi-timeline-server-bundle/pom.xml @@ -21,7 +21,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-trino-bundle/pom.xml b/packaging/hudi-trino-bundle/pom.xml index 53dabba155e7..7e2de2558b3c 100644 --- a/packaging/hudi-trino-bundle/pom.xml +++ b/packaging/hudi-trino-bundle/pom.xml @@ -20,7 +20,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 69a8df5dd768..43eb471d2af4 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-utilities-slim-bundle/pom.xml b/packaging/hudi-utilities-slim-bundle/pom.xml index 26b1103b73fc..a58770d60c5c 100644 --- a/packaging/hudi-utilities-slim-bundle/pom.xml +++ b/packaging/hudi-utilities-slim-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.14.2-rc1 + 0.14.2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/pom.xml b/pom.xml index 04ea7a507b69..4e9424e86654 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ org.apache.hudi hudi pom - 0.14.2-rc1 + 0.14.2-SNAPSHOT Apache Hudi brings stream style processing on big data https://github.com/apache/hudi Hudi From a76e837411c2ab2bbb7a49df5cef7b120c098416 Mon Sep 17 00:00:00 2001 From: Jon Vexler Date: Sat, 10 Aug 2024 00:54:48 -0400 Subject: [PATCH 4/4] [HUDI-8067] Use exec to run the IT (#11751) Co-authored-by: Jonathan Vexler <=> --- docker/setup_demo.sh | 6 ++--- docker/stop_demo.sh | 2 +- hudi-integ-test/pom.xml | 51 +++++++++++++++++++---------------------- 3 files changed, 28 insertions(+), 31 deletions(-) diff --git a/docker/setup_demo.sh b/docker/setup_demo.sh index e847f913a5ac..d183086d26c7 100755 --- a/docker/setup_demo.sh +++ b/docker/setup_demo.sh @@ -24,13 +24,13 @@ if [ "$HUDI_DEMO_ENV" = "--mac-aarch64" ]; then COMPOSE_FILE_NAME="docker-compose_hadoop284_hive233_spark244_mac_aarch64.yml" fi # restart cluster -HUDI_WS=${WS_ROOT} docker-compose -f ${SCRIPT_PATH}/compose/${COMPOSE_FILE_NAME} down +HUDI_WS=${WS_ROOT} docker compose down -f ${SCRIPT_PATH}/compose/${COMPOSE_FILE_NAME} if [ "$HUDI_DEMO_ENV" != "dev" ]; then echo "Pulling docker demo images ..." - HUDI_WS=${WS_ROOT} docker-compose -f ${SCRIPT_PATH}/compose/${COMPOSE_FILE_NAME} pull + HUDI_WS=${WS_ROOT} docker compose -f ${SCRIPT_PATH}/compose/${COMPOSE_FILE_NAME} pull fi sleep 5 -HUDI_WS=${WS_ROOT} docker-compose -f ${SCRIPT_PATH}/compose/${COMPOSE_FILE_NAME} up -d +HUDI_WS=${WS_ROOT} docker compose up -f ${SCRIPT_PATH}/compose/${COMPOSE_FILE_NAME} -d sleep 15 docker exec -it adhoc-1 /bin/bash /var/hoodie/ws/docker/demo/setup_demo_container.sh diff --git a/docker/stop_demo.sh b/docker/stop_demo.sh index 32a0e70c3791..25fb8d827349 100755 --- a/docker/stop_demo.sh +++ b/docker/stop_demo.sh @@ -25,7 +25,7 @@ if [ "$HUDI_DEMO_ENV" = "--mac-aarch64" ]; then COMPOSE_FILE_NAME="docker-compose_hadoop284_hive233_spark244_mac_aarch64.yml" fi # shut down cluster -HUDI_WS=${WS_ROOT} docker-compose -f ${SCRIPT_PATH}/compose/${COMPOSE_FILE_NAME} down +HUDI_WS=${WS_ROOT} docker compose down -f ${SCRIPT_PATH}/compose/${COMPOSE_FILE_NAME} # remove houst mount directory rm -rf /tmp/hadoop_data diff --git a/hudi-integ-test/pom.xml b/hudi-integ-test/pom.xml index 3610fc16ea2f..174ba397c780 100644 --- a/hudi-integ-test/pom.xml +++ b/hudi-integ-test/pom.xml @@ -444,53 +444,50 @@ ${dockerCompose.envFile} - - - - org.scalatest - scalatest-maven-plugin - - - org.scalastyle - scalastyle-maven-plugin - - - org.jacoco - jacoco-maven-plugin - - - com.dkanejs.maven.plugins - docker-compose-maven-plugin - 2.0.1 - up pre-integration-test - up + exec ${docker.compose.skip} - unix:///var/run/docker.sock - ${dockerCompose.file} - true - ${dockerCompose.envFile} + /bin/bash + + -c + export HUDI_WS=`dirname ${project.basedir}`; docker compose -f ${dockerCompose.file} up -d + down post-integration-test - down + exec ${docker.compose.skip} - ${dockerCompose.file} - true + /bin/bash + + -c + export HUDI_WS=`dirname ${project.basedir}`; docker compose -f ${dockerCompose.file} down -v + + + org.scalatest + scalatest-maven-plugin + + + org.scalastyle + scalastyle-maven-plugin + + + org.jacoco + jacoco-maven-plugin + org.apache.rat apache-rat-plugin