diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml index 1aaaba8eb6af..67c7ac16eaa2 100644 --- a/.github/workflows/bot.yml +++ b/.github/workflows/bot.yml @@ -98,7 +98,7 @@ jobs: SCALA_PROFILE: ${{ matrix.scalaProfile }} SPARK_PROFILE: ${{ matrix.sparkProfile }} run: - mvn clean install -T 2 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DskipTests=true $MVN_ARGS -am -pl "hudi-examples/hudi-examples-spark,hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" + mvn clean install -T 2 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DskipTests=true $MVN_ARGS -am -pl "hudi-examples/hudi-examples-spark,$SPARK_COMMON_MODULES,$SPARK_MODULES" - name: Quickstart Test env: SCALA_PROFILE: ${{ matrix.scalaProfile }} @@ -112,7 +112,7 @@ jobs: SPARK_MODULES: ${{ matrix.sparkModules }} if: ${{ !endsWith(env.SPARK_PROFILE, '3.2') }} # skip test spark 3.2 as it's covered by Azure CI run: - mvn test -Punit-tests -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -pl "hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS + mvn test -Punit-tests -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -pl "$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS - name: FT - Spark env: SCALA_PROFILE: ${{ matrix.scalaProfile }} @@ -221,7 +221,6 @@ jobs: strategy: matrix: include: - - flinkProfile: "flink1.13" - flinkProfile: "flink1.14" - flinkProfile: "flink1.15" - flinkProfile: "flink1.16" @@ -300,28 +299,22 @@ jobs: - 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' - flinkProfile: 'flink1.16' - sparkProfile: 'spark3.3' - sparkRuntime: 'spark3.3.2' - - flinkProfile: 'flink1.15' sparkProfile: 'spark3.3' sparkRuntime: 'spark3.3.1' - - flinkProfile: 'flink1.14' + - flinkProfile: 'flink1.15' sparkProfile: 'spark3.2' sparkRuntime: 'spark3.2.3' - - flinkProfile: 'flink1.13' + - flinkProfile: 'flink1.14' sparkProfile: 'spark3.1' sparkRuntime: 'spark3.1.3' - flinkProfile: 'flink1.14' sparkProfile: 'spark3.0' sparkRuntime: 'spark3.0.2' - - flinkProfile: 'flink1.13' + - flinkProfile: 'flink1.14' sparkProfile: 'spark2.4' sparkRuntime: 'spark2.4.8' steps: @@ -381,22 +374,34 @@ jobs: strategy: matrix: include: - - flinkProfile: 'flink1.16' + - flinkProfile: 'flink1.18' sparkProfile: 'spark3' + sparkRuntime: 'spark3.5.0' + - flinkProfile: 'flink1.18' + sparkProfile: 'spark3.5' + sparkRuntime: 'spark3.5.0' + - flinkProfile: 'flink1.18' + sparkProfile: 'spark3.4' + sparkRuntime: 'spark3.4.0' + - flinkProfile: 'flink1.17' + sparkProfile: 'spark3.3' sparkRuntime: 'spark3.3.2' - - flinkProfile: 'flink1.15' + - flinkProfile: 'flink1.16' sparkProfile: 'spark3.3' sparkRuntime: 'spark3.3.1' - - flinkProfile: 'flink1.14' + - flinkProfile: 'flink1.15' sparkProfile: 'spark3.2' sparkRuntime: 'spark3.2.3' - - flinkProfile: 'flink1.13' + - flinkProfile: 'flink1.14' sparkProfile: 'spark3.1' sparkRuntime: 'spark3.1.3' - - flinkProfile: 'flink1.13' + - flinkProfile: 'flink1.14' + sparkProfile: 'spark3.0' + sparkRuntime: 'spark3.0.2' + - flinkProfile: 'flink1.14' sparkProfile: 'spark' sparkRuntime: 'spark2.4.8' - - flinkProfile: 'flink1.13' + - flinkProfile: 'flink1.14' sparkProfile: 'spark2.4' sparkRuntime: 'spark2.4.8' steps: diff --git a/README.md b/README.md index 20016f689ad3..1122e121e69e 100644 --- a/README.md +++ b/README.md @@ -31,7 +31,7 @@ analytical datasets on DFS (Cloud stores, HDFS or any Hadoop FileSystem compatib [![Maven Central](https://maven-badges.herokuapp.com/maven-central/org.apache.hudi/hudi/badge.svg)](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.hudi%22) ![GitHub commit activity](https://img.shields.io/github/commit-activity/m/apache/hudi) [![Join on Slack](https://img.shields.io/badge/slack-%23hudi-72eff8?logo=slack&color=48c628&label=Join%20on%20Slack)](https://join.slack.com/t/apache-hudi/shared_invite/zt-1e94d3xro-JvlNO1kSeIHJBTVfLPlI5w) -![Twitter Follow](https://img.shields.io/twitter/follow/ApacheHudi) +[![Twitter Follow](https://img.shields.io/twitter/follow/ApacheHudi)](https://twitter.com/apachehudi) ## Features @@ -66,8 +66,8 @@ git clone https://github.com/apache/hudi.git && cd hudi mvn clean package -DskipTests # Start command -spark-3.2.3-bin-hadoop3.2/bin/spark-shell \ - --jars `ls packaging/hudi-spark-bundle/target/hudi-spark3.2-bundle_2.12-*.*.*-SNAPSHOT.jar` \ +spark-3.5.0-bin-hadoop3/bin/spark-shell \ + --jars `ls packaging/hudi-spark-bundle/target/hudi-spark3.5-bundle_2.12-*.*.*-SNAPSHOT.jar` \ --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer' \ --conf 'spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension' \ --conf 'spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog' \ @@ -85,7 +85,7 @@ mvn clean javadoc:aggregate -Pjavadocs ### Build with different Spark versions The default Spark 2.x version supported is 2.4.4. The default Spark 3.x version, corresponding to `spark3` profile is -3.4.0. The default Scala version is 2.12. Refer to the table below for building with different Spark and Scala versions. +3.5.0. The default Scala version is 2.12. Refer to the table below for building with different Spark and Scala versions. | Maven build options | Expected Spark bundle jar name | Notes | |:--------------------------|:---------------------------------------------|:-------------------------------------------------| @@ -96,9 +96,10 @@ The default Spark 2.x version supported is 2.4.4. The default Spark 3.x version, | `-Dspark3.2` | hudi-spark3.2-bundle_2.12 | For Spark 3.2.x and Scala 2.12 (same as default) | | `-Dspark3.3` | hudi-spark3.3-bundle_2.12 | For Spark 3.3.x and Scala 2.12 | | `-Dspark3.4` | hudi-spark3.4-bundle_2.12 | For Spark 3.4.x and Scala 2.12 | +| `-Dspark3.5` | hudi-spark3.5-bundle_2.12 | For Spark 3.5.x and Scala 2.12 | | `-Dspark2 -Dscala-2.11` | hudi-spark-bundle_2.11 (legacy bundle name) | For Spark 2.4.4 and Scala 2.11 | | `-Dspark2 -Dscala-2.12` | hudi-spark-bundle_2.12 (legacy bundle name) | For Spark 2.4.4 and Scala 2.12 | -| `-Dspark3` | hudi-spark3-bundle_2.12 (legacy bundle name) | For Spark 3.4.x and Scala 2.12 | +| `-Dspark3` | hudi-spark3-bundle_2.12 (legacy bundle name) | For Spark 3.5.x and Scala 2.12 | For example, ``` @@ -131,8 +132,6 @@ Refer to the table below for building with different Flink and Scala versions. | `-Dflink1.15` | hudi-flink1.15-bundle | For Flink 1.15 | | `-Dflink1.14` | hudi-flink1.14-bundle | For Flink 1.14 and Scala 2.12 | | `-Dflink1.14 -Dscala-2.11` | hudi-flink1.14-bundle | For Flink 1.14 and Scala 2.11 | -| `-Dflink1.13` | hudi-flink1.13-bundle | For Flink 1.13 and Scala 2.12 | -| `-Dflink1.13 -Dscala-2.11` | hudi-flink1.13-bundle | For Flink 1.13 and Scala 2.11 | For example, ``` @@ -141,9 +140,6 @@ mvn clean package -DskipTests -Dflink1.15 # Build against Flink 1.14.x and Scala 2.11 mvn clean package -DskipTests -Dflink1.14 -Dscala-2.11 - -# Build against Flink 1.13.x and Scala 2.12 -mvn clean package -DskipTests -Dflink1.13 ``` ## Running Tests diff --git a/azure-pipelines-20230430.yml b/azure-pipelines-20230430.yml index 85d185fbc2c5..c2a5f9d5a44b 100644 --- a/azure-pipelines-20230430.yml +++ b/azure-pipelines-20230430.yml @@ -32,7 +32,6 @@ parameters: - 'hudi-common' - 'hudi-flink-datasource' - 'hudi-flink-datasource/hudi-flink' - - 'hudi-flink-datasource/hudi-flink1.13.x' - 'hudi-flink-datasource/hudi-flink1.14.x' - 'hudi-flink-datasource/hudi-flink1.15.x' - 'hudi-flink-datasource/hudi-flink1.16.x' @@ -42,6 +41,7 @@ parameters: type: object default: - 'hudi-client/hudi-spark-client' + - 'hudi-spark-datasource/hudi-spark' - name: job3UTModules type: object default: @@ -65,7 +65,6 @@ parameters: - '!hudi-examples/hudi-examples-spark' - '!hudi-flink-datasource' - '!hudi-flink-datasource/hudi-flink' - - '!hudi-flink-datasource/hudi-flink1.13.x' - '!hudi-flink-datasource/hudi-flink1.14.x' - '!hudi-flink-datasource/hudi-flink1.15.x' - '!hudi-flink-datasource/hudi-flink1.16.x' @@ -89,12 +88,12 @@ parameters: - '!hudi-examples/hudi-examples-spark' - '!hudi-flink-datasource' - '!hudi-flink-datasource/hudi-flink' - - '!hudi-flink-datasource/hudi-flink1.13.x' - '!hudi-flink-datasource/hudi-flink1.14.x' - '!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' variables: BUILD_PROFILES: '-Dscala-2.12 -Dspark3.2 -Dflink1.18' @@ -144,7 +143,7 @@ stages: grep "testcase" */target/surefire-reports/*.xml */*/target/surefire-reports/*.xml | awk -F'"' ' { print $6,$4,$2 } ' | sort -nr | head -n 100 displayName: Top 100 long-running testcases - job: UT_FT_2 - displayName: FT client/spark-client + displayName: FT client/spark-client & hudi-spark-datasource/hudi-spark timeoutInMinutes: '150' steps: - task: Maven@4 @@ -156,7 +155,7 @@ stages: publishJUnitResults: false jdkVersionOption: '1.8' - task: Maven@4 - displayName: FT client/spark-client + displayName: FT client/spark-client & hudi-spark-datasource/hudi-spark inputs: mavenPomFile: 'pom.xml' goals: 'test' diff --git a/hudi-aws/src/test/java/org/apache/hudi/aws/TestHoodieAWSCredentialsProviderFactory.java b/hudi-aws/src/test/java/org/apache/hudi/aws/TestHoodieAWSCredentialsProviderFactory.java index d65f32109c12..6bbf4041d25a 100644 --- a/hudi-aws/src/test/java/org/apache/hudi/aws/TestHoodieAWSCredentialsProviderFactory.java +++ b/hudi-aws/src/test/java/org/apache/hudi/aws/TestHoodieAWSCredentialsProviderFactory.java @@ -18,9 +18,10 @@ package org.apache.hudi.aws; -import org.apache.hudi.config.HoodieAWSConfig; import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.config.HoodieAWSConfig; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; @@ -40,6 +41,7 @@ public void testGetAWSCredentials() { assertEquals("random-session-token", credentials.sessionToken()); } + @Disabled("HUDI-7114") @Test public void testGetAWSCredentialsWithInvalidAssumeRole() { // This test is to ensure that the AWS credentials provider factory fallbacks to default credentials diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java index 533635ad774a..19faa8f4dbc0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java @@ -120,7 +120,7 @@ private synchronized void stopEmbeddedServerView(boolean resetViewStorageConfig) if (timelineServer.isPresent() && shouldStopTimelineServer) { // Stop only if owner LOG.info("Stopping Timeline service !!"); - timelineServer.get().stop(); + timelineServer.get().stopForBasePath(basePath); } timelineServer = Option.empty(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java index 112b81110048..2ba0d553f58c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java @@ -57,6 +57,7 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieLogCompactException; import org.apache.hudi.exception.HoodieRollbackException; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -783,9 +784,14 @@ protected void archive(HoodieTable table) { return; } try { + final Timer.Context timerContext = metrics.getArchiveCtx(); // We cannot have unbounded commit files. Archive commits if we have to archive HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(config, table); - archiver.archiveIfRequired(context, true); + int instantsToArchive = archiver.archiveIfRequired(context, true); + if (timerContext != null) { + long durationMs = metrics.getDurationInMs(timerContext.stop()); + this.metrics.updateArchiveMetrics(durationMs, instantsToArchive); + } } catch (IOException ioe) { throw new HoodieIOException("Failed to archive", ioe); } @@ -1083,6 +1089,9 @@ public void rollbackFailedBootstrap() { table.rollbackBootstrap(context, createNewInstantTime()); LOG.info("Finished rolling back pending bootstrap"); } + + // if bootstrap failed, lets delete metadata and restart from scratch + HoodieTableMetadataUtil.deleteMetadataTable(config.getBasePath(), context); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineServerHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineServerHelper.java index b5f67fadec4c..47e1b9ee459f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineServerHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineServerHelper.java @@ -23,9 +23,6 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; /** @@ -33,10 +30,6 @@ */ public class EmbeddedTimelineServerHelper { - private static final Logger LOG = LoggerFactory.getLogger(EmbeddedTimelineService.class); - - private static Option TIMELINE_SERVER = Option.empty(); - /** * Instantiate Embedded Timeline Server. * @param context Hoodie Engine Context @@ -44,45 +37,28 @@ public class EmbeddedTimelineServerHelper { * @return TimelineServer if configured to run * @throws IOException */ - public static synchronized Option createEmbeddedTimelineService( + public static Option createEmbeddedTimelineService( HoodieEngineContext context, HoodieWriteConfig config) throws IOException { - if (config.isEmbeddedTimelineServerReuseEnabled()) { - if (!TIMELINE_SERVER.isPresent() || !TIMELINE_SERVER.get().canReuseFor(config.getBasePath())) { - TIMELINE_SERVER = Option.of(startTimelineService(context, config)); - } else { - updateWriteConfigWithTimelineServer(TIMELINE_SERVER.get(), config); - } - return TIMELINE_SERVER; - } if (config.isEmbeddedTimelineServerEnabled()) { - return Option.of(startTimelineService(context, config)); + Option hostAddr = context.getProperty(EngineProperty.EMBEDDED_SERVER_HOST); + EmbeddedTimelineService timelineService = EmbeddedTimelineService.getOrStartEmbeddedTimelineService(context, hostAddr.orElse(null), config); + updateWriteConfigWithTimelineServer(timelineService, config); + return Option.of(timelineService); } else { return Option.empty(); } } - private static EmbeddedTimelineService startTimelineService( - HoodieEngineContext context, HoodieWriteConfig config) throws IOException { - // Run Embedded Timeline Server - LOG.info("Starting Timeline service !!"); - Option hostAddr = context.getProperty(EngineProperty.EMBEDDED_SERVER_HOST); - EmbeddedTimelineService timelineService = new EmbeddedTimelineService( - context, hostAddr.orElse(null), config); - timelineService.startServer(); - updateWriteConfigWithTimelineServer(timelineService, config); - return timelineService; - } - /** * Adjusts hoodie write config with timeline server settings. * @param timelineServer Embedded Timeline Server * @param config Hoodie Write Config */ public static void updateWriteConfigWithTimelineServer(EmbeddedTimelineService timelineServer, - HoodieWriteConfig config) { + HoodieWriteConfig config) { // Allow executor to find this newly instantiated timeline service if (config.isEmbeddedTimelineServerEnabled()) { config.setViewStorageConfig(timelineServer.getRemoteFileSystemViewConfig()); } } -} +} \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java index feda7d2b1859..5432e9b34efd 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; @@ -29,37 +30,109 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.timeline.service.TimelineService; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; /** * Timeline Service that runs as part of write client. */ public class EmbeddedTimelineService { + // lock used when starting/stopping/modifying embedded services + private static final Object SERVICE_LOCK = new Object(); private static final Logger LOG = LoggerFactory.getLogger(EmbeddedTimelineService.class); - + private static final AtomicInteger NUM_SERVERS_RUNNING = new AtomicInteger(0); + // Map of TimelineServiceIdentifier to existing timeline service running + private static final Map RUNNING_SERVICES = new HashMap<>(); + private static final Registry METRICS_REGISTRY = Registry.getRegistry("TimelineService"); + private static final String NUM_EMBEDDED_TIMELINE_SERVERS = "numEmbeddedTimelineServers"; private int serverPort; private String hostAddr; - private HoodieEngineContext context; + private final HoodieEngineContext context; private final SerializableConfiguration hadoopConf; private final HoodieWriteConfig writeConfig; - private final String basePath; + private TimelineService.Config serviceConfig; + private final TimelineServiceIdentifier timelineServiceIdentifier; + private final Set basePaths; // the set of base paths using this EmbeddedTimelineService private transient FileSystemViewManager viewManager; private transient TimelineService server; - public EmbeddedTimelineService(HoodieEngineContext context, String embeddedTimelineServiceHostAddr, HoodieWriteConfig writeConfig) { + private EmbeddedTimelineService(HoodieEngineContext context, String embeddedTimelineServiceHostAddr, HoodieWriteConfig writeConfig, + TimelineServiceIdentifier timelineServiceIdentifier) { setHostAddr(embeddedTimelineServiceHostAddr); this.context = context; this.writeConfig = writeConfig; - this.basePath = writeConfig.getBasePath(); + this.timelineServiceIdentifier = timelineServiceIdentifier; + this.basePaths = new HashSet<>(); + this.basePaths.add(writeConfig.getBasePath()); this.hadoopConf = context.getHadoopConf(); this.viewManager = createViewManager(); } + /** + * Returns an existing embedded timeline service if one is running for the given configuration and reuse is enabled, or starts a new one. + * @param context The {@link HoodieEngineContext} for the client + * @param embeddedTimelineServiceHostAddr The host address to use for the service (nullable) + * @param writeConfig The {@link HoodieWriteConfig} for the client + * @return A running {@link EmbeddedTimelineService} + * @throws IOException if an error occurs while starting the service + */ + public static EmbeddedTimelineService getOrStartEmbeddedTimelineService(HoodieEngineContext context, String embeddedTimelineServiceHostAddr, HoodieWriteConfig writeConfig) throws IOException { + return getOrStartEmbeddedTimelineService(context, embeddedTimelineServiceHostAddr, writeConfig, TimelineService::new); + } + + static EmbeddedTimelineService getOrStartEmbeddedTimelineService(HoodieEngineContext context, String embeddedTimelineServiceHostAddr, HoodieWriteConfig writeConfig, + TimelineServiceCreator timelineServiceCreator) throws IOException { + TimelineServiceIdentifier timelineServiceIdentifier = getTimelineServiceIdentifier(embeddedTimelineServiceHostAddr, writeConfig); + // if reuse is enabled, check if any existing instances are compatible + if (writeConfig.isEmbeddedTimelineServerReuseEnabled()) { + synchronized (SERVICE_LOCK) { + if (RUNNING_SERVICES.containsKey(timelineServiceIdentifier)) { + RUNNING_SERVICES.get(timelineServiceIdentifier).addBasePath(writeConfig.getBasePath()); + LOG.info("Reusing existing embedded timeline server with configuration: " + RUNNING_SERVICES.get(timelineServiceIdentifier).serviceConfig); + return RUNNING_SERVICES.get(timelineServiceIdentifier); + } + // if no compatible instance is found, create a new one + EmbeddedTimelineService service = createAndStartService(context, embeddedTimelineServiceHostAddr, writeConfig, + timelineServiceCreator, timelineServiceIdentifier); + RUNNING_SERVICES.put(timelineServiceIdentifier, service); + return service; + } + } + // if not, create a new instance. If reuse is not enabled, there is no need to add it to RUNNING_SERVICES + return createAndStartService(context, embeddedTimelineServiceHostAddr, writeConfig, timelineServiceCreator, timelineServiceIdentifier); + } + + private static EmbeddedTimelineService createAndStartService(HoodieEngineContext context, String embeddedTimelineServiceHostAddr, HoodieWriteConfig writeConfig, + TimelineServiceCreator timelineServiceCreator, + TimelineServiceIdentifier timelineServiceIdentifier) throws IOException { + EmbeddedTimelineService service = new EmbeddedTimelineService(context, embeddedTimelineServiceHostAddr, writeConfig, timelineServiceIdentifier); + service.startServer(timelineServiceCreator); + METRICS_REGISTRY.set(NUM_EMBEDDED_TIMELINE_SERVERS, NUM_SERVERS_RUNNING.incrementAndGet()); + return service; + } + + public static void shutdownAllTimelineServers() { + RUNNING_SERVICES.entrySet().forEach(entry -> { + LOG.info("Closing Timeline server"); + entry.getValue().server.close(); + METRICS_REGISTRY.set(NUM_EMBEDDED_TIMELINE_SERVERS, NUM_SERVERS_RUNNING.decrementAndGet()); + LOG.info("Closed Timeline server"); + }); + RUNNING_SERVICES.clear(); + } + private FileSystemViewManager createViewManager() { // Using passed-in configs to build view storage configs FileSystemViewStorageConfig.Builder builder = @@ -73,7 +146,7 @@ private FileSystemViewManager createViewManager() { return FileSystemViewManager.createViewManagerWithTableMetadata(context, writeConfig.getMetadataConfig(), builder.build(), writeConfig.getCommonConfig()); } - public void startServer() throws IOException { + private void startServer(TimelineServiceCreator timelineServiceCreator) throws IOException { TimelineService.Config.Builder timelineServiceConfBuilder = TimelineService.Config.builder() .serverPort(writeConfig.getEmbeddedTimelineServerPort()) .numThreads(writeConfig.getEmbeddedTimelineServerThreads()) @@ -106,12 +179,20 @@ public void startServer() throws IOException { .enableInstantStateRequests(true); } - server = new TimelineService(context, hadoopConf.newCopy(), timelineServiceConfBuilder.build(), - FSUtils.getFs(basePath, hadoopConf.newCopy()), viewManager); + this.serviceConfig = timelineServiceConfBuilder.build(); + + server = timelineServiceCreator.create(context, hadoopConf.newCopy(), serviceConfig, + FSUtils.getFs(writeConfig.getBasePath(), hadoopConf.newCopy()), createViewManager()); serverPort = server.startService(); LOG.info("Started embedded timeline server at " + hostAddr + ":" + serverPort); } + @FunctionalInterface + interface TimelineServiceCreator { + TimelineService create(HoodieEngineContext context, Configuration hadoopConf, TimelineService.Config timelineServerConf, + FileSystem fileSystem, FileSystemViewManager globalFileSystemViewManager) throws IOException; + } + private void setHostAddr(String embeddedTimelineServiceHostAddr) { if (embeddedTimelineServiceHostAddr != null) { LOG.info("Overriding hostIp to (" + embeddedTimelineServiceHostAddr + ") found in spark-conf. It was " + this.hostAddr); @@ -146,19 +227,80 @@ public FileSystemViewManager getViewManager() { return viewManager; } - public boolean canReuseFor(String basePath) { - return this.server != null - && this.viewManager != null - && this.basePath.equals(basePath); + /** + * Adds a new base path to the set that are managed by this instance. + * @param basePath the new base path to add + */ + private void addBasePath(String basePath) { + basePaths.add(basePath); } - public void stop() { - if (null != server) { + /** + * Stops the embedded timeline service for the given base path. If a timeline service is managing multiple tables, it will only be shutdown once all tables have been stopped. + * @param basePath For the table to stop the service for + */ + public void stopForBasePath(String basePath) { + synchronized (SERVICE_LOCK) { + basePaths.remove(basePath); + if (basePaths.isEmpty()) { + RUNNING_SERVICES.remove(timelineServiceIdentifier); + } + } + if (this.server != null) { + this.server.unregisterBasePath(basePath); + } + // continue rest of shutdown outside of the synchronized block to avoid excess blocking + if (basePaths.isEmpty() && null != server) { LOG.info("Closing Timeline server"); this.server.close(); + METRICS_REGISTRY.set(NUM_EMBEDDED_TIMELINE_SERVERS, NUM_SERVERS_RUNNING.decrementAndGet()); this.server = null; this.viewManager = null; LOG.info("Closed Timeline server"); } } -} + + private static TimelineServiceIdentifier getTimelineServiceIdentifier(String hostAddr, HoodieWriteConfig writeConfig) { + return new TimelineServiceIdentifier(hostAddr, writeConfig.getMarkersType(), writeConfig.isMetadataTableEnabled(), + writeConfig.isEarlyConflictDetectionEnable(), writeConfig.isTimelineServerBasedInstantStateEnabled()); + } + + static class TimelineServiceIdentifier { + private final String hostAddr; + private final MarkerType markerType; + private final boolean isMetadataEnabled; + private final boolean isEarlyConflictDetectionEnable; + private final boolean isTimelineServerBasedInstantStateEnabled; + + public TimelineServiceIdentifier(String hostAddr, MarkerType markerType, boolean isMetadataEnabled, boolean isEarlyConflictDetectionEnable, + boolean isTimelineServerBasedInstantStateEnabled) { + this.hostAddr = hostAddr; + this.markerType = markerType; + this.isMetadataEnabled = isMetadataEnabled; + this.isEarlyConflictDetectionEnable = isEarlyConflictDetectionEnable; + this.isTimelineServerBasedInstantStateEnabled = isTimelineServerBasedInstantStateEnabled; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof TimelineServiceIdentifier)) { + return false; + } + TimelineServiceIdentifier that = (TimelineServiceIdentifier) o; + if (this.hostAddr != null && that.hostAddr != null) { + return isMetadataEnabled == that.isMetadataEnabled && isEarlyConflictDetectionEnable == that.isEarlyConflictDetectionEnable + && isTimelineServerBasedInstantStateEnabled == that.isTimelineServerBasedInstantStateEnabled && hostAddr.equals(that.hostAddr) && markerType == that.markerType; + } else { + return (hostAddr == null && that.hostAddr == null); + } + } + + @Override + public int hashCode() { + return Objects.hash(hostAddr, markerType, isMetadataEnabled, isEarlyConflictDetectionEnable, isTimelineServerBasedInstantStateEnabled); + } + } +} \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/HoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/HoodieTimelineArchiver.java index 3277039f31bd..cb0b748dd5d7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/HoodieTimelineArchiver.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/HoodieTimelineArchiver.java @@ -37,6 +37,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metrics.HoodieMetrics; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; import org.apache.hudi.table.marker.WriteMarkers; @@ -74,6 +75,7 @@ public class HoodieTimelineArchiver { private final TransactionManager txnManager; private final LSMTimelineWriter timelineWriter; + private final HoodieMetrics metrics; public HoodieTimelineArchiver(HoodieWriteConfig config, HoodieTable table) { this.config = config; @@ -84,16 +86,17 @@ public HoodieTimelineArchiver(HoodieWriteConfig config, HoodieTable Pair minAndMaxInstants = getMinAndMaxInstantsToKeep(table, metaClient); this.minInstantsToKeep = minAndMaxInstants.getLeft(); this.maxInstantsToKeep = minAndMaxInstants.getRight(); + this.metrics = new HoodieMetrics(config); } - public boolean archiveIfRequired(HoodieEngineContext context) throws IOException { + public int archiveIfRequired(HoodieEngineContext context) throws IOException { return archiveIfRequired(context, false); } /** * Check if commits need to be archived. If yes, archive commits. */ - public boolean archiveIfRequired(HoodieEngineContext context, boolean acquireLock) throws IOException { + public int archiveIfRequired(HoodieEngineContext context, boolean acquireLock) throws IOException { try { if (acquireLock) { // there is no owner or instant time per se for archival. @@ -101,7 +104,6 @@ public boolean archiveIfRequired(HoodieEngineContext context, boolean acquireLoc } // Sort again because the cleaning and rollback instants could break the sequence. List instantsToArchive = getInstantsToArchive().sorted().collect(Collectors.toList()); - boolean success = true; if (!instantsToArchive.isEmpty()) { LOG.info("Archiving instants " + instantsToArchive); Consumer exceptionHandler = e -> { @@ -111,13 +113,13 @@ public boolean archiveIfRequired(HoodieEngineContext context, boolean acquireLoc }; this.timelineWriter.write(instantsToArchive, Option.of(action -> deleteAnyLeftOverMarkers(context, action)), Option.of(exceptionHandler)); LOG.info("Deleting archived instants " + instantsToArchive); - success = deleteArchivedInstants(instantsToArchive, context); + deleteArchivedInstants(instantsToArchive, context); // triggers compaction and cleaning only after archiving action this.timelineWriter.compactAndClean(context); } else { LOG.info("No Instants to archive"); } - return success; + return instantsToArchive.size(); } finally { if (acquireLock) { txnManager.endTransaction(Option.empty()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 6a36e5025bcc..51895751101c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -476,8 +476,8 @@ public class HoodieWriteConfig extends HoodieConfig { .key("hoodie.embed.timeline.server.reuse.enabled") .defaultValue(false) .markAdvanced() - .withDocumentation("Controls whether the timeline server instance should be cached and reused across the JVM (across task lifecycles)" - + "to avoid startup costs. This should rarely be changed."); + .withDocumentation("Controls whether the timeline server instance should be cached and reused across the tables" + + "to avoid startup costs and server overhead. This should only be used if you are running multiple writers in the same JVM."); public static final ConfigProperty EMBEDDED_TIMELINE_SERVER_PORT_NUM = ConfigProperty .key("hoodie.embed.timeline.server.port") diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java index a41aa82a3e8c..3ca75d3e2649 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java @@ -19,13 +19,9 @@ package org.apache.hudi.index.bucket; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.utils.LazyIterableIterator; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.index.HoodieIndex; @@ -37,8 +33,6 @@ import java.util.Arrays; import java.util.List; -import static org.apache.hudi.index.HoodieIndexUtils.tagAsNewRecordIfNeeded; - /** * Hash indexing mechanism. */ @@ -65,30 +59,6 @@ public HoodieData updateLocation(HoodieData writeStatu return writeStatuses; } - @Override - public HoodieData> tagLocation( - HoodieData> records, HoodieEngineContext context, - HoodieTable hoodieTable) - throws HoodieIndexException { - // Get bucket location mapper for the given partitions - List partitions = records.map(HoodieRecord::getPartitionPath).distinct().collectAsList(); - LOG.info("Get BucketIndexLocationMapper for partitions: " + partitions); - BucketIndexLocationMapper mapper = getLocationMapper(hoodieTable, partitions); - - return records.mapPartitions(iterator -> - new LazyIterableIterator, HoodieRecord>(iterator) { - @Override - protected HoodieRecord computeNext() { - // TODO maybe batch the operation to improve performance - HoodieRecord record = inputItr.next(); - Option loc = mapper.getRecordLocation(record.getKey()); - return tagAsNewRecordIfNeeded(record, loc); - } - }, - false - ); - } - @Override public boolean requiresTagging(WriteOperationType operationType) { switch (operationType) { @@ -127,9 +97,4 @@ public boolean isImplicitWithStorage() { public int getNumBuckets() { return numBuckets; } - - /** - * Get a location mapper for the given table & partitionPath - */ - protected abstract BucketIndexLocationMapper getLocationMapper(HoodieTable table, List partitionPath); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieConsistentBucketIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieConsistentBucketIndex.java index 156d14b7cf5c..125bc970d65f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieConsistentBucketIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieConsistentBucketIndex.java @@ -19,12 +19,14 @@ package org.apache.hudi.index.bucket; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.utils.LazyIterableIterator; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.ConsistentHashingNode; import org.apache.hudi.common.model.HoodieConsistentHashingMetadata; import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; @@ -35,10 +37,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Serializable; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.index.HoodieIndexUtils.tagAsNewRecordIfNeeded; + /** * Consistent hashing bucket index implementation, with auto-adjust bucket number. * NOTE: bucket resizing is triggered by clustering. @@ -71,11 +76,28 @@ public boolean rollbackCommit(String instantTime) { } @Override - protected BucketIndexLocationMapper getLocationMapper(HoodieTable table, List partitionPath) { - return new ConsistentBucketIndexLocationMapper(table, partitionPath); + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, + HoodieTable hoodieTable) + throws HoodieIndexException { + // Get bucket location mapper for the given partitions + List partitions = records.map(HoodieRecord::getPartitionPath).distinct().collectAsList(); + LOG.info("Get BucketIndexLocationMapper for partitions: " + partitions); + ConsistentBucketIndexLocationMapper mapper = new ConsistentBucketIndexLocationMapper(hoodieTable, partitions); + + return records.mapPartitions(iterator -> + new LazyIterableIterator, HoodieRecord>(iterator) { + @Override + protected HoodieRecord computeNext() { + // TODO maybe batch the operation to improve performance + HoodieRecord record = inputItr.next(); + Option loc = mapper.getRecordLocation(record.getKey()); + return tagAsNewRecordIfNeeded(record, loc); + } + }, false); } - public class ConsistentBucketIndexLocationMapper implements BucketIndexLocationMapper { + public class ConsistentBucketIndexLocationMapper implements Serializable { /** * Mapping from partitionPath -> bucket identifier @@ -90,7 +112,6 @@ public ConsistentBucketIndexLocationMapper(HoodieTable table, List parti })); } - @Override public Option getRecordLocation(HoodieKey key) { String partitionPath = key.getPartitionPath(); ConsistentHashingNode node = partitionToIdentifier.get(partitionPath).getBucket(key, indexKeyFields); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieSimpleBucketIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieSimpleBucketIndex.java index fa2289ed87e7..a38fa489a2a4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieSimpleBucketIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieSimpleBucketIndex.java @@ -18,29 +18,29 @@ package org.apache.hudi.index.bucket; +import org.apache.hudi.client.utils.LazyIterableIterator; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.index.HoodieIndexUtils; import org.apache.hudi.table.HoodieTable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.HashMap; -import java.util.List; import java.util.Map; -import java.util.stream.Collectors; + +import static org.apache.hudi.index.HoodieIndexUtils.tagAsNewRecordIfNeeded; /** * Simple bucket index implementation, with fixed bucket number. */ public class HoodieSimpleBucketIndex extends HoodieBucketIndex { - private static final Logger LOG = LoggerFactory.getLogger(HoodieSimpleBucketIndex.class); - public HoodieSimpleBucketIndex(HoodieWriteConfig config) { super(config); } @@ -79,27 +79,23 @@ public boolean canIndexLogFiles() { } @Override - protected BucketIndexLocationMapper getLocationMapper(HoodieTable table, List partitionPath) { - return new SimpleBucketIndexLocationMapper(table, partitionPath); - } - - public class SimpleBucketIndexLocationMapper implements BucketIndexLocationMapper { - - /** - * Mapping from partitionPath -> bucketId -> fileInfo - */ - private final Map> partitionPathFileIDList; - - public SimpleBucketIndexLocationMapper(HoodieTable table, List partitions) { - partitionPathFileIDList = partitions.stream() - .collect(Collectors.toMap(p -> p, p -> loadBucketIdToFileIdMappingForPartition(table, p))); - } - - @Override - public Option getRecordLocation(HoodieKey key) { - int bucketId = getBucketID(key); - Map bucketIdToFileIdMapping = partitionPathFileIDList.get(key.getPartitionPath()); - return Option.ofNullable(bucketIdToFileIdMapping.getOrDefault(bucketId, null)); - } + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, + HoodieTable hoodieTable) + throws HoodieIndexException { + Map> partitionPathFileIDList = new HashMap<>(); + return records.mapPartitions(iterator -> new LazyIterableIterator, HoodieRecord>(iterator) { + @Override + protected HoodieRecord computeNext() { + HoodieRecord record = inputItr.next(); + int bucketId = getBucketID(record.getKey()); + String partitionPath = record.getPartitionPath(); + if (!partitionPathFileIDList.containsKey(partitionPath)) { + partitionPathFileIDList.put(partitionPath, loadBucketIdToFileIdMappingForPartition(hoodieTable, partitionPath)); + } + HoodieRecordLocation loc = partitionPathFileIDList.get(partitionPath).getOrDefault(bucketId, null); + return tagAsNewRecordIfNeeded(record, Option.ofNullable(loc)); + } + }, false); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java index 135e4866cc5d..ab41a94c2a91 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java @@ -62,9 +62,11 @@ private List> fetchRecordKeysWithPositions(HoodieBaseFile public Stream> locations() { HoodieBaseFile baseFile = partitionPathBaseFilePair.getRight(); + String commitTime = baseFile.getCommitTime(); + String fileId = baseFile.getFileId(); return fetchRecordKeysWithPositions(baseFile).stream() .map(entry -> Pair.of(entry.getLeft(), - new HoodieRecordLocation(baseFile.getCommitTime(), baseFile.getFileId(), entry.getRight()))); + new HoodieRecordLocation(commitTime, fileId, entry.getRight()))); } public Stream> globalLocations() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperAvroKeyGenerator.java index a8ae48e1d67e..8431180a2fe6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperAvroKeyGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperAvroKeyGenerator.java @@ -43,24 +43,24 @@ * PartitionId refers to spark's partition Id. * RowId refers to the row index within the spark partition. */ -public class AutoRecordGenWrapperAvroKeyGenerator extends BaseKeyGenerator { +public class AutoRecordGenWrapperAvroKeyGenerator extends BaseKeyGenerator implements AutoRecordKeyGeneratorWrapper { private final BaseKeyGenerator keyGenerator; - private final int partitionId; - private final String instantTime; + private Integer partitionId; + private String instantTime; private int rowId; public AutoRecordGenWrapperAvroKeyGenerator(TypedProperties config, BaseKeyGenerator keyGenerator) { super(config); this.keyGenerator = keyGenerator; this.rowId = 0; - this.partitionId = config.getInteger(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG); - this.instantTime = config.getString(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG); + partitionId = null; + instantTime = null; } @Override public String getRecordKey(GenericRecord record) { - return HoodieRecord.generateSequenceId(instantTime, partitionId, rowId++); + return generateSequenceId(rowId++); } @Override @@ -80,4 +80,19 @@ public List getPartitionPathFields() { public boolean isConsistentLogicalTimestampEnabled() { return keyGenerator.isConsistentLogicalTimestampEnabled(); } + + @Override + public BaseKeyGenerator getPartitionKeyGenerator() { + return keyGenerator; + } + + private String generateSequenceId(long recordIndex) { + if (partitionId == null) { + this.partitionId = config.getInteger(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG); + } + if (instantTime == null) { + this.instantTime = config.getString(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG); + } + return HoodieRecord.generateSequenceId(instantTime, partitionId, recordIndex); + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIndexLocationMapper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/AutoRecordKeyGeneratorWrapper.java similarity index 67% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIndexLocationMapper.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/AutoRecordKeyGeneratorWrapper.java index 1ce68ef97bf2..e136bc89cbb5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIndexLocationMapper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/AutoRecordKeyGeneratorWrapper.java @@ -17,19 +17,16 @@ * under the License. */ -package org.apache.hudi.index.bucket; +package org.apache.hudi.keygen; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.util.Option; - -import java.io.Serializable; - -public interface BucketIndexLocationMapper extends Serializable { +/** + * Interface for {@link KeyGenerator} implementations that + * generate a unique record key internally. + */ +public interface AutoRecordKeyGeneratorWrapper { /** - * Get record location given hoodie key + * @returns the underlying key generator used for the partition path. */ - Option getRecordLocation(HoodieKey key); - + BaseKeyGenerator getPartitionKeyGenerator(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java index 792d0cd08442..1fc23cf1e8de 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java @@ -53,11 +53,28 @@ public class HoodieMetrics { public static final String TOTAL_RECORDS_DELETED = "totalRecordsDeleted"; public static final String TOTAL_CORRUPTED_LOG_BLOCKS_STR = "totalCorruptedLogBlocks"; public static final String TOTAL_ROLLBACK_LOG_BLOCKS_STR = "totalRollbackLogBlocks"; - + public static final String DURATION_STR = "duration"; + public static final String DELETE_FILES_NUM_STR = "numFilesDeleted"; + public static final String DELETE_INSTANTS_NUM_STR = "numInstantsArchived"; + public static final String FINALIZED_FILES_NUM_STR = "numFilesFinalized"; + public static final String CONFLICT_RESOLUTION_STR = "conflict_resolution"; + public static final String COMMIT_LATENCY_IN_MS_STR = "commitLatencyInMs"; + public static final String COMMIT_FRESHNESS_IN_MS_STR = "commitFreshnessInMs"; + public static final String COMMIT_TIME_STR = "commitTime"; + public static final String SUCCESS_EXTENSION = ".success"; + public static final String FAILURE_EXTENSION = ".failure"; + + public static final String TIMER_ACTION = "timer"; + public static final String COUNTER_ACTION = "counter"; + public static final String ARCHIVE_ACTION = "archive"; + public static final String FINALIZE_ACTION = "finalize"; + public static final String INDEX_ACTION = "index"; + private Metrics metrics; // Some timers public String rollbackTimerName = null; public String cleanTimerName = null; + public String archiveTimerName = null; public String commitTimerName = null; public String logCompactionTimerName = null; public String deltaCommitTimerName = null; @@ -74,6 +91,7 @@ public class HoodieMetrics { private String tableName; private Timer rollbackTimer = null; private Timer cleanTimer = null; + private Timer archiveTimer = null; private Timer commitTimer = null; private Timer deltaCommitTimer = null; private Timer finalizeTimer = null; @@ -92,20 +110,21 @@ public HoodieMetrics(HoodieWriteConfig config) { this.tableName = config.getTableName(); if (config.isMetricsOn()) { metrics = Metrics.getInstance(config); - this.rollbackTimerName = getMetricsName("timer", HoodieTimeline.ROLLBACK_ACTION); - this.cleanTimerName = getMetricsName("timer", HoodieTimeline.CLEAN_ACTION); - this.commitTimerName = getMetricsName("timer", HoodieTimeline.COMMIT_ACTION); - this.deltaCommitTimerName = getMetricsName("timer", HoodieTimeline.DELTA_COMMIT_ACTION); - this.replaceCommitTimerName = getMetricsName("timer", HoodieTimeline.REPLACE_COMMIT_ACTION); - this.finalizeTimerName = getMetricsName("timer", "finalize"); - this.compactionTimerName = getMetricsName("timer", HoodieTimeline.COMPACTION_ACTION); - this.logCompactionTimerName = getMetricsName("timer", HoodieTimeline.LOG_COMPACTION_ACTION); - this.indexTimerName = getMetricsName("timer", "index"); - this.conflictResolutionTimerName = getMetricsName("timer", "conflict_resolution"); - this.conflictResolutionSuccessCounterName = getMetricsName("counter", "conflict_resolution.success"); - this.conflictResolutionFailureCounterName = getMetricsName("counter", "conflict_resolution.failure"); - this.compactionRequestedCounterName = getMetricsName("counter", "compaction.requested"); - this.compactionCompletedCounterName = getMetricsName("counter", "compaction.completed"); + this.rollbackTimerName = getMetricsName(TIMER_ACTION, HoodieTimeline.ROLLBACK_ACTION); + this.cleanTimerName = getMetricsName(TIMER_ACTION, HoodieTimeline.CLEAN_ACTION); + this.archiveTimerName = getMetricsName(TIMER_ACTION, ARCHIVE_ACTION); + this.commitTimerName = getMetricsName(TIMER_ACTION, HoodieTimeline.COMMIT_ACTION); + this.deltaCommitTimerName = getMetricsName(TIMER_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION); + this.replaceCommitTimerName = getMetricsName(TIMER_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION); + this.finalizeTimerName = getMetricsName(TIMER_ACTION, FINALIZE_ACTION); + this.compactionTimerName = getMetricsName(TIMER_ACTION, HoodieTimeline.COMPACTION_ACTION); + this.logCompactionTimerName = getMetricsName(TIMER_ACTION, HoodieTimeline.LOG_COMPACTION_ACTION); + this.indexTimerName = getMetricsName(TIMER_ACTION, INDEX_ACTION); + this.conflictResolutionTimerName = getMetricsName(TIMER_ACTION, CONFLICT_RESOLUTION_STR); + this.conflictResolutionSuccessCounterName = getMetricsName(COUNTER_ACTION, CONFLICT_RESOLUTION_STR + SUCCESS_EXTENSION); + this.conflictResolutionFailureCounterName = getMetricsName(COUNTER_ACTION, CONFLICT_RESOLUTION_STR + FAILURE_EXTENSION); + this.compactionRequestedCounterName = getMetricsName(COUNTER_ACTION, HoodieTimeline.COMPACTION_ACTION + HoodieTimeline.REQUESTED_EXTENSION); + this.compactionCompletedCounterName = getMetricsName(COUNTER_ACTION, HoodieTimeline.COMPACTION_ACTION + HoodieTimeline.COMPLETED_EXTENSION); } } @@ -152,6 +171,13 @@ public Timer.Context getCleanCtx() { return cleanTimer == null ? null : cleanTimer.time(); } + public Timer.Context getArchiveCtx() { + if (config.isMetricsOn() && archiveTimer == null) { + archiveTimer = createTimer(archiveTimerName); + } + return archiveTimer == null ? null : archiveTimer.time(); + } + public Timer.Context getCommitCtx() { if (config.isMetricsOn() && commitTimer == null) { commitTimer = createTimer(commitTimerName); @@ -255,48 +281,60 @@ private void updateCommitTimingMetrics(long commitEpochTimeInMs, long durationIn Pair, Option> eventTimePairMinMax = metadata.getMinAndMaxEventTime(); if (eventTimePairMinMax.getLeft().isPresent()) { long commitLatencyInMs = commitEpochTimeInMs + durationInMs - eventTimePairMinMax.getLeft().get(); - metrics.registerGauge(getMetricsName(actionType, "commitLatencyInMs"), commitLatencyInMs); + metrics.registerGauge(getMetricsName(actionType, COMMIT_LATENCY_IN_MS_STR), commitLatencyInMs); } if (eventTimePairMinMax.getRight().isPresent()) { long commitFreshnessInMs = commitEpochTimeInMs + durationInMs - eventTimePairMinMax.getRight().get(); - metrics.registerGauge(getMetricsName(actionType, "commitFreshnessInMs"), commitFreshnessInMs); + metrics.registerGauge(getMetricsName(actionType, COMMIT_FRESHNESS_IN_MS_STR), commitFreshnessInMs); } - metrics.registerGauge(getMetricsName(actionType, "commitTime"), commitEpochTimeInMs); - metrics.registerGauge(getMetricsName(actionType, "duration"), durationInMs); + metrics.registerGauge(getMetricsName(actionType, COMMIT_TIME_STR), commitEpochTimeInMs); + metrics.registerGauge(getMetricsName(actionType, DURATION_STR), durationInMs); } } public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) { if (config.isMetricsOn()) { LOG.info( - String.format("Sending rollback metrics (duration=%d, numFilesDeleted=%d)", durationInMs, numFilesDeleted)); - metrics.registerGauge(getMetricsName("rollback", "duration"), durationInMs); - metrics.registerGauge(getMetricsName("rollback", "numFilesDeleted"), numFilesDeleted); + String.format("Sending rollback metrics (%s=%d, %s=%d)", DURATION_STR, durationInMs, + DELETE_FILES_NUM_STR, numFilesDeleted)); + metrics.registerGauge(getMetricsName(HoodieTimeline.ROLLBACK_ACTION, DURATION_STR), durationInMs); + metrics.registerGauge(getMetricsName(HoodieTimeline.ROLLBACK_ACTION, DELETE_FILES_NUM_STR), numFilesDeleted); } } public void updateCleanMetrics(long durationInMs, int numFilesDeleted) { if (config.isMetricsOn()) { LOG.info( - String.format("Sending clean metrics (duration=%d, numFilesDeleted=%d)", durationInMs, numFilesDeleted)); - metrics.registerGauge(getMetricsName("clean", "duration"), durationInMs); - metrics.registerGauge(getMetricsName("clean", "numFilesDeleted"), numFilesDeleted); + String.format("Sending clean metrics (%s=%d, %s=%d)", DURATION_STR, durationInMs, + DELETE_FILES_NUM_STR, numFilesDeleted)); + metrics.registerGauge(getMetricsName(HoodieTimeline.CLEAN_ACTION, DURATION_STR), durationInMs); + metrics.registerGauge(getMetricsName(HoodieTimeline.CLEAN_ACTION, DELETE_FILES_NUM_STR), numFilesDeleted); + } + } + + public void updateArchiveMetrics(long durationInMs, int numInstantsArchived) { + if (config.isMetricsOn()) { + LOG.info( + String.format("Sending archive metrics (%s=%d, %s=%d)", DURATION_STR, durationInMs, + DELETE_INSTANTS_NUM_STR, numInstantsArchived)); + metrics.registerGauge(getMetricsName(ARCHIVE_ACTION, DURATION_STR), durationInMs); + metrics.registerGauge(getMetricsName(ARCHIVE_ACTION, DELETE_INSTANTS_NUM_STR), numInstantsArchived); } } public void updateFinalizeWriteMetrics(long durationInMs, long numFilesFinalized) { if (config.isMetricsOn()) { - LOG.info(String.format("Sending finalize write metrics (duration=%d, numFilesFinalized=%d)", durationInMs, - numFilesFinalized)); - metrics.registerGauge(getMetricsName("finalize", "duration"), durationInMs); - metrics.registerGauge(getMetricsName("finalize", "numFilesFinalized"), numFilesFinalized); + LOG.info(String.format("Sending finalize write metrics (%s=%d, %s=%d)", DURATION_STR, durationInMs, + FINALIZED_FILES_NUM_STR, numFilesFinalized)); + metrics.registerGauge(getMetricsName(FINALIZE_ACTION, DURATION_STR), durationInMs); + metrics.registerGauge(getMetricsName(FINALIZE_ACTION, FINALIZED_FILES_NUM_STR), numFilesFinalized); } } public void updateIndexMetrics(final String action, final long durationInMs) { if (config.isMetricsOn()) { - LOG.info(String.format("Sending index metrics (%s.duration, %d)", action, durationInMs)); - metrics.registerGauge(getMetricsName("index", String.format("%s.duration", action)), durationInMs); + LOG.info(String.format("Sending index metrics (%s.%s, %d)", action, DURATION_STR, durationInMs)); + metrics.registerGauge(getMetricsName(INDEX_ACTION, String.format("%s.%s", action, DURATION_STR)), durationInMs); } } @@ -306,7 +344,7 @@ public String getMetricsName(String action, String metric) { } public void updateClusteringFileCreationMetrics(long durationInMs) { - reportMetrics("replacecommit", "fileCreationTime", durationInMs); + reportMetrics(HoodieTimeline.REPLACE_COMMIT_ACTION, "fileCreationTime", durationInMs); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java index f71d394238ea..47ee23bcc2fb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java @@ -86,6 +86,8 @@ public static synchronized Metrics getInstance(HoodieWriteConfig metricConfig) { public static synchronized void shutdownAllMetrics() { METRICS_INSTANCE_PER_BASEPATH.values().forEach(Metrics::shutdown); + // to avoid reusing already stopped metrics + METRICS_INSTANCE_PER_BASEPATH.clear(); } private List addAdditionalMetricsExporters(HoodieWriteConfig metricConfig) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java index 1394e6626268..34fd7a07f653 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java @@ -18,42 +18,76 @@ package org.apache.hudi.metrics.prometheus; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.metrics.MetricsReporter; import com.codahale.metrics.MetricRegistry; +import io.prometheus.client.Collector; import io.prometheus.client.CollectorRegistry; import io.prometheus.client.dropwizard.DropwizardExports; +import io.prometheus.client.dropwizard.samplebuilder.DefaultSampleBuilder; +import io.prometheus.client.dropwizard.samplebuilder.SampleBuilder; import io.prometheus.client.exporter.HTTPServer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; /** * Implementation of Prometheus reporter, which connects to the Http server, and get metrics * from that server. */ public class PrometheusReporter extends MetricsReporter { + private static final Pattern LABEL_PATTERN = Pattern.compile("\\s*,\\s*"); private static final Logger LOG = LoggerFactory.getLogger(PrometheusReporter.class); + private static final Map PORT_TO_COLLECTOR_REGISTRY = new HashMap<>(); + private static final Map PORT_TO_SERVER = new HashMap<>(); - private HTTPServer httpServer; private final DropwizardExports metricExports; private final CollectorRegistry collectorRegistry; + private final int serverPort; public PrometheusReporter(HoodieWriteConfig config, MetricRegistry registry) { - int serverPort = config.getPrometheusPort(); - collectorRegistry = new CollectorRegistry(); - metricExports = new DropwizardExports(registry); + this.serverPort = config.getPrometheusPort(); + if (!PORT_TO_SERVER.containsKey(serverPort) || !PORT_TO_COLLECTOR_REGISTRY.containsKey(serverPort)) { + startHttpServer(serverPort); + } + List labelNames = new ArrayList<>(); + List labelValues = new ArrayList<>(); + if (StringUtils.nonEmpty(config.getPushGatewayLabels())) { + LABEL_PATTERN.splitAsStream(config.getPushGatewayLabels().trim()).map(s -> s.split(":", 2)) + .forEach(parts -> { + labelNames.add(parts[0]); + labelValues.add(parts[1]); + }); + } + metricExports = new DropwizardExports(registry, new LabeledSampleBuilder(labelNames, labelValues)); + this.collectorRegistry = PORT_TO_COLLECTOR_REGISTRY.get(serverPort); metricExports.register(collectorRegistry); - try { - httpServer = new HTTPServer(new InetSocketAddress(serverPort), collectorRegistry); - } catch (Exception e) { - String msg = "Could not start PrometheusReporter HTTP server on port " + serverPort; - LOG.error(msg, e); - throw new HoodieException(msg, e); + } + + private static synchronized void startHttpServer(int serverPort) { + if (!PORT_TO_COLLECTOR_REGISTRY.containsKey(serverPort)) { + PORT_TO_COLLECTOR_REGISTRY.put(serverPort, new CollectorRegistry()); + } + if (!PORT_TO_SERVER.containsKey(serverPort)) { + try { + HTTPServer server = new HTTPServer(new InetSocketAddress(serverPort), PORT_TO_COLLECTOR_REGISTRY.get(serverPort)); + PORT_TO_SERVER.put(serverPort, server); + Runtime.getRuntime().addShutdownHook(new Thread(server::stop)); + } catch (Exception e) { + String msg = "Could not start PrometheusReporter HTTP server on port " + serverPort; + LOG.error(msg, e); + throw new HoodieException(msg, e); + } } } @@ -68,8 +102,31 @@ public void report() { @Override public void stop() { collectorRegistry.unregister(metricExports); + HTTPServer httpServer = PORT_TO_SERVER.remove(serverPort); if (httpServer != null) { httpServer.stop(); } + PORT_TO_COLLECTOR_REGISTRY.remove(serverPort); + } + + private static class LabeledSampleBuilder implements SampleBuilder { + private final DefaultSampleBuilder defaultMetricSampleBuilder = new DefaultSampleBuilder(); + private final List labelNames; + private final List labelValues; + + public LabeledSampleBuilder(List labelNames, List labelValues) { + this.labelNames = labelNames; + this.labelValues = labelValues; + } + + @Override + public Collector.MetricFamilySamples.Sample createSample(String dropwizardName, String nameSuffix, List additionalLabelNames, List additionalLabelValues, double value) { + return defaultMetricSampleBuilder.createSample( + dropwizardName, + nameSuffix, + labelNames, + labelValues, + value); + } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index aa521fcc3e55..b37fa0302a05 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -837,8 +837,12 @@ private void validateSchema() throws HoodieUpsertException, HoodieInsertExceptio try { TableSchemaResolver schemaResolver = new TableSchemaResolver(getMetaClient()); + Option existingTableSchema = schemaResolver.getTableAvroSchemaIfPresent(false); + if (!existingTableSchema.isPresent()) { + return; + } Schema writerSchema = HoodieAvroUtils.createHoodieWriteSchema(config.getSchema()); - Schema tableSchema = HoodieAvroUtils.createHoodieWriteSchema(schemaResolver.getTableAvroSchema(false)); + Schema tableSchema = HoodieAvroUtils.createHoodieWriteSchema(existingTableSchema.get()); AvroSchemaUtils.checkSchemaCompatible(tableSchema, writerSchema, shouldValidate, allowProjection, getDropPartitionColNames()); } catch (Exception e) { throw new HoodieException("Failed to read schema/check compatibility for base path " + metaClient.getBasePath(), e); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java index 3b5d12332145..a70bfd256c08 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java @@ -41,7 +41,9 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -118,17 +120,23 @@ HoodieCleanerPlan requestClean(HoodieEngineContext context) { context.setJobStatus(this.getClass().getSimpleName(), "Generating list of file slices to be cleaned: " + config.getTableName()); - Map>> cleanOpsWithPartitionMeta = context - .map(partitionsToClean, partitionPathToClean -> Pair.of(partitionPathToClean, planner.getDeletePaths(partitionPathToClean, earliestInstant)), cleanerParallelism) - .stream() - .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); - - Map> cleanOps = cleanOpsWithPartitionMeta.entrySet().stream() - .collect(Collectors.toMap(Map.Entry::getKey, - e -> CleanerUtils.convertToHoodieCleanFileInfoList(e.getValue().getValue()))); - - List partitionsToDelete = cleanOpsWithPartitionMeta.entrySet().stream().filter(entry -> entry.getValue().getKey()).map(Map.Entry::getKey) - .collect(Collectors.toList()); + Map> cleanOps = new HashMap<>(); + List partitionsToDelete = new ArrayList<>(); + for (int i = 0; i < partitionsToClean.size(); i += cleanerParallelism) { + // Handles at most 'cleanerParallelism' number of partitions once at a time to avoid overlarge memory pressure to the timeline server + // (remote or local embedded), thus to reduce the risk of an OOM exception. + List subPartitionsToClean = partitionsToClean.subList(i, Math.min(i + cleanerParallelism, partitionsToClean.size())); + Map>> cleanOpsWithPartitionMeta = context + .map(subPartitionsToClean, partitionPathToClean -> Pair.of(partitionPathToClean, planner.getDeletePaths(partitionPathToClean, earliestInstant)), cleanerParallelism) + .stream() + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + + cleanOps.putAll(cleanOpsWithPartitionMeta.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> CleanerUtils.convertToHoodieCleanFileInfoList(e.getValue().getValue())))); + + partitionsToDelete.addAll(cleanOpsWithPartitionMeta.entrySet().stream().filter(entry -> entry.getValue().getKey()).map(Map.Entry::getKey) + .collect(Collectors.toList())); + } return new HoodieCleanerPlan(earliestInstant .map(x -> new HoodieActionInstant(x.getTimestamp(), x.getAction(), x.getState().name())).orElse(null), diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java index 112034fd877a..efbca863e507 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java @@ -254,7 +254,7 @@ private Pair> getFilesToCleanKeepingLatestVersions( // In other words, the file versions only apply to the active file groups. deletePaths.addAll(getReplacedFilesEligibleToClean(savepointedFiles, partitionPath, Option.empty())); boolean toDeletePartition = false; - List fileGroups = fileSystemView.getAllFileGroups(partitionPath).collect(Collectors.toList()); + List fileGroups = fileSystemView.getAllFileGroupsStateless(partitionPath).collect(Collectors.toList()); for (HoodieFileGroup fileGroup : fileGroups) { int keepVersions = config.getCleanerFileVersionsRetained(); // do not cleanup slice required for pending compaction @@ -329,7 +329,7 @@ private Pair> getFilesToCleanKeepingLatestCommits(S // all replaced file groups before earliestCommitToRetain are eligible to clean deletePaths.addAll(getReplacedFilesEligibleToClean(savepointedFiles, partitionPath, earliestCommitToRetain)); // add active files - List fileGroups = fileSystemView.getAllFileGroups(partitionPath).collect(Collectors.toList()); + List fileGroups = fileSystemView.getAllFileGroupsStateless(partitionPath).collect(Collectors.toList()); for (HoodieFileGroup fileGroup : fileGroups) { List fileSliceList = fileGroup.getAllFileSlices().collect(Collectors.toList()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java index 2d2c2a36643d..0d07bed531a4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java @@ -121,7 +121,7 @@ protected Stream getFileSlicesEligibleForClustering(String partition) .collect(Collectors.toSet()); fgIdsInPendingCompactionLogCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getKey).collect(Collectors.toSet())); - return hoodieTable.getSliceView().getLatestFileSlices(partition) + return hoodieTable.getSliceView().getLatestFileSlicesStateless(partition) // file ids already in clustering are not eligible .filter(slice -> !fgIdsInPendingCompactionLogCompactionAndClustering.contains(slice.getFileGroupId())); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java index 0f3beb136b22..2626bc599186 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java @@ -117,7 +117,7 @@ public HoodieCompactionPlan generateCompactionPlan(String compactionInstant) thr Option instantRange = CompactHelpers.getInstance().getInstantRange(metaClient); List operations = engineContext.flatMap(partitionPaths, partitionPath -> fileSystemView - .getLatestFileSlices(partitionPath) + .getLatestFileSlicesStateless(partitionPath) .filter(slice -> filterFileSlice(slice, lastCompletedInstantTime, fgIdsInPendingCompactionAndClustering, instantRange)) .map(s -> { List logFiles = s.getLogFiles() diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java index 2306763beb80..50e4363ee202 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java @@ -62,6 +62,8 @@ */ public class TimelineServerBasedWriteMarkers extends WriteMarkers { private static final Logger LOG = LoggerFactory.getLogger(TimelineServerBasedWriteMarkers.class); + private static final TypeReference BOOLEAN_TYPE_REFERENCE = new TypeReference() {}; + private static final TypeReference> SET_TYPE_REFERENCE = new TypeReference>() {}; private final HttpRequestClient httpRequestClient; @@ -84,7 +86,7 @@ public boolean deleteMarkerDir(HoodieEngineContext context, int parallelism) { Map paramsMap = Collections.singletonMap(MARKER_DIR_PATH_PARAM, markerDirPath.toString()); try { return httpRequestClient.executeRequest( - DELETE_MARKER_DIR_URL, paramsMap, new TypeReference() {}, RequestMethod.POST); + DELETE_MARKER_DIR_URL, paramsMap, BOOLEAN_TYPE_REFERENCE, RequestMethod.POST); } catch (IOException e) { throw new HoodieRemoteException("Failed to delete marker directory " + markerDirPath.toString(), e); } @@ -95,7 +97,7 @@ public boolean doesMarkerDirExist() { Map paramsMap = Collections.singletonMap(MARKER_DIR_PATH_PARAM, markerDirPath.toString()); try { return httpRequestClient.executeRequest( - MARKERS_DIR_EXISTS_URL, paramsMap, new TypeReference() {}, RequestMethod.GET); + MARKERS_DIR_EXISTS_URL, paramsMap, BOOLEAN_TYPE_REFERENCE, RequestMethod.GET); } catch (IOException e) { throw new HoodieRemoteException("Failed to check marker directory " + markerDirPath.toString(), e); } @@ -106,7 +108,7 @@ public Set createdAndMergedDataPaths(HoodieEngineContext context, int pa Map paramsMap = Collections.singletonMap(MARKER_DIR_PATH_PARAM, markerDirPath.toString()); try { Set markerPaths = httpRequestClient.executeRequest( - CREATE_AND_MERGE_MARKERS_URL, paramsMap, new TypeReference>() {}, RequestMethod.GET); + CREATE_AND_MERGE_MARKERS_URL, paramsMap, SET_TYPE_REFERENCE, RequestMethod.GET); return markerPaths.stream().map(WriteMarkers::stripMarkerSuffix).collect(Collectors.toSet()); } catch (IOException e) { throw new HoodieRemoteException("Failed to get CREATE and MERGE data file paths in " @@ -119,7 +121,7 @@ public Set allMarkerFilePaths() { Map paramsMap = Collections.singletonMap(MARKER_DIR_PATH_PARAM, markerDirPath.toString()); try { return httpRequestClient.executeRequest( - ALL_MARKERS_URL, paramsMap, new TypeReference>() {}, RequestMethod.GET); + ALL_MARKERS_URL, paramsMap, SET_TYPE_REFERENCE, RequestMethod.GET); } catch (IOException e) { throw new HoodieRemoteException("Failed to get all markers in " + markerDirPath.toString(), e); } @@ -173,8 +175,7 @@ private boolean executeCreateMarkerRequest(Map paramsMap, String boolean success; try { success = httpRequestClient.executeRequest( - CREATE_MARKER_URL, paramsMap, new TypeReference() { - }, HttpRequestClient.RequestMethod.POST); + CREATE_MARKER_URL, paramsMap, BOOLEAN_TYPE_REFERENCE, HttpRequestClient.RequestMethod.POST); } catch (IOException e) { throw new HoodieRemoteException("Failed to create marker file " + partitionPath + "/" + markerFileName, e); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToFourUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToFourUpgradeHandler.java index 4da675ea8200..c7cb544aec94 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToFourUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToFourUpgradeHandler.java @@ -22,12 +22,14 @@ import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.metadata.MetadataPartitionType; import java.util.Hashtable; import java.util.Map; +import static org.apache.hudi.common.table.HoodieTableConfig.DATABASE_NAME; import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_CHECKSUM; import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_METADATA_PARTITIONS; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.metadataPartitionExists; @@ -40,6 +42,10 @@ public class ThreeToFourUpgradeHandler implements UpgradeHandler { @Override public Map upgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime, SupportsUpgradeDowngrade upgradeDowngradeHelper) { Map tablePropsToAdd = new Hashtable<>(); + String database = config.getString(DATABASE_NAME); + if (StringUtils.nonEmpty(database)) { + tablePropsToAdd.put(DATABASE_NAME, database); + } tablePropsToAdd.put(TABLE_CHECKSUM, String.valueOf(HoodieTableConfig.generateChecksum(config.getProps()))); // if metadata is enabled and files partition exist then update TABLE_METADATA_INDEX_COMPLETED // schema for the files partition is same between the two versions diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/util/HttpRequestClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/util/HttpRequestClient.java index 65131cc77428..d977f8afa2cf 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/util/HttpRequestClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/util/HttpRequestClient.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.module.afterburner.AfterburnerModule; import org.apache.http.client.fluent.Request; import org.apache.http.client.fluent.Response; import org.apache.http.client.utils.URIBuilder; @@ -37,7 +38,7 @@ */ public class HttpRequestClient { private static final Logger LOG = LoggerFactory.getLogger(HttpRequestClient.class); - private final ObjectMapper mapper; + private static final ObjectMapper MAPPER = new ObjectMapper().registerModule(new AfterburnerModule()); private final String serverHost; private final int serverPort; private final int timeoutSecs; @@ -51,7 +52,6 @@ public HttpRequestClient(HoodieWriteConfig writeConfig) { } public HttpRequestClient(String serverHost, int serverPort, int timeoutSecs, int maxRetry) { - this.mapper = new ObjectMapper(); this.serverHost = serverHost; this.serverPort = serverPort; this.timeoutSecs = timeoutSecs; @@ -59,7 +59,7 @@ public HttpRequestClient(String serverHost, int serverPort, int timeoutSecs, int } public T executeRequestWithRetry(String requestPath, Map queryParameters, - TypeReference reference, RequestMethod method) { + TypeReference reference, RequestMethod method) { int retry = maxRetry; while (--retry >= 0) { try { @@ -72,14 +72,14 @@ public T executeRequestWithRetry(String requestPath, Map que } public T executeRequest(String requestPath, Map queryParameters, - TypeReference reference, RequestMethod method) throws IOException { + TypeReference reference, RequestMethod method) throws IOException { URIBuilder builder = new URIBuilder().setHost(serverHost).setPort(serverPort).setPath(requestPath).setScheme("http"); queryParameters.forEach(builder::addParameter); String url = builder.toString(); - LOG.debug("Sending request : (" + url + ")"); + LOG.debug("Sending request : ( {} )", url); Response response; int timeout = this.timeoutSecs * 1000; // msec switch (method) { @@ -92,7 +92,7 @@ public T executeRequest(String requestPath, Map queryParamet break; } String content = response.returnContent().asString(); - return (T) mapper.readValue(content, reference); + return (T) MAPPER.readValue(content, reference); } public enum RequestMethod { diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/embedded/TestEmbeddedTimelineService.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/embedded/TestEmbeddedTimelineService.java new file mode 100644 index 000000000000..f863316bc088 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/embedded/TestEmbeddedTimelineService.java @@ -0,0 +1,189 @@ +package org.apache.hudi.client.embedded; +/* + * 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. + */ + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.timeline.service.TimelineService; + +import org.apache.hadoop.conf.Configuration; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * These tests are mainly focused on testing the creation and reuse of the embedded timeline server. + */ +public class TestEmbeddedTimelineService extends HoodieCommonTestHarness { + + @Test + public void embeddedTimelineServiceReused() throws Exception { + HoodieEngineContext engineContext = new HoodieLocalEngineContext(new Configuration()); + HoodieWriteConfig writeConfig1 = HoodieWriteConfig.newBuilder() + .withPath(tempDir.resolve("table1").toString()) + .withEmbeddedTimelineServerEnabled(true) + .withEmbeddedTimelineServerReuseEnabled(true) + .build(); + EmbeddedTimelineService.TimelineServiceCreator mockCreator = Mockito.mock(EmbeddedTimelineService.TimelineServiceCreator.class); + TimelineService mockService = Mockito.mock(TimelineService.class); + when(mockCreator.create(any(), any(), any(), any(), any())).thenReturn(mockService); + when(mockService.startService()).thenReturn(123); + EmbeddedTimelineService service1 = EmbeddedTimelineService.getOrStartEmbeddedTimelineService(engineContext, null, writeConfig1, mockCreator); + + HoodieWriteConfig writeConfig2 = HoodieWriteConfig.newBuilder() + .withPath(tempDir.resolve("table2").toString()) + .withEmbeddedTimelineServerEnabled(true) + .withEmbeddedTimelineServerReuseEnabled(true) + .build(); + EmbeddedTimelineService.TimelineServiceCreator mockCreator2 = Mockito.mock(EmbeddedTimelineService.TimelineServiceCreator.class); + // do not mock the create method since that should never be called + EmbeddedTimelineService service2 = EmbeddedTimelineService.getOrStartEmbeddedTimelineService(engineContext, null, writeConfig2, mockCreator2); + assertSame(service1, service2); + + // test shutdown happens after the last path is removed + service1.stopForBasePath(writeConfig2.getBasePath()); + verify(mockService, never()).close(); + verify(mockService, times(1)).unregisterBasePath(writeConfig2.getBasePath()); + + service2.stopForBasePath(writeConfig1.getBasePath()); + verify(mockService, times(1)).unregisterBasePath(writeConfig1.getBasePath()); + verify(mockService, times(1)).close(); + } + + @Test + public void embeddedTimelineServiceCreatedForDifferentMetadataConfig() throws Exception { + HoodieEngineContext engineContext = new HoodieLocalEngineContext(new Configuration()); + HoodieWriteConfig writeConfig1 = HoodieWriteConfig.newBuilder() + .withPath(tempDir.resolve("table1").toString()) + .withEmbeddedTimelineServerEnabled(true) + .withEmbeddedTimelineServerReuseEnabled(true) + .build(); + EmbeddedTimelineService.TimelineServiceCreator mockCreator = Mockito.mock(EmbeddedTimelineService.TimelineServiceCreator.class); + TimelineService mockService = Mockito.mock(TimelineService.class); + when(mockCreator.create(any(), any(), any(), any(), any())).thenReturn(mockService); + when(mockService.startService()).thenReturn(321); + EmbeddedTimelineService service1 = EmbeddedTimelineService.getOrStartEmbeddedTimelineService(engineContext, null, writeConfig1, mockCreator); + + HoodieWriteConfig writeConfig2 = HoodieWriteConfig.newBuilder() + .withPath(tempDir.resolve("table2").toString()) + .withEmbeddedTimelineServerEnabled(true) + .withEmbeddedTimelineServerReuseEnabled(true) + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .enable(false) + .build()) + .build(); + EmbeddedTimelineService.TimelineServiceCreator mockCreator2 = Mockito.mock(EmbeddedTimelineService.TimelineServiceCreator.class); + TimelineService mockService2 = Mockito.mock(TimelineService.class); + when(mockCreator2.create(any(), any(), any(), any(), any())).thenReturn(mockService2); + when(mockService2.startService()).thenReturn(456); + EmbeddedTimelineService service2 = EmbeddedTimelineService.getOrStartEmbeddedTimelineService(engineContext, null, writeConfig2, mockCreator2); + assertNotSame(service1, service2); + + // test shutdown happens immediately since each server has only one path associated with it + service1.stopForBasePath(writeConfig1.getBasePath()); + verify(mockService, times(1)).close(); + + service2.stopForBasePath(writeConfig2.getBasePath()); + verify(mockService2, times(1)).close(); + } + + @Test + public void embeddedTimelineServerNotReusedIfReuseDisabled() throws Exception { + HoodieEngineContext engineContext = new HoodieLocalEngineContext(new Configuration()); + HoodieWriteConfig writeConfig1 = HoodieWriteConfig.newBuilder() + .withPath(tempDir.resolve("table1").toString()) + .withEmbeddedTimelineServerEnabled(true) + .withEmbeddedTimelineServerReuseEnabled(true) + .build(); + EmbeddedTimelineService.TimelineServiceCreator mockCreator = Mockito.mock(EmbeddedTimelineService.TimelineServiceCreator.class); + TimelineService mockService = Mockito.mock(TimelineService.class); + when(mockCreator.create(any(), any(), any(), any(), any())).thenReturn(mockService); + when(mockService.startService()).thenReturn(789); + EmbeddedTimelineService service1 = EmbeddedTimelineService.getOrStartEmbeddedTimelineService(engineContext, null, writeConfig1, mockCreator); + + HoodieWriteConfig writeConfig2 = HoodieWriteConfig.newBuilder() + .withPath(tempDir.resolve("table2").toString()) + .withEmbeddedTimelineServerEnabled(true) + .withEmbeddedTimelineServerReuseEnabled(false) + .build(); + EmbeddedTimelineService.TimelineServiceCreator mockCreator2 = Mockito.mock(EmbeddedTimelineService.TimelineServiceCreator.class); + TimelineService mockService2 = Mockito.mock(TimelineService.class); + when(mockCreator2.create(any(), any(), any(), any(), any())).thenReturn(mockService2); + when(mockService2.startService()).thenReturn(987); + EmbeddedTimelineService service2 = EmbeddedTimelineService.getOrStartEmbeddedTimelineService(engineContext, null, writeConfig2, mockCreator2); + assertNotSame(service1, service2); + + // test shutdown happens immediately since each server has only one path associated with it + service1.stopForBasePath(writeConfig1.getBasePath()); + verify(mockService, times(1)).unregisterBasePath(writeConfig1.getBasePath()); + verify(mockService, times(1)).close(); + + service2.stopForBasePath(writeConfig2.getBasePath()); + verify(mockService2, times(1)).unregisterBasePath(writeConfig2.getBasePath()); + verify(mockService2, times(1)).close(); + } + + @Test + public void embeddedTimelineServerIsNotReusedAfterStopped() throws Exception { + HoodieEngineContext engineContext = new HoodieLocalEngineContext(new Configuration()); + HoodieWriteConfig writeConfig1 = HoodieWriteConfig.newBuilder() + .withPath(tempDir.resolve("table1").toString()) + .withEmbeddedTimelineServerEnabled(true) + .withEmbeddedTimelineServerReuseEnabled(true) + .build(); + EmbeddedTimelineService.TimelineServiceCreator mockCreator = Mockito.mock(EmbeddedTimelineService.TimelineServiceCreator.class); + TimelineService mockService = Mockito.mock(TimelineService.class); + when(mockCreator.create(any(), any(), any(), any(), any())).thenReturn(mockService); + when(mockService.startService()).thenReturn(555); + EmbeddedTimelineService service1 = EmbeddedTimelineService.getOrStartEmbeddedTimelineService(engineContext, null, writeConfig1, mockCreator); + + service1.stopForBasePath(writeConfig1.getBasePath()); + + HoodieWriteConfig writeConfig2 = HoodieWriteConfig.newBuilder() + .withPath(tempDir.resolve("table2").toString()) + .withEmbeddedTimelineServerEnabled(true) + .withEmbeddedTimelineServerReuseEnabled(true) + .build(); + EmbeddedTimelineService.TimelineServiceCreator mockCreator2 = Mockito.mock(EmbeddedTimelineService.TimelineServiceCreator.class); + TimelineService mockService2 = Mockito.mock(TimelineService.class); + when(mockCreator2.create(any(), any(), any(), any(), any())).thenReturn(mockService2); + when(mockService2.startService()).thenReturn(111); + EmbeddedTimelineService service2 = EmbeddedTimelineService.getOrStartEmbeddedTimelineService(engineContext, null, writeConfig2, mockCreator2); + // a new service will be started since the original was shutdown already + assertNotSame(service1, service2); + + // test shutdown happens immediately since each server has only one path associated with it + service1.stopForBasePath(writeConfig1.getBasePath()); + verify(mockService, times(1)).unregisterBasePath(writeConfig1.getBasePath()); + verify(mockService, times(1)).close(); + + service2.stopForBasePath(writeConfig2.getBasePath()); + verify(mockService2, times(1)).unregisterBasePath(writeConfig2.getBasePath()); + verify(mockService2, times(1)).close(); + } +} \ No newline at end of file diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestHoodieJavaWriteClientInsert.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestHoodieJavaWriteClientInsert.java index ea13939ad2e6..f6d984f83f8c 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestHoodieJavaWriteClientInsert.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestHoodieJavaWriteClientInsert.java @@ -116,9 +116,7 @@ public void testWriteClientAndTableServiceClientWithTimelineServer( HoodieJavaWriteClient writeClient; if (passInTimelineServer) { - EmbeddedTimelineService timelineService = - new EmbeddedTimelineService(context, null, writeConfig); - timelineService.startServer(); + EmbeddedTimelineService timelineService = EmbeddedTimelineService.getOrStartEmbeddedTimelineService(context, null, writeConfig); writeConfig.setViewStorageConfig(timelineService.getRemoteFileSystemViewConfig()); writeClient = new HoodieJavaWriteClient(context, writeConfig, true, Option.of(timelineService)); // Both the write client and the table service client should use the same passed-in @@ -127,7 +125,7 @@ public void testWriteClientAndTableServiceClientWithTimelineServer( assertEquals(timelineService, writeClient.getTableServiceClient().getTimelineServer().get()); // Write config should not be changed assertEquals(writeConfig, writeClient.getConfig()); - timelineService.stop(); + timelineService.stopForBasePath(writeConfig.getBasePath()); } else { writeClient = new HoodieJavaWriteClient(context, writeConfig); // Only one timeline server should be instantiated, and the same timeline server diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperKeyGenerator.java index ce767665a6f9..5b8287c58d40 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/AutoRecordGenWrapperKeyGenerator.java @@ -47,62 +47,76 @@ * PartitionId refers to spark's partition Id. * RowId refers to the row index within the spark partition. */ -public class AutoRecordGenWrapperKeyGenerator extends BuiltinKeyGenerator { +public class AutoRecordGenWrapperKeyGenerator extends BuiltinKeyGenerator implements AutoRecordKeyGeneratorWrapper { - private final BuiltinKeyGenerator builtinKeyGenerator; - private final int partitionId; - private final String instantTime; + private final BuiltinKeyGenerator keyGenerator; + private Integer partitionId; + private String instantTime; private int rowId; - public AutoRecordGenWrapperKeyGenerator(TypedProperties config, BuiltinKeyGenerator builtinKeyGenerator) { + public AutoRecordGenWrapperKeyGenerator(TypedProperties config, BuiltinKeyGenerator keyGenerator) { super(config); - this.builtinKeyGenerator = builtinKeyGenerator; + this.keyGenerator = keyGenerator; this.rowId = 0; - this.partitionId = config.getInteger(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG); - this.instantTime = config.getString(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG); + partitionId = null; + instantTime = null; } @Override public String getRecordKey(GenericRecord record) { - return HoodieRecord.generateSequenceId(instantTime, partitionId, rowId++); + return generateSequenceId(rowId++); } @Override public String getPartitionPath(GenericRecord record) { - return builtinKeyGenerator.getPartitionPath(record); + return keyGenerator.getPartitionPath(record); } @Override public String getRecordKey(Row row) { - return HoodieRecord.generateSequenceId(instantTime, partitionId, rowId++); + return generateSequenceId(rowId++); } @Override public UTF8String getRecordKey(InternalRow internalRow, StructType schema) { - return UTF8String.fromString(HoodieRecord.generateSequenceId(instantTime, partitionId, rowId++)); + return UTF8String.fromString(generateSequenceId(rowId++)); } @Override public String getPartitionPath(Row row) { - return builtinKeyGenerator.getPartitionPath(row); + return keyGenerator.getPartitionPath(row); } @Override public UTF8String getPartitionPath(InternalRow internalRow, StructType schema) { - return builtinKeyGenerator.getPartitionPath(internalRow, schema); + return keyGenerator.getPartitionPath(internalRow, schema); } @Override public List getRecordKeyFieldNames() { - return builtinKeyGenerator.getRecordKeyFieldNames(); + return keyGenerator.getRecordKeyFieldNames(); } public List getPartitionPathFields() { - return builtinKeyGenerator.getPartitionPathFields(); + return keyGenerator.getPartitionPathFields(); } public boolean isConsistentLogicalTimestampEnabled() { - return builtinKeyGenerator.isConsistentLogicalTimestampEnabled(); + return keyGenerator.isConsistentLogicalTimestampEnabled(); } + @Override + public BuiltinKeyGenerator getPartitionKeyGenerator() { + return keyGenerator; + } + + private String generateSequenceId(long recordIndex) { + if (partitionId == null) { + this.partitionId = config.getInteger(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG); + } + if (instantTime == null) { + this.instantTime = config.getString(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG); + } + return HoodieRecord.generateSequenceId(instantTime, partitionId, recordIndex); + } } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala index 818bf7600472..d84679eaf923 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala @@ -18,6 +18,7 @@ package org.apache.hudi +import org.apache.avro.Schema.Type import org.apache.avro.generic.GenericRecord import org.apache.avro.{JsonProperties, Schema} import org.apache.hudi.HoodieSparkUtils.sparkAdapter @@ -242,4 +243,12 @@ object AvroConversionUtils { val nameParts = qualifiedName.split('.') (nameParts.last, nameParts.init.mkString(".")) } + + private def handleUnion(schema: Schema): Schema = { + if (schema.getType == Type.UNION) { + val index = if (schema.getTypes.get(0).getType == Schema.Type.NULL) 1 else 0 + return schema.getTypes.get(index) + } + schema + } } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/SparkKeyGenUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/SparkKeyGenUtils.scala index 932fa0096cf0..7b91ae5a728e 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/SparkKeyGenUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/SparkKeyGenUtils.scala @@ -21,11 +21,8 @@ import org.apache.hudi.common.config.TypedProperties import org.apache.hudi.common.util.StringUtils import org.apache.hudi.common.util.ValidationUtils.checkArgument import org.apache.hudi.keygen.constant.KeyGeneratorOptions -import org.apache.hudi.keygen.{BaseKeyGenerator, CustomAvroKeyGenerator, CustomKeyGenerator, GlobalAvroDeleteKeyGenerator, GlobalDeleteKeyGenerator, KeyGenerator, NonpartitionedAvroKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator} +import org.apache.hudi.keygen.{AutoRecordKeyGeneratorWrapper, AutoRecordGenWrapperKeyGenerator, CustomAvroKeyGenerator, CustomKeyGenerator, GlobalAvroDeleteKeyGenerator, GlobalDeleteKeyGenerator, KeyGenerator, NonpartitionedAvroKeyGenerator, NonpartitionedKeyGenerator} import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory -import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory.getKeyGeneratorClassName - -import scala.collection.JavaConverters._ object SparkKeyGenUtils { @@ -34,26 +31,34 @@ object SparkKeyGenUtils { * @return partition columns */ def getPartitionColumns(props: TypedProperties): String = { - val keyGeneratorClass = getKeyGeneratorClassName(props) - getPartitionColumns(keyGeneratorClass, props) + val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props) + getPartitionColumns(keyGenerator, props) } /** * @param keyGen key generator class name * @return partition columns */ - def getPartitionColumns(keyGenClass: String, typedProperties: TypedProperties): String = { + def getPartitionColumns(keyGenClass: KeyGenerator, typedProperties: TypedProperties): String = { + // For {@link AutoRecordGenWrapperKeyGenerator} or {@link AutoRecordGenWrapperAvroKeyGenerator}, + // get the base key generator for the partition paths + var baseKeyGen = keyGenClass match { + case autoRecordKeyGenerator: AutoRecordKeyGeneratorWrapper => + autoRecordKeyGenerator.getPartitionKeyGenerator + case _ => keyGenClass + } + // For CustomKeyGenerator and CustomAvroKeyGenerator, the partition path filed format // is: "field_name: field_type", we extract the field_name from the partition path field. - if (keyGenClass.equals(classOf[CustomKeyGenerator].getCanonicalName) || keyGenClass.equals(classOf[CustomAvroKeyGenerator].getCanonicalName)) { + if (baseKeyGen.isInstanceOf[CustomKeyGenerator] || baseKeyGen.isInstanceOf[CustomAvroKeyGenerator]) { typedProperties.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()) .split(",").map(pathField => { pathField.split(CustomAvroKeyGenerator.SPLIT_REGEX) - .headOption.getOrElse(s"Illegal partition path field format: '$pathField' for ${keyGenClass}")}).mkString(",") - } else if (keyGenClass.equals(classOf[NonpartitionedKeyGenerator].getCanonicalName) - || keyGenClass.equals(classOf[NonpartitionedAvroKeyGenerator].getCanonicalName) - || keyGenClass.equals(classOf[GlobalDeleteKeyGenerator].getCanonicalName) - || keyGenClass.equals(classOf[GlobalAvroDeleteKeyGenerator].getCanonicalName)) { + .headOption.getOrElse(s"Illegal partition path field format: '$pathField' for ${baseKeyGen}")}).mkString(",") + } else if (baseKeyGen.isInstanceOf[NonpartitionedKeyGenerator] + || baseKeyGen.isInstanceOf[NonpartitionedAvroKeyGenerator] + || baseKeyGen.isInstanceOf[GlobalDeleteKeyGenerator] + || baseKeyGen.isInstanceOf[GlobalAvroDeleteKeyGenerator]) { StringUtils.EMPTY_STRING } else { checkArgument(typedProperties.containsKey(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()), "Partition path needs to be set") diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java index 88ca30bd6e43..d0896645427e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java @@ -164,6 +164,18 @@ private static Iterable providerClassResolutionStrategyAndTableType() return opts; } + @ParameterizedTest + @MethodSource("configParamsDirectBased") + public void testHoodieClientBasicMultiWriterWithEarlyConflictDetectionDirect(String tableType, String earlyConflictDetectionStrategy) throws Exception { + testHoodieClientBasicMultiWriterWithEarlyConflictDetection(tableType, MarkerType.DIRECT.name(), earlyConflictDetectionStrategy); + } + + @ParameterizedTest + @MethodSource("configParamsTimelineServerBased") + public void testHoodieClientBasicMultiWriterWithEarlyConflictDetectionTimelineServerBased(String tableType, String earlyConflictDetectionStrategy) throws Exception { + testHoodieClientBasicMultiWriterWithEarlyConflictDetection(tableType, MarkerType.TIMELINE_SERVER_BASED.name(), earlyConflictDetectionStrategy); + } + /** * Test multi-writers with early conflict detect enable, including * 1. MOR + Direct marker @@ -184,9 +196,7 @@ private static Iterable providerClassResolutionStrategyAndTableType() * @param markerType * @throws Exception */ - @ParameterizedTest - @MethodSource("configParams") - public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String tableType, String markerType, String earlyConflictDetectionStrategy) throws Exception { + private void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String tableType, String markerType, String earlyConflictDetectionStrategy) throws Exception { if (tableType.equalsIgnoreCase(HoodieTableType.MERGE_ON_READ.name())) { setUpMORTestTable(); } @@ -955,14 +965,21 @@ private JavaRDD startCommitForUpdate(HoodieWriteConfig writeConfig, return result; } - public static Stream configParams() { + public static Stream configParamsTimelineServerBased() { + Object[][] data = + new Object[][] { + {"COPY_ON_WRITE", AsyncTimelineServerBasedDetectionStrategy.class.getName()}, + {"MERGE_ON_READ", AsyncTimelineServerBasedDetectionStrategy.class.getName()} + }; + return Stream.of(data).map(Arguments::of); + } + + public static Stream configParamsDirectBased() { Object[][] data = new Object[][] { - {"COPY_ON_WRITE", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineServerBasedDetectionStrategy.class.getName()}, - {"MERGE_ON_READ", MarkerType.TIMELINE_SERVER_BASED.name(), AsyncTimelineServerBasedDetectionStrategy.class.getName()}, - {"MERGE_ON_READ", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedDetectionStrategy.class.getName()}, - {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleDirectMarkerBasedDetectionStrategy.class.getName()}, - {"COPY_ON_WRITE", MarkerType.DIRECT.name(), SimpleTransactionDirectMarkerBasedDetectionStrategy.class.getName()} + {"MERGE_ON_READ", SimpleDirectMarkerBasedDetectionStrategy.class.getName()}, + {"COPY_ON_WRITE", SimpleDirectMarkerBasedDetectionStrategy.class.getName()}, + {"COPY_ON_WRITE", SimpleTransactionDirectMarkerBasedDetectionStrategy.class.getName()} }; return Stream.of(data).map(Arguments::of); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDWriteClient.java index 9cffce2b07bb..784c3a3b7844 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDWriteClient.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDWriteClient.java @@ -84,9 +84,7 @@ public void testWriteClientAndTableServiceClientWithTimelineServer( SparkRDDWriteClient writeClient; if (passInTimelineServer) { - EmbeddedTimelineService timelineService = - new EmbeddedTimelineService(context(), null, writeConfig); - timelineService.startServer(); + EmbeddedTimelineService timelineService = EmbeddedTimelineService.getOrStartEmbeddedTimelineService(context(), null, writeConfig); writeConfig.setViewStorageConfig(timelineService.getRemoteFileSystemViewConfig()); writeClient = new SparkRDDWriteClient(context(), writeConfig, Option.of(timelineService)); // Both the write client and the table service client should use the same passed-in @@ -95,7 +93,7 @@ public void testWriteClientAndTableServiceClientWithTimelineServer( assertEquals(timelineService, writeClient.getTableServiceClient().getTimelineServer().get()); // Write config should not be changed assertEquals(writeConfig, writeClient.getConfig()); - timelineService.stop(); + timelineService.stopForBasePath(writeConfig.getBasePath()); } else { writeClient = new SparkRDDWriteClient(context(), writeConfig); // Only one timeline server should be instantiated, and the same timeline server diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java index 3b4a39c72c07..86cc078e9894 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java @@ -53,7 +53,7 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; +import org.junit.jupiter.params.provider.EnumSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,7 +86,6 @@ public void setUp() throws Exception { initPath(); initSparkContexts(); initFileSystem(); - initTimelineService(); dataGen = new HoodieTestDataGenerator(0x1f86); } @@ -129,30 +128,46 @@ public void initTimelineService() { } } + private enum TestCase { + USE_EXISTING_TIMELINE_SERVER(true, false), + EMBEDDED_TIMELINE_SERVER_PER_TABLE(false, false), + SINGLE_EMBEDDED_TIMELINE_SERVER(false, true); + + private final boolean useExistingTimelineServer; + private final boolean reuseTimelineServer; + + TestCase(boolean useExistingTimelineServer, boolean reuseTimelineServer) { + this.useExistingTimelineServer = useExistingTimelineServer; + this.reuseTimelineServer = reuseTimelineServer; + } + } + @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testMORGetLatestFileSliceWithMetadataTable(boolean useExistingTimelineServer) throws IOException { + @EnumSource(value = TestCase.class) + public void testMORGetLatestFileSliceWithMetadataTable(TestCase testCase) throws IOException { + if (testCase.useExistingTimelineServer) { + initTimelineService(); + } // This test utilizes the `HoodieBackedTestDelayedTableMetadata` to make sure the // synced file system view is always served. // Create two tables to guarantee the timeline server can properly handle multiple base paths with metadata table enabled String basePathStr1 = initializeTable("dataset1"); String basePathStr2 = initializeTable("dataset2"); - try (SparkRDDWriteClient writeClient1 = createWriteClient(basePathStr1, "test_mor_table1", - useExistingTimelineServer ? Option.of(timelineService) : Option.empty()); - SparkRDDWriteClient writeClient2 = createWriteClient(basePathStr2, "test_mor_table2", - useExistingTimelineServer ? Option.of(timelineService) : Option.empty())) { + try (SparkRDDWriteClient writeClient1 = createWriteClient(basePathStr1, "test_mor_table1", testCase.reuseTimelineServer, + testCase.useExistingTimelineServer ? Option.of(timelineService) : Option.empty()); + SparkRDDWriteClient writeClient2 = createWriteClient(basePathStr2, "test_mor_table2", testCase.reuseTimelineServer, + testCase.useExistingTimelineServer ? Option.of(timelineService) : Option.empty())) { for (int i = 0; i < 3; i++) { writeToTable(i, writeClient1); } - for (int i = 0; i < 3; i++) { writeToTable(i, writeClient2); } - runAssertionsForBasePath(useExistingTimelineServer, basePathStr1, writeClient1); - runAssertionsForBasePath(useExistingTimelineServer, basePathStr2, writeClient2); + runAssertionsForBasePath(testCase.useExistingTimelineServer, basePathStr1, writeClient1); + runAssertionsForBasePath(testCase.useExistingTimelineServer, basePathStr2, writeClient2); } } @@ -229,7 +244,7 @@ protected HoodieTableType getTableType() { return HoodieTableType.MERGE_ON_READ; } - private SparkRDDWriteClient createWriteClient(String basePath, String tableName, Option timelineService) { + private SparkRDDWriteClient createWriteClient(String basePath, String tableName, boolean reuseTimelineServer, Option timelineService) { HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() .withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) @@ -247,6 +262,7 @@ private SparkRDDWriteClient createWriteClient(String basePath, String tableName, .withRemoteServerPort(timelineService.isPresent() ? timelineService.get().getServerPort() : REMOTE_PORT_NUM.defaultValue()) .build()) + .withEmbeddedTimelineServerReuseEnabled(reuseTimelineServer) .withAutoCommit(false) .forTable(tableName) .build(); @@ -302,4 +318,4 @@ public Boolean call() throws Exception { return result; } } -} +} \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java index 6588f6d63079..03ec64c33860 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -92,7 +92,7 @@ private Runnable getPreExecuteRunnable() { @SuppressWarnings("unchecked") @Test @Timeout(value = 60) - public void testRecordReading() throws Exception { + public void testRecordReading() { final List hoodieRecords = dataGen.generateInserts(instantTime, 100); ArrayList beforeRecord = new ArrayList<>(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java index be8702582299..c03b08dcdec3 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; +import org.apache.hudi.client.BaseHoodieWriteClient; import org.apache.hudi.client.timeline.HoodieTimelineArchiver; import org.apache.hudi.client.timeline.LSMTimelineWriter; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; @@ -56,10 +57,12 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.metrics.HoodieMetricsConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; +import org.apache.hudi.metrics.HoodieMetrics; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieSparkClientTestHarness; @@ -109,6 +112,9 @@ import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; import static org.apache.hudi.config.HoodieArchivalConfig.ARCHIVE_BEYOND_SAVEPOINT; import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP; +import static org.apache.hudi.metrics.HoodieMetrics.ARCHIVE_ACTION; +import static org.apache.hudi.metrics.HoodieMetrics.DELETE_INSTANTS_NUM_STR; +import static org.apache.hudi.metrics.HoodieMetrics.DURATION_STR; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -256,8 +262,8 @@ public void testArchiveEmptyTable() throws Exception { metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table); - boolean result = archiver.archiveIfRequired(context); - assertTrue(result); + int result = archiver.archiveIfRequired(context); + assertEquals(0, result); } @ParameterizedTest @@ -767,7 +773,7 @@ public void testArchiveCommitSavepointNoHole(boolean enableMetadataTable, boolea HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match"); - assertTrue(archiver.archiveIfRequired(context)); + archiver.archiveIfRequired(context); timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); if (archiveBeyondSavepoint) { // commits in active timeline = 101 and 105. @@ -953,8 +959,7 @@ public void testArchiveCommitTimeline(boolean enableMetadataTable) throws Except HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table); - boolean result = archiver.archiveIfRequired(context); - assertTrue(result); + archiver.archiveIfRequired(context); HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline(); List archivedInstants = Arrays.asList(instant1, instant2, instant3); assertEquals(new HashSet<>(archivedInstants), @@ -962,6 +967,25 @@ public void testArchiveCommitTimeline(boolean enableMetadataTable) throws Except assertFalse(wrapperFs.exists(markerPath)); } + @Test + public void testArchiveMetrics() throws Exception { + init(); + HoodieWriteConfig cfg = + HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetricsConfig(HoodieMetricsConfig + .newBuilder() + .on(true) + .withReporterType("INMEMORY") + .build()) + .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2).forTable("test-trip-table").build(); + HoodieMetrics metrics = new HoodieMetrics(cfg); + BaseHoodieWriteClient client = getHoodieWriteClient(cfg); + client.archive(); + assertTrue(metrics.getMetrics().getRegistry().getNames().contains(metrics.getMetricsName(ARCHIVE_ACTION, DURATION_STR))); + assertTrue(metrics.getMetrics().getRegistry().getNames().contains(metrics.getMetricsName(ARCHIVE_ACTION, DELETE_INSTANTS_NUM_STR))); + } + private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expectedTotalInstants) { HoodieTimeline timeline = metaClient.getActiveTimeline().reload() .getTimelineOfActions(Collections.singleton(HoodieTimeline.CLEAN_ACTION)).filterInflights(); @@ -1360,10 +1384,9 @@ public void testGetCommitInstantsToArchiveDuringInflightCommits() throws Excepti // Run archival HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table); - boolean result = archiver.archiveIfRequired(context); + archiver.archiveIfRequired(context); expectedInstants.remove("1000"); expectedInstants.remove("1001"); - assertTrue(result); timeline = metaClient.reloadActiveTimeline().getWriteTimeline(); // Check the count of instants after archive it should have 2 less instants @@ -1383,7 +1406,7 @@ public void testGetCommitInstantsToArchiveDuringInflightCommits() throws Excepti metaClient.reloadActiveTimeline(); // Run archival - assertTrue(archiver.archiveIfRequired(context)); + archiver.archiveIfRequired(context); timeline = metaClient.reloadActiveTimeline().getWriteTimeline(); expectedInstants.removeAll(Arrays.asList("1002", "1003", "1004", "1005")); @@ -1417,8 +1440,7 @@ public void testWithOldestReplaceCommit() throws Exception { HoodieTimeline timeline = metaClient.reloadActiveTimeline(); assertEquals(9, timeline.countInstants(), "Loaded 9 commits and the count should match"); - boolean result = archiver.archiveIfRequired(context); - assertTrue(result); + archiver.archiveIfRequired(context); timeline = metaClient.reloadActiveTimeline(); assertEquals(9, timeline.countInstants(), "Since we have a pending replacecommit at 1001, we should never archive any commit after 1001"); diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index b217d892eb61..b5fbad8e0b5e 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -127,6 +127,16 @@ com.fasterxml.jackson.datatype jackson-datatype-jsr310 + + com.fasterxml.jackson.module + jackson-module-afterburner + + + + + com.fasterxml.jackson.module + jackson-module-afterburner + diff --git a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java index 7ba20795790e..824a94abab4b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java @@ -29,10 +29,12 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -82,7 +84,10 @@ public abstract class BaseHoodieTableFileIndex implements AutoCloseable { protected final HoodieMetadataConfig metadataConfig; + private final HoodieTableQueryType queryType; private final Option specifiedQueryInstant; + private final Option beginInstantTime; + private final Option endInstantTime; private final List queryPaths; private final boolean shouldIncludePendingCommits; @@ -123,6 +128,8 @@ public abstract class BaseHoodieTableFileIndex implements AutoCloseable { * @param shouldIncludePendingCommits flags whether file-index should exclude any pending operations * @param shouldValidateInstant flags to validate whether query instant is present in the timeline * @param fileStatusCache transient cache of fetched [[FileStatus]]es + * @param beginInstantTime begin instant time for incremental query (optional) + * @param endInstantTime end instant time for incremental query (optional) */ public BaseHoodieTableFileIndex(HoodieEngineContext engineContext, HoodieTableMetaClient metaClient, @@ -133,7 +140,9 @@ public BaseHoodieTableFileIndex(HoodieEngineContext engineContext, boolean shouldIncludePendingCommits, boolean shouldValidateInstant, FileStatusCache fileStatusCache, - boolean shouldListLazily) { + boolean shouldListLazily, + Option beginInstantTime, + Option endInstantTime) { this.partitionColumns = metaClient.getTableConfig().getPartitionFields() .orElse(new String[0]); @@ -143,11 +152,14 @@ public BaseHoodieTableFileIndex(HoodieEngineContext engineContext, && HoodieTableMetadataUtil.isFilesPartitionAvailable(metaClient)) .build(); + this.queryType = queryType; this.queryPaths = queryPaths; this.specifiedQueryInstant = specifiedQueryInstant; this.shouldIncludePendingCommits = shouldIncludePendingCommits; this.shouldValidateInstant = shouldValidateInstant; this.shouldListLazily = shouldListLazily; + this.beginInstantTime = beginInstantTime; + this.endInstantTime = endInstantTime; this.basePath = metaClient.getBasePathV2(); @@ -300,7 +312,17 @@ protected List listPartitionPaths(List relativePartitionP protected List listPartitionPaths(List relativePartitionPaths) { List matchedPartitionPaths; try { - matchedPartitionPaths = tableMetadata.getPartitionPathWithPathPrefixes(relativePartitionPaths); + if (isPartitionedTable()) { + if (queryType == HoodieTableQueryType.INCREMENTAL && beginInstantTime.isPresent()) { + HoodieTimeline timelineAfterBeginInstant = TimelineUtils.getCommitsTimelineAfter(metaClient, beginInstantTime.get(), Option.empty()); + HoodieTimeline timelineToQuery = endInstantTime.map(timelineAfterBeginInstant::findInstantsBeforeOrEquals).orElse(timelineAfterBeginInstant); + matchedPartitionPaths = TimelineUtils.getWrittenPartitions(timelineToQuery); + } else { + matchedPartitionPaths = tableMetadata.getPartitionPathWithPathPrefixes(relativePartitionPaths); + } + } else { + matchedPartitionPaths = Collections.singletonList(StringUtils.EMPTY_STRING); + } } catch (IOException e) { throw new HoodieIOException("Error fetching partition paths", e); } @@ -319,6 +341,10 @@ protected void refresh() { doRefresh(); } + private boolean isPartitionedTable() { + return partitionColumns.length > 0 || HoodieTableMetadata.isMetadataTable(basePath.toString()); + } + protected HoodieTimeline getActiveTimeline() { // NOTE: We have to use commits and compactions timeline, to make sure that we're properly // handling the following case: when records are inserted into the new log-file w/in the file-group @@ -402,6 +428,8 @@ private void doRefresh() { // Reset it to null to trigger re-loading of all partition path this.cachedAllPartitionPaths = null; + // Reset to force reload file slices inside partitions + this.cachedAllInputFileSlices = new HashMap<>(); if (!shouldListLazily) { ensurePreloadedPartitions(getAllQueryPartitionPaths()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java index fcfc8a4f0b9f..3c5486c47c74 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java @@ -249,6 +249,11 @@ public static Schema resolveUnionSchema(Schema schema, String fieldSchemaFullNam } List innerTypes = schema.getTypes(); + if (innerTypes.size() == 2 && isNullable(schema)) { + // this is a basic nullable field so handle it more efficiently + return resolveNullableSchema(schema); + } + Schema nonNullType = innerTypes.stream() .filter(it -> it.getType() != Schema.Type.NULL && Objects.equals(it.getFullName(), fieldSchemaFullName)) @@ -286,18 +291,19 @@ public static Schema resolveNullableSchema(Schema schema) { } List innerTypes = schema.getTypes(); - Schema nonNullType = - innerTypes.stream() - .filter(it -> it.getType() != Schema.Type.NULL) - .findFirst() - .orElse(null); - if (innerTypes.size() != 2 || nonNullType == null) { + if (innerTypes.size() != 2) { throw new AvroRuntimeException( String.format("Unsupported Avro UNION type %s: Only UNION of a null type and a non-null type is supported", schema)); } - - return nonNullType; + Schema firstInnerType = innerTypes.get(0); + Schema secondInnerType = innerTypes.get(1); + if ((firstInnerType.getType() != Schema.Type.NULL && secondInnerType.getType() != Schema.Type.NULL) + || (firstInnerType.getType() == Schema.Type.NULL && secondInnerType.getType() == Schema.Type.NULL)) { + throw new AvroRuntimeException( + String.format("Unsupported Avro UNION type %s: Only UNION of a null type and a non-null type is supported", schema)); + } + return firstInnerType.getType() == Schema.Type.NULL ? secondInnerType : firstInnerType; } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index 54c37b333e74..3800d9c10531 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java @@ -265,7 +265,8 @@ public static Schema addMetadataFields(Schema schema) { * @param withOperationField Whether to include the '_hoodie_operation' field */ public static Schema addMetadataFields(Schema schema, boolean withOperationField) { - List parentFields = new ArrayList<>(); + int newFieldsSize = HoodieRecord.HOODIE_META_COLUMNS.size() + (withOperationField ? 1 : 0); + List parentFields = new ArrayList<>(schema.getFields().size() + newFieldsSize); Schema.Field commitTimeField = new Schema.Field(HoodieRecord.COMMIT_TIME_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", JsonProperties.NULL_VALUE); @@ -439,12 +440,6 @@ public static GenericRecord rewriteRecord(GenericRecord oldRecord, Schema newSch copyOldValueOrSetDefault(oldRecord, newRecord, f); } } - - if (!ConvertingGenericData.INSTANCE.validate(newSchema, newRecord)) { - throw new SchemaCompatibilityException( - "Unable to validate the rewritten record " + oldRecord + " against schema " + newSchema); - } - return newRecord; } @@ -455,10 +450,6 @@ public static GenericRecord rewriteRecordWithMetadata(GenericRecord genericRecor } // do not preserve FILENAME_METADATA_FIELD newRecord.put(HoodieRecord.FILENAME_META_FIELD_ORD, fileName); - if (!GenericData.get().validate(newSchema, newRecord)) { - throw new SchemaCompatibilityException( - "Unable to validate the rewritten record " + genericRecord + " against schema " + newSchema); - } return newRecord; } @@ -494,7 +485,7 @@ public static GenericRecord removeFields(GenericRecord record, Set field private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field field) { Schema oldSchema = oldRecord.getSchema(); Field oldSchemaField = oldSchema.getField(field.name()); - Object fieldValue = oldSchemaField == null ? null : oldRecord.get(field.name()); + Object fieldValue = oldSchemaField == null ? null : oldRecord.get(oldSchemaField.pos()); if (fieldValue != null) { // In case field's value is a nested record, we have to rewrite it as well @@ -508,11 +499,14 @@ private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRec } else { newFieldValue = fieldValue; } - newRecord.put(field.name(), newFieldValue); + newRecord.put(field.pos(), newFieldValue); } else if (field.defaultVal() instanceof JsonProperties.Null) { - newRecord.put(field.name(), null); + newRecord.put(field.pos(), null); } else { - newRecord.put(field.name(), field.defaultVal()); + if (!isNullable(field.schema()) && field.defaultVal() == null) { + throw new SchemaCompatibilityException("Field " + field.name() + " has no default value and is null in old record"); + } + newRecord.put(field.pos(), field.defaultVal()); } } @@ -562,7 +556,8 @@ public static Object getFieldVal(GenericRecord record, String key) { * it is consistent with avro after 1.10 */ public static Object getFieldVal(GenericRecord record, String key, boolean returnNullIfNotFound) { - if (record.getSchema().getField(key) == null) { + Schema.Field field = record.getSchema().getField(key); + if (field == null) { if (returnNullIfNotFound) { return null; } else { @@ -572,7 +567,7 @@ public static Object getFieldVal(GenericRecord record, String key, boolean retur throw new AvroRuntimeException("Not a valid schema field: " + key); } } else { - return record.get(key); + return record.get(field.pos()); } } @@ -874,7 +869,8 @@ private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldAvr } // try to get real schema for union type Schema oldSchema = getActualSchemaFromUnion(oldAvroSchema, oldRecord); - Object newRecord = rewriteRecordWithNewSchemaInternal(oldRecord, oldSchema, newSchema, renameCols, fieldNames, validate); + Object newRecord = rewriteRecordWithNewSchemaInternal(oldRecord, oldSchema, newSchema, renameCols, fieldNames); + // validation is recursive so it only needs to be called on the original input if (validate && !ConvertingGenericData.INSTANCE.validate(newSchema, newRecord)) { throw new SchemaCompatibilityException( "Unable to validate the rewritten record " + oldRecord + " against schema " + newSchema); @@ -882,7 +878,7 @@ private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldAvr return newRecord; } - private static Object rewriteRecordWithNewSchemaInternal(Object oldRecord, Schema oldSchema, Schema newSchema, Map renameCols, Deque fieldNames, boolean validate) { + private static Object rewriteRecordWithNewSchemaInternal(Object oldRecord, Schema oldSchema, Schema newSchema, Map renameCols, Deque fieldNames) { switch (newSchema.getType()) { case RECORD: ValidationUtils.checkArgument(oldRecord instanceof IndexedRecord, "cannot rewrite record with different type"); @@ -893,17 +889,17 @@ private static Object rewriteRecordWithNewSchemaInternal(Object oldRecord, Schem Schema.Field field = fields.get(i); String fieldName = field.name(); fieldNames.push(fieldName); - if (oldSchema.getField(field.name()) != null && !renameCols.containsKey(field.name())) { - Schema.Field oldField = oldSchema.getField(field.name()); - newRecord.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema(), renameCols, fieldNames, validate)); + Schema.Field oldField = oldSchema.getField(field.name()); + if (oldField != null && !renameCols.containsKey(field.name())) { + newRecord.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema(), renameCols, fieldNames, false)); } else { String fieldFullName = createFullName(fieldNames); - String fieldNameFromOldSchema = renameCols.getOrDefault(fieldFullName, ""); + String fieldNameFromOldSchema = renameCols.get(fieldFullName); // deal with rename - if (oldSchema.getField(fieldNameFromOldSchema) != null) { + Schema.Field oldFieldRenamed = fieldNameFromOldSchema == null ? null : oldSchema.getField(fieldNameFromOldSchema); + if (oldFieldRenamed != null) { // find rename - Schema.Field oldField = oldSchema.getField(fieldNameFromOldSchema); - newRecord.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema(), renameCols, fieldNames, validate)); + newRecord.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldFieldRenamed.pos()), oldFieldRenamed.schema(), fields.get(i).schema(), renameCols, fieldNames, false)); } else { // deal with default value if (fields.get(i).defaultVal() instanceof JsonProperties.Null) { @@ -927,25 +923,25 @@ private static Object rewriteRecordWithNewSchemaInternal(Object oldRecord, Schem case ARRAY: ValidationUtils.checkArgument(oldRecord instanceof Collection, "cannot rewrite record with different type"); Collection array = (Collection) oldRecord; - List newArray = new ArrayList(array.size()); + List newArray = new ArrayList<>(array.size()); fieldNames.push("element"); for (Object element : array) { - newArray.add(rewriteRecordWithNewSchema(element, oldSchema.getElementType(), newSchema.getElementType(), renameCols, fieldNames, validate)); + newArray.add(rewriteRecordWithNewSchema(element, oldSchema.getElementType(), newSchema.getElementType(), renameCols, fieldNames, false)); } fieldNames.pop(); return newArray; case MAP: ValidationUtils.checkArgument(oldRecord instanceof Map, "cannot rewrite record with different type"); Map map = (Map) oldRecord; - Map newMap = new HashMap<>(map.size(), 1); + Map newMap = new HashMap<>(map.size(), 1.0f); fieldNames.push("value"); for (Map.Entry entry : map.entrySet()) { - newMap.put(entry.getKey(), rewriteRecordWithNewSchema(entry.getValue(), oldSchema.getValueType(), newSchema.getValueType(), renameCols, fieldNames, validate)); + newMap.put(entry.getKey(), rewriteRecordWithNewSchema(entry.getValue(), oldSchema.getValueType(), newSchema.getValueType(), renameCols, fieldNames, false)); } fieldNames.pop(); return newMap; case UNION: - return rewriteRecordWithNewSchema(oldRecord, getActualSchemaFromUnion(oldSchema, oldRecord), getActualSchemaFromUnion(newSchema, oldRecord), renameCols, fieldNames, validate); + return rewriteRecordWithNewSchema(oldRecord, getActualSchemaFromUnion(oldSchema, oldRecord), getActualSchemaFromUnion(newSchema, oldRecord), renameCols, fieldNames, false); default: return rewritePrimaryType(oldRecord, oldSchema, newSchema); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java b/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java index 3db8210cadee..86b7f4cc4573 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java @@ -18,6 +18,7 @@ package org.apache.hudi.common.config; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import java.io.Serializable; @@ -78,6 +79,10 @@ public String getString(String property, String defaultValue) { return containsKey(property) ? getProperty(property) : defaultValue; } + public Option getNonEmptyStringOpt(String property, String defaultValue) { + return Option.ofNullable(StringUtils.emptyToNull(getString(property, defaultValue))); + } + public List getStringList(String property, String delimiter, List defaultVal) { if (!containsKey(property)) { return defaultVal; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java index 954fe75a0ac3..770c811a2e8e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -87,6 +87,8 @@ public class FSUtils { private static final int MAX_ATTEMPTS_RECOVER_LEASE = 10; private static final String HOODIE_ENV_PROPS_PREFIX = "HOODIE_ENV_"; + private static final String LOG_FILE_EXTENSION = ".log"; + private static final PathFilter ALLOW_ALL_FILTER = file -> true; public static Configuration prepareHadoopConf(Configuration conf) { @@ -474,8 +476,11 @@ public static boolean isLogFile(Path logPath) { } public static boolean isLogFile(String fileName) { - Matcher matcher = LOG_FILE_PATTERN.matcher(fileName); - return fileName.contains(".log") && matcher.find(); + if (fileName.contains(LOG_FILE_EXTENSION)) { + Matcher matcher = LOG_FILE_PATTERN.matcher(fileName); + return matcher.find(); + } + return false; } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java index 48fa98417ca9..0fc3149bb73f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java @@ -77,7 +77,9 @@ public enum StorageSchemes { // Volcengine Object Storage TOS("tos", null, null), // Volcengine Cloud HDFS - CFS("cfs", null, null); + CFS("cfs", null, null), + // Hopsworks File System + HOPSFS("hopsfs", false, true); private String scheme; // null for uncertain if write is transactional, please update this for each FS diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java index eae2f58af944..daa1dcb0207f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.avro.Schema; @@ -33,6 +34,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; /** * {@link HoodieRecordPayload} impl that honors ordering field in both preCombine and combineAndGetUpdateValue. @@ -44,6 +46,8 @@ public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload { public static final String DELETE_KEY = "hoodie.payload.delete.field"; public static final String DELETE_MARKER = "hoodie.payload.delete.marker"; private Option eventTime = Option.empty(); + private AtomicBoolean isDeleteComputed = new AtomicBoolean(false); + private boolean isDefaultRecordPayloadDeleted = false; public DefaultHoodieRecordPayload(GenericRecord record, Comparable orderingVal) { super(record, orderingVal); @@ -72,10 +76,13 @@ public Option combineAndGetUpdateValue(IndexedRecord currentValue */ eventTime = updateEventTime(incomingRecord, properties); + if (!isDeleteComputed.getAndSet(true)) { + isDefaultRecordPayloadDeleted = isDeleteRecord(incomingRecord, properties); + } /* * Now check if the incoming record is a delete record. */ - return isDeleteRecord(incomingRecord, properties) ? Option.empty() : Option.of(incomingRecord); + return isDefaultRecordPayloadDeleted ? Option.empty() : Option.of(incomingRecord); } @Override @@ -86,7 +93,25 @@ public Option getInsertValue(Schema schema, Properties properties GenericRecord incomingRecord = HoodieAvroUtils.bytesToAvro(recordBytes, schema); eventTime = updateEventTime(incomingRecord, properties); - return isDeleteRecord(incomingRecord, properties) ? Option.empty() : Option.of(incomingRecord); + if (!isDeleteComputed.getAndSet(true)) { + isDefaultRecordPayloadDeleted = isDeleteRecord(incomingRecord, properties); + } + return isDefaultRecordPayloadDeleted ? Option.empty() : Option.of(incomingRecord); + } + + public boolean isDeleted(Schema schema, Properties props) { + if (recordBytes.length == 0) { + return true; + } + try { + if (!isDeleteComputed.getAndSet(true)) { + GenericRecord incomingRecord = HoodieAvroUtils.bytesToAvro(recordBytes, schema); + isDefaultRecordPayloadDeleted = isDeleteRecord(incomingRecord, props); + } + return isDefaultRecordPayloadDeleted; + } catch (IOException e) { + throw new HoodieIOException("Deserializing bytes to avro failed ", e); + } } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java index 9b31a51d9250..02b1ef352515 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java @@ -68,6 +68,7 @@ import java.util.Iterator; import java.util.List; import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Supplier; import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchema; import static org.apache.hudi.avro.AvroSchemaUtils.containsFieldInSchema; @@ -113,8 +114,12 @@ public TableSchemaResolver(HoodieTableMetaClient metaClient) { this.hasOperationField = Lazy.lazily(this::hasOperationField); } - public Schema getTableAvroSchemaFromDataFile() { - return convertParquetSchemaToAvro(getTableParquetSchemaFromDataFile()); + public Schema getTableAvroSchemaFromDataFile() throws Exception { + return getTableAvroSchemaFromDataFileInternal().orElseThrow(schemaNotFoundError()); + } + + private Option getTableAvroSchemaFromDataFileInternal() { + return getTableParquetSchemaFromDataFile().map(this::convertParquetSchemaToAvro); } /** @@ -135,7 +140,7 @@ public Schema getTableAvroSchema() throws Exception { * @throws Exception */ public Schema getTableAvroSchema(boolean includeMetadataFields) throws Exception { - return getTableAvroSchemaInternal(includeMetadataFields, Option.empty()); + return getTableAvroSchemaInternal(includeMetadataFields, Option.empty()).orElseThrow(schemaNotFoundError()); } /** @@ -148,7 +153,8 @@ public Schema getTableAvroSchema(String timestamp) throws Exception { .filterCompletedInstants() .findInstantsBeforeOrEquals(timestamp) .lastInstant(); - return getTableAvroSchemaInternal(metaClient.getTableConfig().populateMetaFields(), instant); + return getTableAvroSchemaInternal(metaClient.getTableConfig().populateMetaFields(), instant) + .orElseThrow(schemaNotFoundError()); } /** @@ -157,7 +163,7 @@ public Schema getTableAvroSchema(String timestamp) throws Exception { * @param instant as of which table's schema will be fetched */ public Schema getTableAvroSchema(HoodieInstant instant, boolean includeMetadataFields) throws Exception { - return getTableAvroSchemaInternal(includeMetadataFields, Option.of(instant)); + return getTableAvroSchemaInternal(includeMetadataFields, Option.of(instant)).orElseThrow(schemaNotFoundError()); } /** @@ -188,11 +194,15 @@ public MessageType getTableParquetSchema(boolean includeMetadataField) throws Ex */ @Deprecated public Schema getTableAvroSchemaWithoutMetadataFields() throws Exception { - return getTableAvroSchema(false); + return getTableAvroSchemaInternal(false, Option.empty()).orElseThrow(schemaNotFoundError()); + } + + public Option getTableAvroSchemaIfPresent(boolean includeMetadataFields) { + return getTableAvroSchemaInternal(includeMetadataFields, Option.empty()); } - private Schema getTableAvroSchemaInternal(boolean includeMetadataFields, Option instantOpt) { - Schema schema = + private Option getTableAvroSchemaInternal(boolean includeMetadataFields, Option instantOpt) { + Option schema = (instantOpt.isPresent() ? getTableSchemaFromCommitMetadata(instantOpt.get(), includeMetadataFields) : getTableSchemaFromLatestCommitMetadata(includeMetadataFields)) @@ -203,18 +213,18 @@ private Schema getTableAvroSchemaInternal(boolean includeMetadataFields, Option< ? HoodieAvroUtils.addMetadataFields(tableSchema, hasOperationField.get()) : tableSchema) ) - .orElseGet(() -> { - Schema schemaFromDataFile = getTableAvroSchemaFromDataFile(); + .or(() -> { + Option schemaFromDataFile = getTableAvroSchemaFromDataFileInternal(); return includeMetadataFields ? schemaFromDataFile - : HoodieAvroUtils.removeMetadataFields(schemaFromDataFile); + : schemaFromDataFile.map(HoodieAvroUtils::removeMetadataFields); }); // TODO partition columns have to be appended in all read-paths - if (metaClient.getTableConfig().shouldDropPartitionColumns()) { + if (metaClient.getTableConfig().shouldDropPartitionColumns() && schema.isPresent()) { return metaClient.getTableConfig().getPartitionFields() - .map(partitionFields -> appendPartitionColumns(schema, Option.ofNullable(partitionFields))) - .orElse(schema); + .map(partitionFields -> appendPartitionColumns(schema.get(), Option.ofNullable(partitionFields))) + .or(() -> schema); } return schema; @@ -257,7 +267,7 @@ private Option getTableSchemaFromCommitMetadata(HoodieInstant instant, b /** * Fetches the schema for a table from any the table's data files */ - private MessageType getTableParquetSchemaFromDataFile() { + private Option getTableParquetSchemaFromDataFile() { Option> instantAndCommitMetadata = getLatestCommitMetadataWithValidData(); try { switch (metaClient.getTableType()) { @@ -270,10 +280,11 @@ private MessageType getTableParquetSchemaFromDataFile() { if (instantAndCommitMetadata.isPresent()) { HoodieCommitMetadata commitMetadata = instantAndCommitMetadata.get().getRight(); Iterator filePaths = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePathV2()).values().iterator(); - return fetchSchemaFromFiles(filePaths); + return Option.of(fetchSchemaFromFiles(filePaths)); } else { - throw new IllegalArgumentException("Could not find any data file written for commit, " + LOG.warn("Could not find any data file written for commit, " + "so could not get schema for table " + metaClient.getBasePath()); + return Option.empty(); } default: LOG.error("Unknown table type " + metaClient.getTableType()); @@ -308,7 +319,7 @@ private MessageType convertAvroSchemaToParquet(Schema schema) { */ public Option getTableAvroSchemaFromLatestCommit(boolean includeMetadataFields) throws Exception { if (metaClient.isTimelineNonEmpty()) { - return Option.of(getTableAvroSchemaInternal(includeMetadataFields, Option.empty())); + return getTableAvroSchemaInternal(includeMetadataFields, Option.empty()); } return Option.empty(); @@ -569,4 +580,8 @@ public static Schema appendPartitionColumns(Schema dataSchema, Option return dataSchema; } + + private Supplier schemaNotFoundError() { + return () -> new IllegalArgumentException("No schema found for table at " + metaClient.getBasePathV2().toString()); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DTOUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DTOUtils.java index ef5a88694876..4399860d6b4b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DTOUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DTOUtils.java @@ -41,9 +41,9 @@ public static List fileGroupDTOsfromFileGroups(List fileGroupDTOS = new ArrayList<>(); + List fileGroupDTOS = new ArrayList<>(fileGroups.size()); fileGroupDTOS.add(FileGroupDTO.fromFileGroup(fileGroups.get(0), true)); - fileGroupDTOS.addAll(fileGroups.subList(1, fileGroups.size()).stream() + fileGroupDTOS.addAll(fileGroups.stream().skip(1) .map(fg -> FileGroupDTO.fromFileGroup(fg, false)).collect(Collectors.toList())); return fileGroupDTOS; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index b3dc0fbce0ad..14f55b76889d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -433,6 +433,19 @@ protected Map, FileStatus[]> listPartitions( return fileStatusMap; } + /** + * Returns all files situated at the given partition. + */ + private FileStatus[] getAllFilesInPartition(String relativePartitionPath) throws IOException { + Path partitionPath = FSUtils.getPartitionPath(metaClient.getBasePathV2(), relativePartitionPath); + long beginLsTs = System.currentTimeMillis(); + FileStatus[] statuses = listPartition(partitionPath); + long endLsTs = System.currentTimeMillis(); + LOG.debug("#files found in partition (" + relativePartitionPath + ") =" + statuses.length + ", Time taken =" + + (endLsTs - beginLsTs)); + return statuses; + } + /** * Allows lazily loading the partitions if needed. * @@ -449,15 +462,7 @@ private void ensurePartitionLoadedCorrectly(String partition) { // Not loaded yet try { LOG.info("Building file system view for partition (" + partitionPathStr + ")"); - - Path partitionPath = FSUtils.getPartitionPath(metaClient.getBasePathV2(), partitionPathStr); - long beginLsTs = System.currentTimeMillis(); - FileStatus[] statuses = listPartition(partitionPath); - long endLsTs = System.currentTimeMillis(); - LOG.debug("#files found in partition (" + partitionPathStr + ") =" + statuses.length + ", Time taken =" - + (endLsTs - beginLsTs)); - List groups = addFilesToView(statuses); - + List groups = addFilesToView(getAllFilesInPartition(partitionPathStr)); if (groups.isEmpty()) { storePartitionView(partitionPathStr, new ArrayList<>()); } @@ -598,24 +603,32 @@ private FileSlice filterUncommittedLogs(FileSlice fileSlice) { } protected HoodieFileGroup addBootstrapBaseFileIfPresent(HoodieFileGroup fileGroup) { + return addBootstrapBaseFileIfPresent(fileGroup, this::getBootstrapBaseFile); + } + + protected HoodieFileGroup addBootstrapBaseFileIfPresent(HoodieFileGroup fileGroup, Function> bootstrapBaseFileMappingFunc) { boolean hasBootstrapBaseFile = fileGroup.getAllFileSlices() .anyMatch(fs -> fs.getBaseInstantTime().equals(METADATA_BOOTSTRAP_INSTANT_TS)); if (hasBootstrapBaseFile) { HoodieFileGroup newFileGroup = new HoodieFileGroup(fileGroup); newFileGroup.getAllFileSlices().filter(fs -> fs.getBaseInstantTime().equals(METADATA_BOOTSTRAP_INSTANT_TS)) .forEach(fs -> fs.setBaseFile( - addBootstrapBaseFileIfPresent(fs.getFileGroupId(), fs.getBaseFile().get()))); + addBootstrapBaseFileIfPresent(fs.getFileGroupId(), fs.getBaseFile().get(), bootstrapBaseFileMappingFunc))); return newFileGroup; } return fileGroup; } protected FileSlice addBootstrapBaseFileIfPresent(FileSlice fileSlice) { + return addBootstrapBaseFileIfPresent(fileSlice, this::getBootstrapBaseFile); + } + + protected FileSlice addBootstrapBaseFileIfPresent(FileSlice fileSlice, Function> bootstrapBaseFileMappingFunc) { if (fileSlice.getBaseInstantTime().equals(METADATA_BOOTSTRAP_INSTANT_TS)) { FileSlice copy = new FileSlice(fileSlice); copy.getBaseFile().ifPresent(dataFile -> { Option edf = getBootstrapBaseFile(copy.getFileGroupId()); - edf.ifPresent(e -> dataFile.setBootstrapBaseFile(e.getBootstrapBaseFile())); + bootstrapBaseFileMappingFunc.apply(copy.getFileGroupId()).ifPresent(e -> dataFile.setBootstrapBaseFile(e.getBootstrapBaseFile())); }); return copy; } @@ -623,10 +636,16 @@ protected FileSlice addBootstrapBaseFileIfPresent(FileSlice fileSlice) { } protected HoodieBaseFile addBootstrapBaseFileIfPresent(HoodieFileGroupId fileGroupId, HoodieBaseFile baseFile) { + return addBootstrapBaseFileIfPresent(fileGroupId, baseFile, this::getBootstrapBaseFile); + } + + protected HoodieBaseFile addBootstrapBaseFileIfPresent( + HoodieFileGroupId fileGroupId, + HoodieBaseFile baseFile, + Function> bootstrapBaseFileMappingFunc) { if (baseFile.getCommitTime().equals(METADATA_BOOTSTRAP_INSTANT_TS)) { HoodieBaseFile copy = new HoodieBaseFile(baseFile); - Option edf = getBootstrapBaseFile(fileGroupId); - edf.ifPresent(e -> copy.setBootstrapBaseFile(e.getBootstrapBaseFile())); + bootstrapBaseFileMappingFunc.apply(fileGroupId).ifPresent(e -> copy.setBootstrapBaseFile(e.getBootstrapBaseFile())); return copy; } return baseFile; @@ -706,7 +725,6 @@ public final Stream getLatestBaseFilesBeforeOrOn(String partitio public final Map> getAllLatestBaseFilesBeforeOrOn(String maxCommitTime) { try { readLock.lock(); - List formattedPartitionList = ensureAllPartitionsLoadedCorrectly(); return formattedPartitionList.stream().collect(Collectors.toMap( Function.identity(), @@ -824,6 +842,31 @@ public final Stream getLatestFileSlices(String partitionStr) { } } + @Override + public final Stream getLatestFileSlicesStateless(String partitionStr) { + String partition = formatPartitionKey(partitionStr); + if (isPartitionAvailableInStore(partition)) { + return getLatestFileSlices(partition); + } else { + try { + Stream fileSliceStream = buildFileGroups(getAllFilesInPartition(partition), visibleCommitsAndCompactionTimeline, true).stream() + .filter(fg -> !isFileGroupReplaced(fg)) + .map(HoodieFileGroup::getLatestFileSlice) + .filter(Option::isPresent).map(Option::get) + .flatMap(slice -> this.filterUncommittedFiles(slice, true)); + if (bootstrapIndex.useIndex()) { + final Map bootstrapBaseFileMappings = getBootstrapBaseFileMappings(partition); + if (!bootstrapBaseFileMappings.isEmpty()) { + return fileSliceStream.map(fileSlice -> addBootstrapBaseFileIfPresent(fileSlice, fileGroupId -> Option.ofNullable(bootstrapBaseFileMappings.get(fileGroupId)))); + } + } + return fileSliceStream; + } catch (IOException e) { + throw new HoodieIOException("Failed to fetch all files in partition " + partition, e); + } + } + } + /** * Get Latest File Slice for a given fileId in a given partition. */ @@ -1014,6 +1057,39 @@ public final Stream getAllFileGroups(String partitionStr) { return getAllFileGroupsIncludingReplaced(partitionStr).filter(fg -> !isFileGroupReplaced(fg)); } + @Override + public final Stream getAllFileGroupsStateless(String partitionStr) { + String partition = formatPartitionKey(partitionStr); + if (isPartitionAvailableInStore(partition)) { + return getAllFileGroups(partition); + } else { + try { + Stream fileGroupStream = buildFileGroups(getAllFilesInPartition(partition), visibleCommitsAndCompactionTimeline, true).stream() + .filter(fg -> !isFileGroupReplaced(fg)); + if (bootstrapIndex.useIndex()) { + final Map bootstrapBaseFileMappings = getBootstrapBaseFileMappings(partition); + if (!bootstrapBaseFileMappings.isEmpty()) { + return fileGroupStream.map(fileGroup -> addBootstrapBaseFileIfPresent(fileGroup, fileGroupId -> Option.ofNullable(bootstrapBaseFileMappings.get(fileGroupId)))); + } + } + return fileGroupStream; + } catch (IOException e) { + throw new HoodieIOException("Failed to fetch all files in partition " + partition, e); + } + } + } + + private Map getBootstrapBaseFileMappings(String partition) { + try (BootstrapIndex.IndexReader reader = bootstrapIndex.createReader()) { + LOG.info("Bootstrap Index available for partition " + partition); + List sourceFileMappings = + reader.getSourceFileMappingForPartition(partition); + return sourceFileMappings.stream() + .map(s -> new BootstrapBaseFileMapping(new HoodieFileGroupId(s.getPartitionPath(), + s.getFileId()), s.getBootstrapFileStatus())).collect(Collectors.toMap(BootstrapBaseFileMapping::getFileGroupId, s -> s)); + } + } + private Stream getAllFileGroupsIncludingReplaced(final String partitionStr) { try { readLock.lock(); @@ -1029,22 +1105,38 @@ private Stream getAllFileGroupsIncludingReplaced(final String p @Override public Stream getReplacedFileGroupsBeforeOrOn(String maxCommitTime, String partitionPath) { - return getAllFileGroupsIncludingReplaced(partitionPath).filter(fg -> isFileGroupReplacedBeforeOrOn(fg.getFileGroupId(), maxCommitTime)); + String partition = formatPartitionKey(partitionPath); + if (hasReplacedFilesInPartition(partition)) { + return getAllFileGroupsIncludingReplaced(partition).filter(fg -> isFileGroupReplacedBeforeOrOn(fg.getFileGroupId(), maxCommitTime)); + } + return Stream.empty(); } @Override public Stream getReplacedFileGroupsBefore(String maxCommitTime, String partitionPath) { - return getAllFileGroupsIncludingReplaced(partitionPath).filter(fg -> isFileGroupReplacedBefore(fg.getFileGroupId(), maxCommitTime)); + String partition = formatPartitionKey(partitionPath); + if (hasReplacedFilesInPartition(partition)) { + return getAllFileGroupsIncludingReplaced(partition).filter(fg -> isFileGroupReplacedBefore(fg.getFileGroupId(), maxCommitTime)); + } + return Stream.empty(); } @Override public Stream getReplacedFileGroupsAfterOrOn(String minCommitTime, String partitionPath) { - return getAllFileGroupsIncludingReplaced(partitionPath).filter(fg -> isFileGroupReplacedAfterOrOn(fg.getFileGroupId(), minCommitTime)); + String partition = formatPartitionKey(partitionPath); + if (hasReplacedFilesInPartition(partition)) { + return getAllFileGroupsIncludingReplaced(partition).filter(fg -> isFileGroupReplacedAfterOrOn(fg.getFileGroupId(), minCommitTime)); + } + return Stream.empty(); } @Override public Stream getAllReplacedFileGroups(String partitionPath) { - return getAllFileGroupsIncludingReplaced(partitionPath).filter(fg -> isFileGroupReplaced(fg.getFileGroupId())); + String partition = formatPartitionKey(partitionPath); + if (hasReplacedFilesInPartition(partition)) { + return getAllFileGroupsIncludingReplaced(partition).filter(fg -> isFileGroupReplaced(fg.getFileGroupId())); + } + return Stream.empty(); } @Override @@ -1263,6 +1355,11 @@ protected abstract Option> getPendingLogCompac */ protected abstract void removeReplacedFileIdsAtInstants(Set instants); + /** + * Returns whether there are replaced files within the given partition. + */ + protected abstract boolean hasReplacedFilesInPartition(String partitionPath); + /** * Track instant time for file groups replaced. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java index bb98c97e28d5..f1b56ebe5196 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java @@ -408,6 +408,11 @@ protected void removeReplacedFileIdsAtInstants(Set instants) { fgIdToReplaceInstants.entrySet().removeIf(entry -> instants.contains(entry.getValue().getTimestamp())); } + @Override + protected boolean hasReplacedFilesInPartition(String partitionPath) { + return fgIdToReplaceInstants.keySet().stream().anyMatch(fg -> fg.getPartitionPath().equals(partitionPath)); + } + @Override protected Option getReplaceInstant(final HoodieFileGroupId fileGroupId) { return Option.ofNullable(fgIdToReplaceInstants.get(fileGroupId)); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java index e30b9f425d28..56d7c7cc25cf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java @@ -182,6 +182,11 @@ public Stream getLatestFileSlices(String partitionPath) { return execute(partitionPath, preferredView::getLatestFileSlices, secondaryView::getLatestFileSlices); } + @Override + public Stream getLatestFileSlicesStateless(String partitionPath) { + return execute(partitionPath, preferredView::getLatestFileSlicesStateless, secondaryView::getLatestFileSlicesStateless); + } + @Override public Stream getLatestUnCompactedFileSlices(String partitionPath) { return execute(partitionPath, preferredView::getLatestUnCompactedFileSlices, @@ -222,6 +227,11 @@ public Stream getAllFileGroups(String partitionPath) { return execute(partitionPath, preferredView::getAllFileGroups, secondaryView::getAllFileGroups); } + @Override + public Stream getAllFileGroupsStateless(String partitionPath) { + return execute(partitionPath, preferredView::getAllFileGroupsStateless, secondaryView::getAllFileGroupsStateless); + } + @Override public Stream getReplacedFileGroupsBeforeOrOn(String maxCommitTime, String partitionPath) { return execute(maxCommitTime, partitionPath, preferredView::getReplacedFileGroupsBeforeOrOn, secondaryView::getReplacedFileGroupsBeforeOrOn); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java index f42f9bf2216c..4363a7daf271 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java @@ -43,6 +43,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.module.afterburner.AfterburnerModule; import org.apache.http.Consts; import org.apache.http.client.fluent.Request; import org.apache.http.client.fluent.Response; @@ -63,8 +64,11 @@ */ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, Serializable { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper().registerModule(new AfterburnerModule()); + private static final String BASE_URL = "/v1/hoodie/view"; public static final String LATEST_PARTITION_SLICES_URL = String.format("%s/%s", BASE_URL, "slices/partition/latest/"); + public static final String LATEST_PARTITION_SLICES_STATELESS_URL = String.format("%s/%s", BASE_URL, "slices/partition/latest/stateless/"); public static final String LATEST_PARTITION_SLICE_URL = String.format("%s/%s", BASE_URL, "slices/file/latest/"); public static final String LATEST_PARTITION_UNCOMPACTED_SLICES_URL = String.format("%s/%s", BASE_URL, "slices/uncompacted/partition/latest/"); @@ -98,6 +102,9 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, public static final String ALL_FILEGROUPS_FOR_PARTITION_URL = String.format("%s/%s", BASE_URL, "filegroups/all/partition/"); + public static final String ALL_FILEGROUPS_FOR_PARTITION_STATELESS_URL = + String.format("%s/%s", BASE_URL, "filegroups/all/partition/stateless/"); + public static final String ALL_REPLACED_FILEGROUPS_BEFORE_OR_ON = String.format("%s/%s", BASE_URL, "filegroups/replaced/beforeoron/"); @@ -112,7 +119,6 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, public static final String PENDING_CLUSTERING_FILEGROUPS = String.format("%s/%s", BASE_URL, "clustering/pending/"); - public static final String LAST_INSTANT = String.format("%s/%s", BASE_URL, "timeline/instant/last"); public static final String LAST_INSTANTS = String.format("%s/%s", BASE_URL, "timeline/instants/last"); @@ -136,13 +142,22 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, private static final Logger LOG = LoggerFactory.getLogger(RemoteHoodieTableFileSystemView.class); + private static final TypeReference> FILE_SLICE_DTOS_REFERENCE = new TypeReference>() {}; + private static final TypeReference> FILE_GROUP_DTOS_REFERENCE = new TypeReference>() {}; + private static final TypeReference BOOLEAN_TYPE_REFERENCE = new TypeReference() {}; + private static final TypeReference> COMPACTION_OP_DTOS_REFERENCE = new TypeReference>() {}; + private static final TypeReference> CLUSTERING_OP_DTOS_REFERENCE = new TypeReference>() {}; + private static final TypeReference> INSTANT_DTOS_REFERENCE = new TypeReference>() {}; + private static final TypeReference TIMELINE_DTO_REFERENCE = new TypeReference() {}; + private static final TypeReference> BASE_FILE_DTOS_REFERENCE = new TypeReference>() {}; + private static final TypeReference>> BASE_FILE_MAP_REFERENCE = new TypeReference>>() {}; + private static final TypeReference>> FILE_SLICE_MAP_REFERENCE = new TypeReference>>() {}; private final String serverHost; private final int serverPort; private final String basePath; private final HoodieTableMetaClient metaClient; private HoodieTimeline timeline; - private final ObjectMapper mapper; private final int timeoutMs; private boolean closed = false; @@ -159,7 +174,6 @@ public RemoteHoodieTableFileSystemView(String server, int port, HoodieTableMetaC public RemoteHoodieTableFileSystemView(HoodieTableMetaClient metaClient, FileSystemViewStorageConfig viewConf) { this.basePath = metaClient.getBasePath(); - this.mapper = new ObjectMapper(); this.metaClient = metaClient; this.timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants(); this.serverHost = viewConf.getRemoteViewServerHost(); @@ -175,7 +189,7 @@ public RemoteHoodieTableFileSystemView(HoodieTableMetaClient metaClient, FileSys } } - private T executeRequest(String requestPath, Map queryParameters, TypeReference reference, + private T executeRequest(String requestPath, Map queryParameters, TypeReference reference, RequestMethod method) throws IOException { ValidationUtils.checkArgument(!closed, "View already closed"); @@ -192,7 +206,7 @@ private T executeRequest(String requestPath, Map queryParame LOG.info("Sending request : (" + url + ")"); Response response = retryHelper != null ? retryHelper.start(() -> get(timeoutMs, url, method)) : get(timeoutMs, url, method); String content = response.returnContent().asString(Consts.UTF_8); - return (T) mapper.readValue(content, reference); + return (T) OBJECT_MAPPER.readValue(content, reference); } private Map getParamsWithPartitionPath(String partitionPath) { @@ -250,7 +264,7 @@ public Stream getLatestBaseFiles() { private Stream getLatestBaseFilesFromParams(Map paramsMap, String requestPath) { try { List dataFiles = executeRequest(requestPath, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + BASE_FILE_DTOS_REFERENCE, RequestMethod.GET); return dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -273,8 +287,7 @@ public Map> getAllLatestBaseFilesBeforeOrOn(Strin Map> dataFileMap = executeRequest( ALL_LATEST_BASE_FILES_BEFORE_ON_INSTANT_URL, paramsMap, - new TypeReference>>() { - }, + BASE_FILE_MAP_REFERENCE, RequestMethod.GET); return dataFileMap.entrySet().stream().collect( Collectors.toMap( @@ -291,8 +304,7 @@ public Option getBaseFileOn(String partitionPath, String instant new String[] {INSTANT_PARAM, FILEID_PARAM}, new String[] {instantTime, fileId}); try { List dataFiles = executeRequest(LATEST_DATA_FILE_ON_INSTANT_URL, paramsMap, - new TypeReference>() { - }, RequestMethod.GET); + BASE_FILE_DTOS_REFERENCE, RequestMethod.GET); return Option.fromJavaOptional(dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile).findFirst()); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -317,6 +329,18 @@ public Stream getLatestFileSlices(String partitionPath) { Map paramsMap = getParamsWithPartitionPath(partitionPath); try { List dataFiles = executeRequest(LATEST_PARTITION_SLICES_URL, paramsMap, + FILE_SLICE_DTOS_REFERENCE, RequestMethod.GET); + return dataFiles.stream().map(FileSliceDTO::toFileSlice); + } catch (IOException e) { + throw new HoodieRemoteException(e); + } + } + + @Override + public Stream getLatestFileSlicesStateless(String partitionPath) { + Map paramsMap = getParamsWithPartitionPath(partitionPath); + try { + List dataFiles = executeRequest(LATEST_PARTITION_SLICES_STATELESS_URL, paramsMap, new TypeReference>() {}, RequestMethod.GET); return dataFiles.stream().map(FileSliceDTO::toFileSlice); } catch (IOException e) { @@ -329,7 +353,7 @@ public Option getLatestFileSlice(String partitionPath, String fileId) Map paramsMap = getParamsWithAdditionalParam(partitionPath, FILEID_PARAM, fileId); try { List dataFiles = executeRequest(LATEST_PARTITION_SLICE_URL, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + FILE_SLICE_DTOS_REFERENCE, RequestMethod.GET); return Option.fromJavaOptional(dataFiles.stream().map(FileSliceDTO::toFileSlice).findFirst()); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -341,7 +365,7 @@ public Stream getLatestUnCompactedFileSlices(String partitionPath) { Map paramsMap = getParamsWithPartitionPath(partitionPath); try { List dataFiles = executeRequest(LATEST_PARTITION_UNCOMPACTED_SLICES_URL, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + FILE_SLICE_DTOS_REFERENCE, RequestMethod.GET); return dataFiles.stream().map(FileSliceDTO::toFileSlice); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -356,7 +380,7 @@ public Stream getLatestFileSlicesBeforeOrOn(String partitionPath, Str new String[] {maxCommitTime, String.valueOf(includeFileSlicesInPendingCompaction)}); try { List dataFiles = executeRequest(LATEST_SLICES_BEFORE_ON_INSTANT_URL, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + FILE_SLICE_DTOS_REFERENCE, RequestMethod.GET); return dataFiles.stream().map(FileSliceDTO::toFileSlice); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -371,7 +395,7 @@ public Map> getAllLatestFileSlicesBeforeOrOn(String ma try { Map> fileSliceMap = executeRequest(ALL_LATEST_SLICES_BEFORE_ON_INSTANT_URL, paramsMap, - new TypeReference>>() {}, RequestMethod.GET); + FILE_SLICE_MAP_REFERENCE, RequestMethod.GET); return fileSliceMap.entrySet().stream().collect( Collectors.toMap( Map.Entry::getKey, @@ -386,7 +410,7 @@ public Stream getLatestMergedFileSlicesBeforeOrOn(String partitionPat Map paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxInstantTime); try { List dataFiles = executeRequest(LATEST_SLICES_MERGED_BEFORE_ON_INSTANT_URL, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + FILE_SLICE_DTOS_REFERENCE, RequestMethod.GET); return dataFiles.stream().map(FileSliceDTO::toFileSlice); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -399,7 +423,7 @@ public Stream getLatestFileSliceInRange(List commitsToReturn) getParams(INSTANTS_PARAM, StringUtils.join(commitsToReturn.toArray(new String[0]), ",")); try { List dataFiles = executeRequest(LATEST_SLICES_RANGE_INSTANT_URL, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + FILE_SLICE_DTOS_REFERENCE, RequestMethod.GET); return dataFiles.stream().map(FileSliceDTO::toFileSlice); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -411,7 +435,7 @@ public Stream getAllFileSlices(String partitionPath) { Map paramsMap = getParamsWithPartitionPath(partitionPath); try { List dataFiles = - executeRequest(ALL_SLICES_URL, paramsMap, new TypeReference>() {}, RequestMethod.GET); + executeRequest(ALL_SLICES_URL, paramsMap, FILE_SLICE_DTOS_REFERENCE, RequestMethod.GET); return dataFiles.stream().map(FileSliceDTO::toFileSlice); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -423,7 +447,19 @@ public Stream getAllFileGroups(String partitionPath) { Map paramsMap = getParamsWithPartitionPath(partitionPath); try { List fileGroups = executeRequest(ALL_FILEGROUPS_FOR_PARTITION_URL, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + FILE_GROUP_DTOS_REFERENCE, RequestMethod.GET); + return DTOUtils.fileGroupDTOsToFileGroups(fileGroups, metaClient); + } catch (IOException e) { + throw new HoodieRemoteException(e); + } + } + + @Override + public Stream getAllFileGroupsStateless(String partitionPath) { + Map paramsMap = getParamsWithPartitionPath(partitionPath); + try { + List fileGroups = executeRequest(ALL_FILEGROUPS_FOR_PARTITION_STATELESS_URL, paramsMap, + new TypeReference>() {}, RequestMethod.GET); return DTOUtils.fileGroupDTOsToFileGroups(fileGroups, metaClient); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -435,7 +471,7 @@ public Stream getReplacedFileGroupsBeforeOrOn(String maxCommitT Map paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxCommitTime); try { List fileGroups = executeRequest(ALL_REPLACED_FILEGROUPS_BEFORE_OR_ON, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + FILE_GROUP_DTOS_REFERENCE, RequestMethod.GET); return DTOUtils.fileGroupDTOsToFileGroups(fileGroups, metaClient); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -447,7 +483,7 @@ public Stream getReplacedFileGroupsBefore(String maxCommitTime, Map paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxCommitTime); try { List fileGroups = executeRequest(ALL_REPLACED_FILEGROUPS_BEFORE, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + FILE_GROUP_DTOS_REFERENCE, RequestMethod.GET); return DTOUtils.fileGroupDTOsToFileGroups(fileGroups, metaClient); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -459,7 +495,7 @@ public Stream getReplacedFileGroupsAfterOrOn(String minCommitTi Map paramsMap = getParamsWithAdditionalParam(partitionPath, MIN_INSTANT_PARAM, minCommitTime); try { List fileGroups = executeRequest(ALL_REPLACED_FILEGROUPS_AFTER_OR_ON, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + FILE_GROUP_DTOS_REFERENCE, RequestMethod.GET); return DTOUtils.fileGroupDTOsToFileGroups(fileGroups, metaClient); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -471,7 +507,7 @@ public Stream getAllReplacedFileGroups(String partitionPath) { Map paramsMap = getParamsWithPartitionPath(partitionPath); try { List fileGroups = executeRequest(ALL_REPLACED_FILEGROUPS_PARTITION, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + FILE_GROUP_DTOS_REFERENCE, RequestMethod.GET); return DTOUtils.fileGroupDTOsToFileGroups(fileGroups, metaClient); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -483,7 +519,7 @@ public boolean refresh() { try { // refresh the local timeline first. this.timeline = metaClient.reloadActiveTimeline().filterCompletedAndCompactionInstants(); - return executeRequest(REFRESH_TABLE, paramsMap, new TypeReference() {}, RequestMethod.POST); + return executeRequest(REFRESH_TABLE, paramsMap, BOOLEAN_TYPE_REFERENCE, RequestMethod.POST); } catch (IOException e) { throw new HoodieRemoteException(e); } @@ -493,7 +529,7 @@ public boolean refresh() { public Void loadAllPartitions() { Map paramsMap = getParams(); try { - executeRequest(LOAD_ALL_PARTITIONS_URL, paramsMap, new TypeReference() {}, RequestMethod.POST); + executeRequest(LOAD_ALL_PARTITIONS_URL, paramsMap, BOOLEAN_TYPE_REFERENCE, RequestMethod.POST); return null; } catch (IOException e) { throw new HoodieRemoteException(e); @@ -505,7 +541,7 @@ public Stream> getPendingCompactionOperations( Map paramsMap = getParams(); try { List dtos = executeRequest(PENDING_COMPACTION_OPS, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + COMPACTION_OP_DTOS_REFERENCE, RequestMethod.GET); return dtos.stream().map(CompactionOpDTO::toCompactionOperation); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -517,7 +553,7 @@ public Stream> getPendingLogCompactionOperatio Map paramsMap = getParams(); try { List dtos = executeRequest(PENDING_LOG_COMPACTION_OPS, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + COMPACTION_OP_DTOS_REFERENCE, RequestMethod.GET); return dtos.stream().map(CompactionOpDTO::toCompactionOperation); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -529,7 +565,7 @@ public Stream> getFileGroupsInPendingClus Map paramsMap = getParams(); try { List dtos = executeRequest(PENDING_CLUSTERING_FILEGROUPS, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + CLUSTERING_OP_DTOS_REFERENCE, RequestMethod.GET); return dtos.stream().map(ClusteringOpDTO::toClusteringOperation); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -551,7 +587,7 @@ public Option getLastInstant() { Map paramsMap = getParams(); try { List instants = - executeRequest(LAST_INSTANT, paramsMap, new TypeReference>() {}, RequestMethod.GET); + executeRequest(LAST_INSTANT, paramsMap, INSTANT_DTOS_REFERENCE, RequestMethod.GET); return Option.fromJavaOptional(instants.stream().map(InstantDTO::toInstant).findFirst()); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -563,7 +599,7 @@ public HoodieTimeline getTimeline() { Map paramsMap = getParams(); try { TimelineDTO timeline = - executeRequest(TIMELINE, paramsMap, new TypeReference() {}, RequestMethod.GET); + executeRequest(TIMELINE, paramsMap, TIMELINE_DTO_REFERENCE, RequestMethod.GET); return TimelineDTO.toTimeline(timeline, metaClient); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -580,8 +616,7 @@ public Option getLatestBaseFile(String partitionPath, String fil Map paramsMap = getParamsWithAdditionalParam(partitionPath, FILEID_PARAM, fileId); try { List dataFiles = executeRequest(LATEST_PARTITION_DATA_FILE_URL, paramsMap, - new TypeReference>() { - }, RequestMethod.GET); + BASE_FILE_DTOS_REFERENCE, RequestMethod.GET); return Option.fromJavaOptional(dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile).findFirst()); } catch (IOException e) { throw new HoodieRemoteException(e); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java index 267956563836..b2b05e324810 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java @@ -553,6 +553,12 @@ protected void removeReplacedFileIdsAtInstants(Set instants) { ); } + @Override + protected boolean hasReplacedFilesInPartition(String partitionPath) { + return rocksDB.prefixSearch(schemaHelper.getColFamilyForReplacedFileGroups(), schemaHelper.getPrefixForReplacedFileGroup(partitionPath)) + .findAny().isPresent(); + } + @Override protected Option getReplaceInstant(final HoodieFileGroupId fileGroupId) { String lookupKey = schemaHelper.getKeyForReplacedFileGroup(fileGroupId); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java index 6fedb8684c98..1bcd1de61bc5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java @@ -107,6 +107,19 @@ interface SliceViewWithLatestSlice { */ Stream getLatestFileSlices(String partitionPath); + /** + * Stream all the latest file slices in the given partition + * without caching the file group mappings. + * + *

This is useful for some table services such as compaction and clustering, these services may search around the files to clean + * within some ancient data partitions, if there triggers a full table service for enormous number of partitions, the cache could + * cause a huge memory pressure to the timeline server which induces an OOM exception. + * + *

The caching of these file groups does not benefit to writers most often because the writers + * write to recent data partitions usually. + */ + Stream getLatestFileSlicesStateless(String partitionPath); + /** * Get Latest File Slice for a given fileId in a given partition. */ @@ -168,6 +181,18 @@ interface SliceView extends SliceViewWithLatestSlice { */ Stream getAllFileGroups(String partitionPath); + /** + * Stream all the file groups for a given partition without caching the file group mappings. + * + *

This is useful for some table services such as cleaning, the cleaning service may search around the files to clean + * within some ancient data partitions, if there triggers a full table cleaning for enormous number of partitions, the cache could + * cause a huge memory pressure to the timeline server which induces an OOM exception. + * + *

The caching of these file groups does not benefit to writers most often because the writers + * write to recent data partitions usually. + */ + Stream getAllFileGroupsStateless(String partitionPath); + /** * Return Pending Compaction Operations. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java b/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java index 45b2a13eb72a..ff924e450135 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java @@ -87,6 +87,10 @@ public String getKeyForReplacedFileGroup(HoodieFileGroupId fgId) { return getPartitionFileIdBasedLookup(fgId); } + public String getPrefixForReplacedFileGroup(String partitionPath) { + return String.format("part=%s,id=", partitionPath); + } + public String getKeyForFileGroupsInPendingClustering(HoodieFileGroupId fgId) { return getPartitionFileIdBasedLookup(fgId); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java index d7d79796aec8..5b95bc60312d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java @@ -173,4 +173,14 @@ public static String removeSuffixBy(String input, int ch) { } return input.substring(0, i); } + + public static String truncate(String str, int headLength, int tailLength) { + if (isNullOrEmpty(str) || str.length() <= headLength + tailLength) { + return str; + } + String head = str.substring(0, headLength); + String tail = str.substring(str.length() - tailLength); + + return head + "..." + tail; + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java index a4420fc62b3f..eed60d2a5372 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java @@ -57,6 +57,7 @@ public void testStorageSchemes() { assertFalse(StorageSchemes.isAtomicCreationSupported("oci")); assertFalse(StorageSchemes.isAtomicCreationSupported("tos")); assertFalse(StorageSchemes.isAtomicCreationSupported("cfs")); + assertTrue(StorageSchemes.isAtomicCreationSupported("hopsfs")); } @Test diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java index 1cb146ec97e7..6fdb85c29f1c 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java @@ -109,6 +109,8 @@ public void testDeletedRecord(String key) throws IOException { DefaultHoodieRecordPayload payload1 = new DefaultHoodieRecordPayload(record1, 1); DefaultHoodieRecordPayload payload2 = new DefaultHoodieRecordPayload(delRecord1, 2); + assertFalse(payload1.isDeleted(schema, props)); + assertTrue(payload2.isDeleted(schema, props)); assertEquals(payload1.preCombine(payload2, props), payload2); assertEquals(payload2.preCombine(payload1, props), payload2); @@ -145,9 +147,13 @@ public void testDeleteKey() throws IOException { DefaultHoodieRecordPayload deletePayload = new DefaultHoodieRecordPayload(delRecord, 2); DefaultHoodieRecordPayload defaultDeletePayload = new DefaultHoodieRecordPayload(defaultDeleteRecord, 2); + assertFalse(payload.isDeleted(schema, props)); + assertTrue(deletePayload.isDeleted(schema, props)); + assertFalse(defaultDeletePayload.isDeleted(schema, props)); // if custom marker is present, should honor that irrespective of hoodie_is_deleted + assertEquals(record, payload.getInsertValue(schema, props).get()); - assertEquals(defaultDeleteRecord, defaultDeletePayload.getInsertValue(schema, props).get()); assertFalse(deletePayload.getInsertValue(schema, props).isPresent()); + assertTrue(defaultDeletePayload.getInsertValue(schema, props).isPresent()); // if custom marker is present, should honor that irrespective of hoodie_is_deleted assertEquals(delRecord, payload.combineAndGetUpdateValue(delRecord, schema, props).get()); assertEquals(defaultDeleteRecord, payload.combineAndGetUpdateValue(defaultDeleteRecord, schema, props).get()); @@ -174,6 +180,7 @@ public void testDeleteKeyConfiguration() throws IOException { } try { + payload = new DefaultHoodieRecordPayload(record, 1); payload.combineAndGetUpdateValue(record, schema, props).get(); fail("Should fail"); } catch (IllegalArgumentException e) { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java index ab8a1fd3aa28..558dc59f40c9 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java @@ -90,11 +90,12 @@ import static org.apache.hudi.common.model.HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE_PREFIX; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; -import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotSame; import static org.junit.jupiter.api.Assertions.assertTrue; /** @@ -328,6 +329,109 @@ public void testViewForFileSlicesWithPartitionMetadataFile() throws Exception { assertEquals(2, fsView.getAllFileGroups(partitionPath).count()); } + @Test + public void testViewForGetAllFileGroupsStateless() throws Exception { + String partitionPath1 = "2023/11/22"; + new File(basePath + "/" + partitionPath1).mkdirs(); + new File(basePath + "/" + partitionPath1 + "/" + HOODIE_PARTITION_METAFILE_PREFIX + ".parquet").mkdirs(); + String partitionPath2 = "2023/11/23"; + new File(basePath + "/" + partitionPath2).mkdirs(); + new File(basePath + "/" + partitionPath2 + "/" + HOODIE_PARTITION_METAFILE_PREFIX + ".parquet").mkdirs(); + + // create 2 fileId in partition1 + String fileId1 = UUID.randomUUID().toString(); + String fileId2 = UUID.randomUUID().toString(); + String commitTime1 = "1"; + String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); + String fileName2 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); + new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile(); + new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile(); + + HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); + HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1); + saveAsComplete(commitTimeline, instant1, Option.empty()); + + // create 2 fileId in partition2 + String fileId3 = UUID.randomUUID().toString(); + String fileId4 = UUID.randomUUID().toString(); + String commitTime2 = "2"; + String fileName3 = FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId3); + String fileName4 = FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId4); + new File(basePath + "/" + partitionPath2 + "/" + fileName3).createNewFile(); + new File(basePath + "/" + partitionPath2 + "/" + fileName4).createNewFile(); + + HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2); + saveAsComplete(commitTimeline, instant2, Option.empty()); + + fsView.sync(); + // invokes the stateless API first then the normal API, assert the result equality with different file group objects + List actual1 = fsView.getAllFileGroupsStateless(partitionPath1).collect(Collectors.toList()); + List expected1 = fsView.getAllFileGroups(partitionPath1).collect(Collectors.toList()); + for (int i = 0; i < expected1.size(); i++) { + assertThat("The stateless API should return the same result", actual1.get(i).toString(), is(expected1.get(i).toString())); + assertNotSame(actual1.get(i), expected1.get(i), "The stateless API does not cache"); + } + + List expected2 = fsView.getAllFileGroupsStateless(partitionPath2).collect(Collectors.toList()); + List actual2 = fsView.getAllFileGroups(partitionPath2).collect(Collectors.toList()); + for (int i = 0; i < expected2.size(); i++) { + assertThat("The stateless API should return the same result", actual2.get(i).toString(), is(expected2.get(i).toString())); + assertNotSame(actual2.get(i), expected2.get(i), "The stateless API does not cache"); + } + } + + @Test + public void testViewForGetLatestFileSlicesStateless() throws Exception { + String partitionPath1 = "2023/11/22"; + new File(basePath + "/" + partitionPath1).mkdirs(); + new File(basePath + "/" + partitionPath1 + "/" + HOODIE_PARTITION_METAFILE_PREFIX + ".parquet").mkdirs(); + String partitionPath2 = "2023/11/23"; + new File(basePath + "/" + partitionPath2).mkdirs(); + new File(basePath + "/" + partitionPath2 + "/" + HOODIE_PARTITION_METAFILE_PREFIX + ".parquet").mkdirs(); + + // create 2 fileId in partition1 + String fileId1 = UUID.randomUUID().toString(); + String fileId2 = UUID.randomUUID().toString(); + String commitTime1 = "1"; + String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); + String fileName2 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); + new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile(); + new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile(); + + HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); + HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1); + saveAsComplete(commitTimeline, instant1, Option.empty()); + + // create 2 fileId in partition2 + String fileId3 = UUID.randomUUID().toString(); + String fileId4 = UUID.randomUUID().toString(); + String commitTime2 = "2"; + String fileName3 = FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId3); + String fileName4 = FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId4); + new File(basePath + "/" + partitionPath2 + "/" + fileName3).createNewFile(); + new File(basePath + "/" + partitionPath2 + "/" + fileName4).createNewFile(); + + HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2); + saveAsComplete(commitTimeline, instant2, Option.empty()); + + fsView.sync(); + + // invokes the stateless API first then the normal API, assert the result equality with different file slice objects + List actual1 = fsView.getLatestFileSlicesStateless(partitionPath1).collect(Collectors.toList()); + List expected1 = fsView.getLatestFileSlices(partitionPath1).collect(Collectors.toList()); + for (int i = 0; i < expected1.size(); i++) { + assertThat("The stateless API should return the same result", actual1.get(i), is(expected1.get(i))); + assertNotSame(actual1.get(i), expected1.get(i), "The stateless API does not cache"); + } + + List expected2 = fsView.getLatestFileSlicesStateless(partitionPath2).collect(Collectors.toList()); + List actual2 = fsView.getLatestFileSlices(partitionPath2).collect(Collectors.toList()); + for (int i = 0; i < expected2.size(); i++) { + assertThat("The stateless API should return the same result", actual2.get(i), is(expected2.get(i))); + assertNotSame(actual2.get(i), expected2.get(i), "The stateless API does not cache"); + } + } + @Test protected void testInvalidLogFiles() throws Exception { String partitionPath = "2016/05/01"; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java index 616e895d221f..1f28e0094f16 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java @@ -81,12 +81,20 @@ public static HoodieTableMetaClient init(String basePath, HoodieTableType tableT } public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType, String bootstrapBasePath, boolean bootstrapIndexEnable, String keyGenerator) throws IOException { + return init(basePath, tableType, bootstrapBasePath, bootstrapIndexEnable, keyGenerator, "datestr"); + } + + public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType, String bootstrapBasePath, boolean bootstrapIndexEnable, String keyGenerator, + String partitionFieldConfigValue) throws IOException { Properties props = new Properties(); props.setProperty(HoodieTableConfig.BOOTSTRAP_BASE_PATH.key(), bootstrapBasePath); props.put(HoodieTableConfig.BOOTSTRAP_INDEX_ENABLE.key(), bootstrapIndexEnable); if (keyGenerator != null) { props.put("hoodie.datasource.write.keygenerator.class", keyGenerator); - props.put("hoodie.datasource.write.partitionpath.field", "datestr"); + } + if (keyGenerator != null && !keyGenerator.equals("org.apache.hudi.keygen.NonpartitionedKeyGenerator")) { + props.put("hoodie.datasource.write.partitionpath.field", partitionFieldConfigValue); + props.put(HoodieTableConfig.PARTITION_FIELDS.key(), partitionFieldConfigValue); } return init(getDefaultHadoopConf(), basePath, tableType, props); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestStringUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestStringUtils.java index 3bdf6d48b395..54985056bf08 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestStringUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestStringUtils.java @@ -114,4 +114,11 @@ private static String toHexString(byte[] bytes) { } return sb.toString(); } + + @Test + public void testTruncate() { + assertNull(StringUtils.truncate(null, 10, 10)); + assertEquals("http://use...ons/latest", StringUtils.truncate("http://username:password@myregistry.com:5000/versions/latest", 10, 10)); + assertEquals("http://abc.com", StringUtils.truncate("http://abc.com", 10, 10)); + } } diff --git a/hudi-examples/hudi-examples-dbt/README.md b/hudi-examples/hudi-examples-dbt/README.md index 8fe796d37c52..22f745911261 100644 --- a/hudi-examples/hudi-examples-dbt/README.md +++ b/hudi-examples/hudi-examples-dbt/README.md @@ -18,45 +18,113 @@ This dbt project transforms demonstrates hudi integration with dbt, it has a few models to demonstrate the different ways in which you can create hudi datasets using dbt. -### What is this repo? -What this repo _is_: -- A self-contained playground dbt project, useful for testing out scripts, and communicating some of the core dbt concepts. +This directory serves as a self-contained playground dbt project, useful for testing out scripts, and communicating some of the core dbt concepts. -### Running this project -To get up and running with this project: -1. Install dbt using [these instructions](https://docs.getdbt.com/docs/installation). +### Setup -2. Install [dbt-spark](https://github.com/dbt-labs/dbt-spark) package: -```bash -pip install dbt-spark -``` +Switch working directory and have `python3` installed. -3. Clone this repo and change into the `hudi-examples-dbt` directory from the command line: -```bash +```shell cd hudi-examples/hudi-examples-dbt ``` -4. Set up a profile called `spark` to connect to a spark cluster by following [these instructions](https://docs.getdbt.com/reference/warehouse-profiles/spark-profile). If you have access to a data warehouse, you can use those credentials – we recommend setting your [target schema](https://docs.getdbt.com/docs/configure-your-profile#section-populating-your-profile) to be a new schema (dbt will create the schema for you, as long as you have the right privileges). If you don't have access to an existing data warehouse, you can also setup a local postgres database and connect to it in your profile. +### Install dbt + +Create python virtual environment ([Reference](https://docs.getdbt.com/docs/installation)). + +```shell +python3 -m venv dbt-env +source dbt-env/bin/activate +``` + +We are using `thrift` as the connection method ([Reference](https://docs.getdbt.com/docs/core/connect-data-platform/spark-setup)). + +```shell +python3 -m pip install "dbt-spark[PyHive]" +``` + +### Configure dbt for Spark + +Set up a profile called `spark` to connect to a spark cluster via thrift server ([Reference](https://docs.getdbt.com/docs/core/connect-data-platform/spark-setup#thrift)). + +```yaml +spark: + target: dev + outputs: + dev: + type: spark + method: thrift + schema: my_schema + host: localhost + port: 10000 + server_side_parameters: + "spark.driver.memory": "3g" +``` + +_If you have access to a data warehouse, you can use those credentials – we recommend setting your [target schema](https://docs.getdbt.com/docs/configure-your-profile#section-populating-your-profile) to be a new schema (dbt will create the schema for you, as long as you have the right privileges). If you don't have access to an existing data warehouse, you can also setup a local postgres database and connect to it in your profile._ + +### Start Spark Thrift server + +> **NOTE** Using these versions +> - Spark 3.2.3 (with Derby 10.14.2.0) +> - Hudi 0.14.0 + +Start a local Derby server + +```shell +export DERBY_VERSION=10.14.2.0 +wget https://archive.apache.org/dist/db/derby/db-derby-$DERBY_VERSION/db-derby-$DERBY_VERSION-bin.tar.gz -P /opt/ +tar -xf /opt/db-derby-$DERBY_VERSION-bin.tar.gz -C /opt/ +export DERBY_HOME=/opt/db-derby-$DERBY_VERSION-bin +$DERBY_HOME/bin/startNetworkServer -h 0.0.0.0 +``` + +Start a local Thrift server for Spark + +```shell +export SPARK_VERSION=3.2.3 +wget https://archive.apache.org/dist/spark/spark-$SPARK_VERSION/spark-$SPARK_VERSION-bin-hadoop2.7.tgz -P /opt/ +tar -xf /opt/spark-$SPARK_VERSION-bin-hadoop2.7.tgz -C /opt/ +export SPARK_HOME=/opt/spark-$SPARK_VERSION-bin-hadoop2.7 + +# install dependencies +cp $DERBY_HOME/lib/{derby,derbyclient}.jar $SPARK_HOME/jars/ +wget https://repository.apache.org/content/repositories/releases/org/apache/hudi/hudi-spark3.2-bundle_2.12/0.14.0/hudi-spark3.2-bundle_2.12-0.14.0.jar -P $SPARK_HOME/jars/ + +# start Thrift server connecting to Derby as HMS backend +$SPARK_HOME/sbin/start-thriftserver.sh \ +--conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ +--conf spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension \ +--conf spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog \ +--conf spark.sql.warehouse.dir=/tmp/hudi/hive/warehouse \ +--hiveconf hive.metastore.warehouse.dir=/tmp/hudi/hive/warehouse \ +--hiveconf hive.metastore.schema.verification=false \ +--hiveconf datanucleus.schema.autoCreateAll=true \ +--hiveconf javax.jdo.option.ConnectionDriverName=org.apache.derby.jdbc.ClientDriver \ +--hiveconf 'javax.jdo.option.ConnectionURL=jdbc:derby://localhost:1527/default;create=true' +``` -> **NOTE:** You need to include the hudi spark bundle to the spark cluster, the latest supported version is 0.10.1. +### Verify dbt setup -5. Ensure your profile is setup correctly from the command line: -```bash +```shell dbt debug ``` Output of the above command should show this text at the end of the output: -```bash + +``` All checks passed! ``` -6. Run the models: -```bash +### Run the models + +```shell dbt run ``` -Output should look like this: -```bash +Output should look like this + +``` 05:47:28 Running with dbt=1.0.0 05:47:28 Found 5 models, 10 tests, 0 snapshots, 0 analyses, 0 macros, 0 operations, 0 seed files, 0 sources, 0 exposures, 0 metrics 05:47:28 @@ -77,12 +145,16 @@ Output should look like this: 05:47:42 05:47:42 Completed successfully ``` -7. Test the output of the models: -```bash + +### Test the output of the models + +```shell dbt test ``` -Output should look like this: -```bash + +Output should look like this + +``` 05:48:17 Running with dbt=1.0.0 05:48:17 Found 5 models, 10 tests, 0 snapshots, 0 analyses, 0 macros, 0 operations, 0 seed files, 0 sources, 0 exposures, 0 metrics 05:48:17 @@ -116,14 +188,12 @@ Output should look like this: 05:48:26 Done. PASS=10 WARN=0 ERROR=0 SKIP=0 TOTAL=10 ``` -8. Generate documentation for the project: -```bash -dbt docs generate -``` +### Generate documentation -9. View the [documentation](http://127.0.0.1:8080/#!/overview) for the project after running the following command: -```bash +```shell +dbt docs generate dbt docs serve +# then visit http://127.0.0.1:8080/#!/overview ``` --- diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunction.java index 121b1d31f31c..2c721dff2894 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunction.java @@ -112,7 +112,7 @@ protected void sendBootstrapEvent() { int attemptId = getRuntimeContext().getAttemptNumber(); if (attemptId > 0) { // either a partial or global failover, reuses the current inflight instant - if (this.currentInstant != null) { + if (this.currentInstant != null && !metaClient.getActiveTimeline().filterCompletedInstants().containsInstant(currentInstant)) { LOG.info("Recover task[{}] for instant [{}] with attemptId [{}]", taskID, this.currentInstant, attemptId); this.currentInstant = null; return; diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java index a453cac6803e..0966f6995bd6 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java @@ -348,7 +348,9 @@ private void cluster() throws Exception { .addSink(new ClusteringCommitSink(conf)) .name("clustering_commit") .uid("uid_clustering_commit") - .setParallelism(1); + .setParallelism(1) + .getTransformation() + .setMaxParallelism(1); env.execute("flink_hudi_clustering_" + clusteringInstant.getTimestamp()); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java index 57e823ab21cd..99dd45d94b49 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java @@ -298,7 +298,9 @@ private void compact() throws Exception { .addSink(new CompactionCommitSink(conf)) .name("compaction_commit") .uid("uid_compaction_commit") - .setParallelism(1); + .setParallelism(1) + .getTransformation() + .setMaxParallelism(1); env.execute("flink_hudi_compaction_" + String.join(",", compactionInstantTimes)); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunctionWithRateLimit.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunctionWithRateLimit.java index fc9c2177e7c0..4a1962bf9b48 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunctionWithRateLimit.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunctionWithRateLimit.java @@ -18,14 +18,16 @@ package org.apache.hudi.sink.transform; -import org.apache.hudi.adapter.RateLimiterAdapter; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.RateLimiter; import org.apache.hudi.configuration.FlinkOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; +import java.util.concurrent.TimeUnit; + /** * Function that transforms RowData to a HoodieRecord with RateLimit. */ @@ -39,7 +41,7 @@ public class RowDataToHoodieFunctionWithRateLimit hoodieStreamWrite(Configuration conf, DataStrea * @return the compaction pipeline */ public static DataStreamSink compact(Configuration conf, DataStream dataStream) { - return dataStream.transform("compact_plan_generate", + DataStreamSink compactionCommitEventDataStream = dataStream.transform("compact_plan_generate", TypeInformation.of(CompactionPlanEvent.class), new CompactionPlanOperator(conf)) .setParallelism(1) // plan generate must be singleton + .setMaxParallelism(1) // make the distribution strategy deterministic to avoid concurrent modifications // on the same bucket files .keyBy(plan -> plan.getOperation().getFileGroupId().getFileId()) @@ -424,6 +425,8 @@ public static DataStreamSink compact(Configuration conf, .addSink(new CompactionCommitSink(conf)) .name("compact_commit") .setParallelism(1); // compaction commit should be singleton + compactionCommitEventDataStream.getTransformation().setMaxParallelism(1); + return compactionCommitEventDataStream; } /** @@ -452,6 +455,7 @@ public static DataStreamSink cluster(Configuration conf, TypeInformation.of(ClusteringPlanEvent.class), new ClusteringPlanOperator(conf)) .setParallelism(1) // plan generate must be singleton + .setMaxParallelism(1) // plan generate must be singleton .keyBy(plan -> // make the distribution strategy deterministic to avoid concurrent modifications // on the same bucket files @@ -465,15 +469,19 @@ public static DataStreamSink cluster(Configuration conf, ExecNodeUtil.setManagedMemoryWeight(clusteringStream.getTransformation(), conf.getInteger(FlinkOptions.WRITE_SORT_MEMORY) * 1024L * 1024L); } - return clusteringStream.addSink(new ClusteringCommitSink(conf)) + DataStreamSink clusteringCommitEventDataStream = clusteringStream.addSink(new ClusteringCommitSink(conf)) .name("clustering_commit") .setParallelism(1); // clustering commit should be singleton + clusteringCommitEventDataStream.getTransformation().setMaxParallelism(1); + return clusteringCommitEventDataStream; } public static DataStreamSink clean(Configuration conf, DataStream dataStream) { - return dataStream.addSink(new CleanFunction<>(conf)) + DataStreamSink cleanCommitDataStream = dataStream.addSink(new CleanFunction<>(conf)) .setParallelism(1) .name("clean_commits"); + cleanCommitDataStream.getTransformation().setMaxParallelism(1); + return cleanCommitDataStream; } public static DataStreamSink dummySink(DataStream dataStream) { diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadOperator.java index 1fe0b4d1f43a..ffafc6fda938 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadOperator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadOperator.java @@ -18,25 +18,28 @@ package org.apache.hudi.source; -import org.apache.hudi.adapter.AbstractStreamOperatorAdapter; -import org.apache.hudi.adapter.AbstractStreamOperatorFactoryAdapter; -import org.apache.hudi.adapter.MailboxExecutorAdapter; -import org.apache.hudi.adapter.Utils; import org.apache.hudi.metrics.FlinkStreamReadMetrics; import org.apache.hudi.table.format.mor.MergeOnReadInputFormat; import org.apache.hudi.table.format.mor.MergeOnReadInputSplit; +import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.state.JavaSerializer; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.api.operators.StreamSourceContexts; +import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; @@ -60,7 +63,7 @@ * This architecture allows the separation of split reading from processing the checkpoint barriers, * thus removing any potential back-pressure. */ -public class StreamReadOperator extends AbstractStreamOperatorAdapter +public class StreamReadOperator extends AbstractStreamOperator implements OneInputStreamOperator { private static final Logger LOG = LoggerFactory.getLogger(StreamReadOperator.class); @@ -70,7 +73,7 @@ public class StreamReadOperator extends AbstractStreamOperatorAdapter // It's the same thread that runs this operator and checkpoint actions. Use this executor to schedule only // splits for subsequent reading, so that a new checkpoint could be triggered without blocking a long time // for exhausting all scheduled split reading tasks. - private final MailboxExecutorAdapter executor; + private final MailboxExecutor executor; private MergeOnReadInputFormat format; @@ -89,7 +92,7 @@ public class StreamReadOperator extends AbstractStreamOperatorAdapter private transient FlinkStreamReadMetrics readMetrics; private StreamReadOperator(MergeOnReadInputFormat format, ProcessingTimeService timeService, - MailboxExecutorAdapter mailboxExecutor) { + MailboxExecutor mailboxExecutor) { this.format = Preconditions.checkNotNull(format, "The InputFormat should not be null."); this.processingTimeService = timeService; this.executor = Preconditions.checkNotNull(mailboxExecutor, "The mailboxExecutor should not be null."); @@ -119,10 +122,9 @@ public void initializeState(StateInitializationContext context) throws Exception } } - this.sourceContext = Utils.getSourceContext( + this.sourceContext = getSourceContext( getOperatorConfig().getTimeCharacteristic(), getProcessingTimeService(), - getContainingTask(), output, getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval()); @@ -247,8 +249,8 @@ private enum SplitState { IDLE, RUNNING } - private static class OperatorFactory extends AbstractStreamOperatorFactoryAdapter - implements OneInputStreamOperatorFactory { + private static class OperatorFactory extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory, YieldingOperatorFactory { private final MergeOnReadInputFormat format; @@ -259,7 +261,7 @@ private OperatorFactory(MergeOnReadInputFormat format) { @SuppressWarnings("unchecked") @Override public > O createStreamOperator(StreamOperatorParameters parameters) { - StreamReadOperator operator = new StreamReadOperator(format, processingTimeService, getMailboxExecutorAdapter()); + StreamReadOperator operator = new StreamReadOperator(format, processingTimeService, getMailboxExecutor()); operator.setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); return (O) operator; } @@ -269,4 +271,19 @@ public Class getStreamOperatorClass(ClassLoader classL return StreamReadOperator.class; } } + + private static SourceFunction.SourceContext getSourceContext( + TimeCharacteristic timeCharacteristic, + ProcessingTimeService processingTimeService, + Output> output, + long watermarkInterval) { + return StreamSourceContexts.getSourceContext( + timeCharacteristic, + processingTimeService, + new Object(), // no actual locking needed + output, + watermarkInterval, + -1, + true); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java index 83816044acb1..bbab47236a89 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java @@ -171,8 +171,8 @@ private void sanityCheck(Configuration conf, ResolvedSchema schema) { if (!OptionsResolver.isAppendMode(conf)) { checkRecordKey(conf, schema); - checkPreCombineKey(conf, schema); } + checkPreCombineKey(conf, schema); } /** diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java index e4b8db335163..b4ef68a39392 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java @@ -207,6 +207,7 @@ public DataStream produceDataStream(StreamExecutionEnvironment execEnv) SingleOutputStreamOperator source = execEnv.addSource(monitoringFunction, getSourceOperatorName("split_monitor")) .uid(Pipelines.opUID("split_monitor", conf)) .setParallelism(1) + .setMaxParallelism(1) .keyBy(MergeOnReadInputSplit::getFileId) .transform("split_reader", typeInfo, factory) .uid(Pipelines.opUID("split_reader", conf)) diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java index 7780204ae3d2..13b93d4c11fa 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java @@ -113,7 +113,7 @@ public static void setPreCombineField(Configuration conf, HoodieTableMetaClient * @param conf The configuration * @param metaClient The meta client */ - public static void inferChangelogMode(Configuration conf, HoodieTableMetaClient metaClient) { + public static void inferChangelogMode(Configuration conf, HoodieTableMetaClient metaClient) throws Exception { TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); Schema tableAvroSchema = tableSchemaResolver.getTableAvroSchemaFromDataFile(); if (tableAvroSchema.getField(HoodieRecord.OPERATION_METADATA_FIELD) != null) { diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java index 9364528eb339..0f8651b4b05e 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java @@ -545,9 +545,11 @@ public void testWriteMultiWriterInvolved(WriteConcurrencyMode writeConcurrencyMo .checkpoint(1) .assertNextEvent() .checkpointComplete(1) - .checkWrittenData(EXPECTED3, 1); + .checkWrittenData(EXPECTED3, 1) + .end(); // step to commit the 2nd txn validateConcurrentCommit(pipeline1); + pipeline1.end(); } } @@ -587,43 +589,66 @@ public void testWriteMultiWriterPartialOverlapping(WriteConcurrencyMode writeCon if (OptionsResolver.isCowTable(conf) && OptionsResolver.isNonBlockingConcurrencyControl(conf)) { validateNonBlockingConcurrencyControlConditions(); } else { - TestHarness pipeline1 = preparePipeline(conf) - .consume(TestData.DATA_SET_INSERT_DUPLICATES) - .assertEmptyDataFiles(); - // now start pipeline2 and suspend the txn commit - Configuration conf2 = conf.clone(); - conf2.setString(FlinkOptions.WRITE_CLIENT_ID, "2"); - TestHarness pipeline2 = preparePipeline(conf2) - .consume(TestData.DATA_SET_INSERT_DUPLICATES) - .assertEmptyDataFiles(); - - // step to commit the 1st txn, should succeed - pipeline1.checkpoint(1) - .assertNextEvent() - .checkpointComplete(1) - .checkWrittenData(EXPECTED3, 1); - - // step to commit the 2nd txn - // should success for concurrent modification of same fileGroups if using non-blocking concurrency control - // should throw exception otherwise - validateConcurrentCommit(pipeline2); + TestHarness pipeline1 = null; + TestHarness pipeline2 = null; + try { + pipeline1 = preparePipeline(conf) + .consume(TestData.DATA_SET_INSERT_DUPLICATES) + .assertEmptyDataFiles(); + // now start pipeline2 and suspend the txn commit + Configuration conf2 = conf.clone(); + conf2.setString(FlinkOptions.WRITE_CLIENT_ID, "2"); + pipeline2 = preparePipeline(conf2) + .consume(TestData.DATA_SET_INSERT_DUPLICATES) + .assertEmptyDataFiles(); + + // step to commit the 1st txn, should succeed + pipeline1.checkpoint(1) + .assertNextEvent() + .checkpointComplete(1) + .checkWrittenData(EXPECTED3, 1); + + // step to commit the 2nd txn + // should success for concurrent modification of same fileGroups if using non-blocking concurrency control + // should throw exception otherwise + validateConcurrentCommit(pipeline2); + } finally { + if (pipeline1 != null) { + pipeline1.end(); + } + if (pipeline2 != null) { + pipeline2.end(); + } + } } } @Test public void testReuseEmbeddedServer() throws IOException { conf.setInteger("hoodie.filesystem.view.remote.timeout.secs", 500); - HoodieFlinkWriteClient writeClient = FlinkWriteClients.createWriteClient(conf); - FileSystemViewStorageConfig viewStorageConfig = writeClient.getConfig().getViewStorageConfig(); - - assertSame(viewStorageConfig.getStorageType(), FileSystemViewStorageType.REMOTE_FIRST); - - // get another write client - writeClient = FlinkWriteClients.createWriteClient(conf); - assertSame(writeClient.getConfig().getViewStorageConfig().getStorageType(), FileSystemViewStorageType.REMOTE_FIRST); - assertEquals(viewStorageConfig.getRemoteViewServerPort(), writeClient.getConfig().getViewStorageConfig().getRemoteViewServerPort()); - assertEquals(viewStorageConfig.getRemoteTimelineClientTimeoutSecs(), 500); - writeClient.close(); + conf.setString("hoodie.metadata.enable","true"); + HoodieFlinkWriteClient writeClient = null; + HoodieFlinkWriteClient writeClient2 = null; + + try { + writeClient = FlinkWriteClients.createWriteClient(conf); + FileSystemViewStorageConfig viewStorageConfig = writeClient.getConfig().getViewStorageConfig(); + + assertSame(viewStorageConfig.getStorageType(), FileSystemViewStorageType.REMOTE_FIRST); + + // get another write client + writeClient2 = FlinkWriteClients.createWriteClient(conf); + assertSame(writeClient2.getConfig().getViewStorageConfig().getStorageType(), FileSystemViewStorageType.REMOTE_FIRST); + assertEquals(viewStorageConfig.getRemoteViewServerPort(), writeClient2.getConfig().getViewStorageConfig().getRemoteViewServerPort()); + assertEquals(viewStorageConfig.getRemoteTimelineClientTimeoutSecs(), 500); + } finally { + if (writeClient != null) { + writeClient.close(); + } + if (writeClient2 != null) { + writeClient2.close(); + } + } } @Test diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnReadWithCompact.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnReadWithCompact.java index e07118cb0521..d7460ecfb050 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnReadWithCompact.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnReadWithCompact.java @@ -159,6 +159,8 @@ public void testNonBlockingConcurrencyControlWithPartialUpdatePayload() throws E // because the data files belongs 3rd commit is not included in the last compaction. Map readOptimizedResult = Collections.singletonMap("par1", "[id1,par1,id1,Danny,23,2,par1]"); TestData.checkWrittenData(tempFile, readOptimizedResult, 1); + pipeline1.end(); + pipeline2.end(); } @Test @@ -227,6 +229,8 @@ public void testNonBlockingConcurrencyControlWithInflightInstant() throws Except // the data files belongs 3rd commit is not included in the last compaction. Map readOptimizedResult = Collections.singletonMap("par1", "[id1,par1,id1,Danny,23,1,par1]"); TestData.checkWrittenData(tempFile, readOptimizedResult, 1); + pipeline1.end(); + pipeline2.end(); } // case1: txn1 is upsert writer, txn2 is bulk_insert writer. @@ -269,6 +273,8 @@ public void testBulkInsertWithNonBlockingConcurrencyControl() throws Exception { // step to commit the 2nd txn, should throw exception pipeline2.endInputThrows(HoodieWriteConflictException.class, "Cannot resolve conflicts"); + pipeline1.end(); + pipeline2.end(); } // case1: txn1 is upsert writer, txn2 is bulk_insert writer. @@ -342,6 +348,8 @@ public void testBulkInsertInSequenceWithNonBlockingConcurrencyControl() throws E // because the data files belongs 3rd commit is not included in the last compaction. Map readOptimizedResult = Collections.singletonMap("par1", "[id1,par1,id1,Danny,23,2,par1]"); TestData.checkWrittenData(tempFile, readOptimizedResult, 1); + pipeline1.end(); + pipeline2.end(); } @Override diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/CollectorOutput.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/CollectorOutput.java index b18cfac51b44..9df912f12995 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/CollectorOutput.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/CollectorOutput.java @@ -18,12 +18,11 @@ package org.apache.hudi.sink.utils; -import org.apache.hudi.adapter.OutputAdapter; - 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; import java.util.ArrayList; @@ -32,7 +31,7 @@ /** * Collecting {@link Output} for {@link StreamRecord}. */ -public class CollectorOutput implements OutputAdapter> { +public class CollectorOutput implements Output> { private final List records; @@ -68,4 +67,9 @@ public void collect(OutputTag outputTag, StreamRecord record) { public void close() { this.records.clear(); } + + @Override + public void emitWatermarkStatus(WatermarkStatus watermarkStatus) { + // no operation + } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStateInitializationContext.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStateInitializationContext.java index e218f29df6fe..23f87b15c65f 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStateInitializationContext.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStateInitializationContext.java @@ -17,17 +17,18 @@ package org.apache.hudi.sink.utils; -import org.apache.hudi.adapter.StateInitializationContextAdapter; - import org.apache.flink.api.common.state.KeyedStateStore; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider; +import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StatePartitionStreamProvider; +import java.util.OptionalLong; + /** * A {@link FunctionInitializationContext} for testing purpose. */ -public class MockStateInitializationContext implements StateInitializationContextAdapter { +public class MockStateInitializationContext implements StateInitializationContext { private final MockOperatorStateStore operatorStateStore; @@ -59,4 +60,9 @@ public Iterable getRawOperatorStateInputs() { public Iterable getRawKeyedStateInputs() { return null; } + + @Override + public OptionalLong getRestoredCheckpointId() { + return OptionalLong.empty(); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStreamingRuntimeContext.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStreamingRuntimeContext.java index 888e349bdd90..e7be9b92d136 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStreamingRuntimeContext.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStreamingRuntimeContext.java @@ -17,10 +17,10 @@ package org.apache.hudi.sink.utils; -import org.apache.hudi.adapter.StreamingRuntimeContextAdapter; - import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.KeyedStateStore; +import org.apache.flink.metrics.groups.OperatorMetricGroup; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.operators.testutils.MockEnvironment; @@ -37,7 +37,7 @@ * *

NOTE: Adapted from Apache Flink, the MockStreamOperator is modified to support MapState. */ -public class MockStreamingRuntimeContext extends StreamingRuntimeContextAdapter { +public class MockStreamingRuntimeContext extends StreamingRuntimeContext { private final boolean isCheckpointingEnabled; @@ -128,4 +128,9 @@ public KeyedStateStore getKeyedStateStore() { return mockOperatorStateStore; } } + + @Override + public OperatorMetricGroup getMetricGroup() { + return UnregisteredMetricsGroup.createOperatorMetricGroup(); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java index 4d9ff3226b06..65d3f176d6a3 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java @@ -18,8 +18,8 @@ package org.apache.hudi.sink.utils; +import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -268,7 +268,8 @@ public TestHarness checkpoint(long checkpointId) throws Exception { * Stop the timeline server. */ public TestHarness stopTimelineServer() { - pipeline.getCoordinator().getWriteClient().getTimelineServer().ifPresent(EmbeddedTimelineService::stop); + HoodieFlinkWriteClient client = pipeline.getCoordinator().getWriteClient(); + client.getTimelineServer().ifPresent(embeddedTimelineService -> embeddedTimelineService.stopForBasePath(client.getConfig().getBasePath())); return this; } @@ -521,6 +522,7 @@ public TestHarness coordinatorFails() throws Exception { public void end() throws Exception { this.pipeline.close(); + this.pipeline = null; } private String lastPendingInstant() { diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java index 40fb28619de4..111bb42e73e3 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java @@ -18,7 +18,6 @@ package org.apache.hudi.table; -import org.apache.hudi.adapter.TestTableEnvs; import org.apache.hudi.common.model.DefaultHoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode; @@ -32,6 +31,7 @@ import org.apache.hudi.utils.TestConfigurations; import org.apache.hudi.utils.TestData; import org.apache.hudi.utils.TestSQL; +import org.apache.hudi.utils.TestTableEnvs; import org.apache.hudi.utils.TestUtils; import org.apache.hudi.utils.factory.CollectSinkTableFactory; diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java index c6522cf32d13..d3a48ae63b7a 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java @@ -109,6 +109,9 @@ void testRequiredOptions() { final MockContext sourceContext11 = MockContext.getInstance(this.conf, schema1, "f2"); assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext11)); assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSink(sourceContext11)); + //miss the pre combine key will be ok + HoodieTableSink tableSink11 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sourceContext11); + assertThat(tableSink11.getConf().getString(FlinkOptions.PRECOMBINE_FIELD), is(FlinkOptions.NO_PRE_COMBINE)); this.conf.set(FlinkOptions.OPERATION, FlinkOptions.OPERATION.defaultValue()); // a non-exists precombine key will throw exception @@ -140,6 +143,12 @@ void testRequiredOptions() { assertThat(tableSource.getConf().getString(FlinkOptions.PAYLOAD_CLASS_NAME), is(FlinkOptions.PAYLOAD_CLASS_NAME.defaultValue())); assertThat(tableSink.getConf().getString(FlinkOptions.PAYLOAD_CLASS_NAME), is(FlinkOptions.PAYLOAD_CLASS_NAME.defaultValue())); + // append mode given the pk but miss the pre combine key will be ok + this.conf.set(FlinkOptions.OPERATION, "insert"); + HoodieTableSink tableSink3 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sourceContext3); + assertThat(tableSink3.getConf().getString(FlinkOptions.PRECOMBINE_FIELD), is(FlinkOptions.NO_PRE_COMBINE)); + this.conf.set(FlinkOptions.OPERATION, FlinkOptions.OPERATION.defaultValue()); + this.conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, DefaultHoodieRecordPayload.class.getName()); final MockContext sourceContext4 = MockContext.getInstance(this.conf, schema3, "f2"); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogFactory.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogFactory.java index 6e7ee2e8f84b..5ee8aac90f80 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogFactory.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogFactory.java @@ -18,7 +18,7 @@ package org.apache.hudi.table.catalog; -import org.apache.hudi.adapter.TestTableEnvs; +import org.apache.hudi.utils.TestTableEnvs; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.TableEnvironment; diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestTableEnvs.java similarity index 98% rename from hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestTableEnvs.java index e65437609a21..fdec322fc9ac 100644 --- a/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestTableEnvs.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hudi.adapter; +package org.apache.hudi.utils; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; diff --git a/hudi-flink-datasource/hudi-flink1.13.x/pom.xml b/hudi-flink-datasource/hudi-flink1.13.x/pom.xml deleted file mode 100644 index 43cf20b379a4..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/pom.xml +++ /dev/null @@ -1,144 +0,0 @@ - - - - - hudi-flink-datasource - org.apache.hudi - 1.0.0-SNAPSHOT - - 4.0.0 - - hudi-flink1.13.x - 1.0.0-SNAPSHOT - 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-table-runtime-blink_${scala.binary.version} - ${flink1.13.version} - provided - - - org.apache.flink - flink-streaming-java_${scala.binary.version} - ${flink1.13.version} - provided - - - org.apache.flink - flink-core - ${flink1.13.version} - provided - - - org.apache.flink - flink-parquet_${scala.binary.version} - ${flink1.13.version} - provided - - - org.apache.flink - flink-json - ${flink1.13.version} - provided - - - org.apache.flink - flink-table-planner-blink_${scala.binary.version} - ${flink1.13.version} - provided - - - - - org.apache.flink - flink-runtime_${scala.binary.version} - ${flink1.13.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.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java deleted file mode 100644 index 51c53f368fb9..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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 { - @Override - public void close() throws Exception { - super.dispose(); - } - - public void finish() throws Exception { - super.close(); - } -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java deleted file mode 100644 index 0ea0968f1758..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.MailboxExecutor; -import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Adapter clazz for {@link AbstractStreamOperatorFactory}. - */ -public abstract class AbstractStreamOperatorFactoryAdapter - extends AbstractStreamOperatorFactory implements YieldingOperatorFactory { - private transient MailboxExecutor mailboxExecutor; - - @Override - public void setMailboxExecutor(MailboxExecutor mailboxExecutor) { - this.mailboxExecutor = mailboxExecutor; - } - - public MailboxExecutorAdapter getMailboxExecutorAdapter() { - return new MailboxExecutorAdapter(getMailboxExecutor()); - } - - /** - * Provides the mailbox executor iff this factory implements {@link YieldingOperatorFactory}. - */ - protected MailboxExecutor getMailboxExecutor() { - return checkNotNull( - mailboxExecutor, "Factory does not implement %s", YieldingOperatorFactory.class); - } -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/DataStreamScanProviderAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/DataStreamScanProviderAdapter.java deleted file mode 100644 index 867395c43f19..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/DataStreamScanProviderAdapter.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.source.DataStreamScanProvider; - -/** - * Adapter clazz for {@code DataStreamScanProvider}. - */ -public interface DataStreamScanProviderAdapter extends DataStreamScanProvider { -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/DataStreamSinkProviderAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/DataStreamSinkProviderAdapter.java deleted file mode 100644 index e8eaa3c62d44..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/DataStreamSinkProviderAdapter.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.sink.DataStreamSinkProvider; - -/** - * Adapter clazz for {@code DataStreamSinkProvider}. - */ -public interface DataStreamSinkProviderAdapter extends DataStreamSinkProvider { -} 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 deleted file mode 100644 index 94ed3b538879..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/HiveCatalogConstants.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.13.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java deleted file mode 100644 index 9ae3ca6912f6..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.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.13.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java deleted file mode 100644 index ea0ba0419214..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.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 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.13.x/src/main/java/org/apache/hudi/adapter/OperatorCoordinatorAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/OperatorCoordinatorAdapter.java deleted file mode 100644 index 887833c90e16..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/OperatorCoordinatorAdapter.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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; - -/** - * Adapter clazz for {@code OperatorCoordinator}. - */ -public interface OperatorCoordinatorAdapter extends OperatorCoordinator { -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java deleted file mode 100644 index 6d058de89bc5..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.guava18.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.13.x/src/main/java/org/apache/hudi/adapter/SortCodeGeneratorAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/SortCodeGeneratorAdapter.java deleted file mode 100644 index a3ee8e6eed17..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/SortCodeGeneratorAdapter.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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.api.TableConfig; -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(TableConfig conf, RowType input, SortSpec sortSpec) { - super(conf, input, sortSpec); - } -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/SupportsRowLevelDeleteAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/SupportsRowLevelDeleteAdapter.java deleted file mode 100644 index cd5c4eb891b0..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/SupportsRowLevelDeleteAdapter.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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; - -/** - * Adapter clazz for {@code org.apache.flink.table.connector.sink.abilities.SupportsRowLevelDelete}. - */ -public interface SupportsRowLevelDeleteAdapter { - - RowLevelDeleteInfoAdapter applyRowLevelDelete(); - - /** - * Adapter clazz for {@code SupportsRowLevelDelete.RowLevelDeleteInfo}. - */ - interface RowLevelDeleteInfoAdapter { - } -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/SupportsRowLevelUpdateAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/SupportsRowLevelUpdateAdapter.java deleted file mode 100644 index 6a62763ec5b7..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/SupportsRowLevelUpdateAdapter.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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 java.util.List; - -/** - * Adapter clazz for {@code org.apache.flink.table.connector.sink.abilities.SupportsRowLevelUpdate}. - */ -public interface SupportsRowLevelUpdateAdapter { - - RowLevelUpdateInfoAdapter applyRowLevelUpdate(List updatedColumns); - - /** - * Adapter clazz for {@code SupportsRowLevelUpdate.RowLevelUpdateInfo}. - */ - interface RowLevelUpdateInfoAdapter { - } -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/Utils.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/Utils.java deleted file mode 100644 index 521fd50c8d8a..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/Utils.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * 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.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; - -/** - * 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 - streamTask.getStreamStatusMaintainer(), - output, - watermarkInterval, - -1); - } - - public static FactoryUtil.DefaultDynamicTableContext getTableContext( - ObjectIdentifier tablePath, - ResolvedCatalogTable catalogTable, - ReadableConfig conf) { - return new FactoryUtil.DefaultDynamicTableContext(tablePath, catalogTable, - 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); - } -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/ColumnarArrayData.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/ColumnarArrayData.java deleted file mode 100644 index 20c63d26f749..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/ColumnarArrayData.java +++ /dev/null @@ -1,270 +0,0 @@ -/* - * 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.data; - -import org.apache.hudi.table.data.vector.MapColumnVector; -import org.apache.hudi.table.data.vector.RowColumnVector; - -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RawValueData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.data.binary.TypedSetters; -import org.apache.flink.table.data.vector.ArrayColumnVector; -import org.apache.flink.table.data.vector.BooleanColumnVector; -import org.apache.flink.table.data.vector.ByteColumnVector; -import org.apache.flink.table.data.vector.BytesColumnVector; -import org.apache.flink.table.data.vector.ColumnVector; -import org.apache.flink.table.data.vector.DecimalColumnVector; -import org.apache.flink.table.data.vector.DoubleColumnVector; -import org.apache.flink.table.data.vector.FloatColumnVector; -import org.apache.flink.table.data.vector.IntColumnVector; -import org.apache.flink.table.data.vector.LongColumnVector; -import org.apache.flink.table.data.vector.ShortColumnVector; -import org.apache.flink.table.data.vector.TimestampColumnVector; - -import java.util.Arrays; - -/** - * Columnar array to support access to vector column data. - * - *

References {@code org.apache.flink.table.data.ColumnarArrayData} to include FLINK-15390. - */ -public final class ColumnarArrayData implements ArrayData, TypedSetters { - - private final ColumnVector data; - private final int offset; - private final int numElements; - - public ColumnarArrayData(ColumnVector data, int offset, int numElements) { - this.data = data; - this.offset = offset; - this.numElements = numElements; - } - - @Override - public int size() { - return numElements; - } - - @Override - public boolean isNullAt(int pos) { - return data.isNullAt(offset + pos); - } - - @Override - public void setNullAt(int pos) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public boolean getBoolean(int pos) { - return ((BooleanColumnVector) data).getBoolean(offset + pos); - } - - @Override - public byte getByte(int pos) { - return ((ByteColumnVector) data).getByte(offset + pos); - } - - @Override - public short getShort(int pos) { - return ((ShortColumnVector) data).getShort(offset + pos); - } - - @Override - public int getInt(int pos) { - return ((IntColumnVector) data).getInt(offset + pos); - } - - @Override - public long getLong(int pos) { - return ((LongColumnVector) data).getLong(offset + pos); - } - - @Override - public float getFloat(int pos) { - return ((FloatColumnVector) data).getFloat(offset + pos); - } - - @Override - public double getDouble(int pos) { - return ((DoubleColumnVector) data).getDouble(offset + pos); - } - - @Override - public StringData getString(int pos) { - BytesColumnVector.Bytes byteArray = getByteArray(pos); - return StringData.fromBytes(byteArray.data, byteArray.offset, byteArray.len); - } - - @Override - public DecimalData getDecimal(int pos, int precision, int scale) { - return ((DecimalColumnVector) data).getDecimal(offset + pos, precision, scale); - } - - @Override - public TimestampData getTimestamp(int pos, int precision) { - return ((TimestampColumnVector) data).getTimestamp(offset + pos, precision); - } - - @Override - public RawValueData getRawValue(int pos) { - throw new UnsupportedOperationException("RawValueData is not supported."); - } - - @Override - public byte[] getBinary(int pos) { - BytesColumnVector.Bytes byteArray = getByteArray(pos); - if (byteArray.len == byteArray.data.length) { - return byteArray.data; - } else { - return Arrays.copyOfRange(byteArray.data, byteArray.offset, byteArray.len); - } - } - - @Override - public ArrayData getArray(int pos) { - return ((ArrayColumnVector) data).getArray(offset + pos); - } - - @Override - public MapData getMap(int pos) { - return ((MapColumnVector) data).getMap(offset + pos); - } - - @Override - public RowData getRow(int pos, int numFields) { - return ((RowColumnVector) data).getRow(offset + pos); - } - - @Override - public void setBoolean(int pos, boolean value) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setByte(int pos, byte value) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setShort(int pos, short value) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setInt(int pos, int value) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setLong(int pos, long value) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setFloat(int pos, float value) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setDouble(int pos, double value) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setDecimal(int pos, DecimalData value, int precision) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setTimestamp(int pos, TimestampData value, int precision) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public boolean[] toBooleanArray() { - boolean[] res = new boolean[numElements]; - for (int i = 0; i < numElements; i++) { - res[i] = getBoolean(i); - } - return res; - } - - @Override - public byte[] toByteArray() { - byte[] res = new byte[numElements]; - for (int i = 0; i < numElements; i++) { - res[i] = getByte(i); - } - return res; - } - - @Override - public short[] toShortArray() { - short[] res = new short[numElements]; - for (int i = 0; i < numElements; i++) { - res[i] = getShort(i); - } - return res; - } - - @Override - public int[] toIntArray() { - int[] res = new int[numElements]; - for (int i = 0; i < numElements; i++) { - res[i] = getInt(i); - } - return res; - } - - @Override - public long[] toLongArray() { - long[] res = new long[numElements]; - for (int i = 0; i < numElements; i++) { - res[i] = getLong(i); - } - return res; - } - - @Override - public float[] toFloatArray() { - float[] res = new float[numElements]; - for (int i = 0; i < numElements; i++) { - res[i] = getFloat(i); - } - return res; - } - - @Override - public double[] toDoubleArray() { - double[] res = new double[numElements]; - for (int i = 0; i < numElements; i++) { - res[i] = getDouble(i); - } - return res; - } - - private BytesColumnVector.Bytes getByteArray(int pos) { - return ((BytesColumnVector) data).getBytes(offset + pos); - } -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/ColumnarMapData.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/ColumnarMapData.java deleted file mode 100644 index bba462f404b3..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/ColumnarMapData.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.data; - -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.vector.ColumnVector; - -/** - * Columnar map to support access to vector column data. - * - *

Referenced from flink 1.14.0 {@code org.apache.flink.table.data.ColumnarMapData}. - */ -public final class ColumnarMapData implements MapData { - - private final ColumnVector keyColumnVector; - private final ColumnVector valueColumnVector; - private final int offset; - private final int numElements; - - public ColumnarMapData( - ColumnVector keyColumnVector, - ColumnVector valueColumnVector, - int offset, - int numElements) { - this.keyColumnVector = keyColumnVector; - this.valueColumnVector = valueColumnVector; - this.offset = offset; - this.numElements = numElements; - } - - @Override - public int size() { - return numElements; - } - - @Override - public ArrayData keyArray() { - return new ColumnarArrayData(keyColumnVector, offset, numElements); - } - - @Override - public ArrayData valueArray() { - return new ColumnarArrayData(valueColumnVector, offset, numElements); - } - - @Override - public boolean equals(Object o) { - throw new UnsupportedOperationException( - "ColumnarMapData do not support equals, please compare fields one by one!"); - } - - @Override - public int hashCode() { - throw new UnsupportedOperationException( - "ColumnarMapData do not support hashCode, please hash fields one by one!"); - } -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/ColumnarRowData.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/ColumnarRowData.java deleted file mode 100644 index 9a95035b2703..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/ColumnarRowData.java +++ /dev/null @@ -1,231 +0,0 @@ -/* - * 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.data; - -import org.apache.hudi.table.data.vector.VectorizedColumnBatch; - -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RawValueData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.data.binary.TypedSetters; -import org.apache.flink.table.data.vector.BytesColumnVector.Bytes; -import org.apache.flink.types.RowKind; - -/** - * Columnar row to support access to vector column data. - * It is a row view in {@link VectorizedColumnBatch}. - * - *

References {@code org.apache.flink.table.data.ColumnarRowData} to include FLINK-15390. - */ -public final class ColumnarRowData implements RowData, TypedSetters { - - private RowKind rowKind = RowKind.INSERT; - private VectorizedColumnBatch vectorizedColumnBatch; - private int rowId; - - public ColumnarRowData() { - } - - public ColumnarRowData(VectorizedColumnBatch vectorizedColumnBatch) { - this(vectorizedColumnBatch, 0); - } - - public ColumnarRowData(VectorizedColumnBatch vectorizedColumnBatch, int rowId) { - this.vectorizedColumnBatch = vectorizedColumnBatch; - this.rowId = rowId; - } - - public void setVectorizedColumnBatch(VectorizedColumnBatch vectorizedColumnBatch) { - this.vectorizedColumnBatch = vectorizedColumnBatch; - this.rowId = 0; - } - - public void setRowId(int rowId) { - this.rowId = rowId; - } - - @Override - public RowKind getRowKind() { - return rowKind; - } - - @Override - public void setRowKind(RowKind kind) { - this.rowKind = kind; - } - - @Override - public int getArity() { - return vectorizedColumnBatch.getArity(); - } - - @Override - public boolean isNullAt(int pos) { - return vectorizedColumnBatch.isNullAt(rowId, pos); - } - - @Override - public boolean getBoolean(int pos) { - return vectorizedColumnBatch.getBoolean(rowId, pos); - } - - @Override - public byte getByte(int pos) { - return vectorizedColumnBatch.getByte(rowId, pos); - } - - @Override - public short getShort(int pos) { - return vectorizedColumnBatch.getShort(rowId, pos); - } - - @Override - public int getInt(int pos) { - return vectorizedColumnBatch.getInt(rowId, pos); - } - - @Override - public long getLong(int pos) { - return vectorizedColumnBatch.getLong(rowId, pos); - } - - @Override - public float getFloat(int pos) { - return vectorizedColumnBatch.getFloat(rowId, pos); - } - - @Override - public double getDouble(int pos) { - return vectorizedColumnBatch.getDouble(rowId, pos); - } - - @Override - public StringData getString(int pos) { - Bytes byteArray = vectorizedColumnBatch.getByteArray(rowId, pos); - return StringData.fromBytes(byteArray.data, byteArray.offset, byteArray.len); - } - - @Override - public DecimalData getDecimal(int pos, int precision, int scale) { - return vectorizedColumnBatch.getDecimal(rowId, pos, precision, scale); - } - - @Override - public TimestampData getTimestamp(int pos, int precision) { - return vectorizedColumnBatch.getTimestamp(rowId, pos, precision); - } - - @Override - public RawValueData getRawValue(int pos) { - throw new UnsupportedOperationException("RawValueData is not supported."); - } - - @Override - public byte[] getBinary(int pos) { - Bytes byteArray = vectorizedColumnBatch.getByteArray(rowId, pos); - if (byteArray.len == byteArray.data.length) { - return byteArray.data; - } else { - byte[] ret = new byte[byteArray.len]; - System.arraycopy(byteArray.data, byteArray.offset, ret, 0, byteArray.len); - return ret; - } - } - - @Override - public RowData getRow(int pos, int numFields) { - return vectorizedColumnBatch.getRow(rowId, pos); - } - - @Override - public ArrayData getArray(int pos) { - return vectorizedColumnBatch.getArray(rowId, pos); - } - - @Override - public MapData getMap(int pos) { - return vectorizedColumnBatch.getMap(rowId, pos); - } - - @Override - public void setNullAt(int pos) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setBoolean(int pos, boolean value) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setByte(int pos, byte value) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setShort(int pos, short value) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setInt(int pos, int value) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setLong(int pos, long value) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setFloat(int pos, float value) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setDouble(int pos, double value) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setDecimal(int pos, DecimalData value, int precision) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public void setTimestamp(int pos, TimestampData value, int precision) { - throw new UnsupportedOperationException("Not support the operation!"); - } - - @Override - public boolean equals(Object o) { - throw new UnsupportedOperationException( - "ColumnarRowData do not support equals, please compare fields one by one!"); - } - - @Override - public int hashCode() { - throw new UnsupportedOperationException( - "ColumnarRowData do not support hashCode, please hash fields one by one!"); - } -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/vector/MapColumnVector.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/vector/MapColumnVector.java deleted file mode 100644 index 6bdf8782f4d3..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/vector/MapColumnVector.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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.data.vector; - -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.vector.ColumnVector; - -/** - * Map column vector. - */ -public interface MapColumnVector extends ColumnVector { - MapData getMap(int i); -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/vector/RowColumnVector.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/vector/RowColumnVector.java deleted file mode 100644 index bd0e9bbe7de7..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/vector/RowColumnVector.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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.data.vector; - -import org.apache.hudi.table.data.ColumnarRowData; - -import org.apache.flink.table.data.vector.ColumnVector; - -/** - * Row column vector. - */ -public interface RowColumnVector extends ColumnVector { - ColumnarRowData getRow(int i); -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/vector/VectorizedColumnBatch.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/vector/VectorizedColumnBatch.java deleted file mode 100644 index bccaec8fdcad..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/vector/VectorizedColumnBatch.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * 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.data.vector; - -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.data.vector.ArrayColumnVector; -import org.apache.flink.table.data.vector.BooleanColumnVector; -import org.apache.flink.table.data.vector.ByteColumnVector; -import org.apache.flink.table.data.vector.BytesColumnVector; -import org.apache.flink.table.data.vector.BytesColumnVector.Bytes; -import org.apache.flink.table.data.vector.ColumnVector; -import org.apache.flink.table.data.vector.DecimalColumnVector; -import org.apache.flink.table.data.vector.DoubleColumnVector; -import org.apache.flink.table.data.vector.FloatColumnVector; -import org.apache.flink.table.data.vector.IntColumnVector; -import org.apache.flink.table.data.vector.LongColumnVector; -import org.apache.flink.table.data.vector.ShortColumnVector; -import org.apache.flink.table.data.vector.TimestampColumnVector; - -import java.io.Serializable; -import java.nio.charset.StandardCharsets; - -/** - * A VectorizedColumnBatch is a set of rows, organized with each column as a vector. It is the unit - * of query execution, organized to minimize the cost per row. - * - *

{@code VectorizedColumnBatch}s are influenced by Apache Hive VectorizedRowBatch. - * - *

References {@code org.apache.flink.table.data.vector.VectorizedColumnBatch} to include FLINK-15390. - */ -public class VectorizedColumnBatch implements Serializable { - private static final long serialVersionUID = 8180323238728166155L; - - /** - * This number is carefully chosen to minimize overhead and typically allows one - * VectorizedColumnBatch to fit in cache. - */ - public static final int DEFAULT_SIZE = 2048; - - private int numRows; - public final ColumnVector[] columns; - - public VectorizedColumnBatch(ColumnVector[] vectors) { - this.columns = vectors; - } - - public void setNumRows(int numRows) { - this.numRows = numRows; - } - - public int getNumRows() { - return numRows; - } - - public int getArity() { - return columns.length; - } - - public boolean isNullAt(int rowId, int colId) { - return columns[colId].isNullAt(rowId); - } - - public boolean getBoolean(int rowId, int colId) { - return ((BooleanColumnVector) columns[colId]).getBoolean(rowId); - } - - public byte getByte(int rowId, int colId) { - return ((ByteColumnVector) columns[colId]).getByte(rowId); - } - - public short getShort(int rowId, int colId) { - return ((ShortColumnVector) columns[colId]).getShort(rowId); - } - - public int getInt(int rowId, int colId) { - return ((IntColumnVector) columns[colId]).getInt(rowId); - } - - public long getLong(int rowId, int colId) { - return ((LongColumnVector) columns[colId]).getLong(rowId); - } - - public float getFloat(int rowId, int colId) { - return ((FloatColumnVector) columns[colId]).getFloat(rowId); - } - - public double getDouble(int rowId, int colId) { - return ((DoubleColumnVector) columns[colId]).getDouble(rowId); - } - - public Bytes getByteArray(int rowId, int colId) { - return ((BytesColumnVector) columns[colId]).getBytes(rowId); - } - - private byte[] getBytes(int rowId, int colId) { - Bytes byteArray = getByteArray(rowId, colId); - if (byteArray.len == byteArray.data.length) { - return byteArray.data; - } else { - return byteArray.getBytes(); - } - } - - public String getString(int rowId, int colId) { - Bytes byteArray = getByteArray(rowId, colId); - return new String(byteArray.data, byteArray.offset, byteArray.len, StandardCharsets.UTF_8); - } - - public DecimalData getDecimal(int rowId, int colId, int precision, int scale) { - return ((DecimalColumnVector) (columns[colId])).getDecimal(rowId, precision, scale); - } - - public TimestampData getTimestamp(int rowId, int colId, int precision) { - return ((TimestampColumnVector) (columns[colId])).getTimestamp(rowId, precision); - } - - public ArrayData getArray(int rowId, int colId) { - return ((ArrayColumnVector) columns[colId]).getArray(rowId); - } - - public RowData getRow(int rowId, int colId) { - return ((RowColumnVector) columns[colId]).getRow(rowId); - } - - public MapData getMap(int rowId, int colId) { - return ((MapColumnVector) columns[colId]).getMap(rowId); - } -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java deleted file mode 100644 index ac9ca59d574d..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java +++ /dev/null @@ -1,579 +0,0 @@ -/* - * 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.data.vector.VectorizedColumnBatch; -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.vector.ColumnVector; -import org.apache.flink.table.data.vector.heap.HeapBooleanVector; -import org.apache.flink.table.data.vector.heap.HeapByteVector; -import org.apache.flink.table.data.vector.heap.HeapBytesVector; -import org.apache.flink.table.data.vector.heap.HeapDoubleVector; -import org.apache.flink.table.data.vector.heap.HeapFloatVector; -import org.apache.flink.table.data.vector.heap.HeapIntVector; -import org.apache.flink.table.data.vector.heap.HeapLongVector; -import org.apache.flink.table.data.vector.heap.HeapShortVector; -import org.apache.flink.table.data.vector.heap.HeapTimestampVector; -import org.apache.flink.table.data.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.runtime.functions.SqlDateTimeUtils.dateToInternal; -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 : dateToInternal((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.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java deleted file mode 100644 index 6d31d26b8d97..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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.hudi.table.data.ColumnarArrayData; - -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.vector.ArrayColumnVector; -import org.apache.flink.table.data.vector.ColumnVector; -import org.apache.flink.table.data.vector.heap.AbstractHeapVector; -import org.apache.flink.table.data.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.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java deleted file mode 100644 index cf39fc981624..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * 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.hudi.table.data.ColumnarMapData; -import org.apache.hudi.table.data.vector.MapColumnVector; - -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.vector.ColumnVector; -import org.apache.flink.table.data.vector.heap.AbstractHeapVector; -import org.apache.flink.table.data.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.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java deleted file mode 100644 index 03da9205d313..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.hudi.table.data.ColumnarRowData; -import org.apache.hudi.table.data.vector.RowColumnVector; -import org.apache.hudi.table.data.vector.VectorizedColumnBatch; - -import org.apache.flink.table.data.vector.heap.AbstractHeapVector; -import org.apache.flink.table.data.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.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/ParquetDecimalVector.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/ParquetDecimalVector.java deleted file mode 100644 index a2f6d5b0cd74..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/ParquetDecimalVector.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.vector.BytesColumnVector; -import org.apache.flink.table.data.vector.ColumnVector; -import org.apache.flink.table.data.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.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/AbstractColumnReader.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/AbstractColumnReader.java deleted file mode 100644 index 07416a371715..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/AbstractColumnReader.java +++ /dev/null @@ -1,325 +0,0 @@ -/* - * 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.vector.writable.WritableColumnVector; -import org.apache.flink.table.data.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.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java deleted file mode 100644 index 67dbb7490260..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java +++ /dev/null @@ -1,473 +0,0 @@ -/* - * 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.data.vector.VectorizedColumnBatch; -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.vector.heap.HeapBooleanVector; -import org.apache.flink.table.data.vector.heap.HeapByteVector; -import org.apache.flink.table.data.vector.heap.HeapBytesVector; -import org.apache.flink.table.data.vector.heap.HeapDoubleVector; -import org.apache.flink.table.data.vector.heap.HeapFloatVector; -import org.apache.flink.table.data.vector.heap.HeapIntVector; -import org.apache.flink.table.data.vector.heap.HeapLongVector; -import org.apache.flink.table.data.vector.heap.HeapShortVector; -import org.apache.flink.table.data.vector.heap.HeapTimestampVector; -import org.apache.flink.table.data.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.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/BaseVectorizedColumnReader.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/BaseVectorizedColumnReader.java deleted file mode 100644 index 073c704c4b24..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/BaseVectorizedColumnReader.java +++ /dev/null @@ -1,313 +0,0 @@ -/* - * 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.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.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/EmptyColumnReader.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/EmptyColumnReader.java deleted file mode 100644 index 8be29289bbab..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/EmptyColumnReader.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.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(); - } -} \ No newline at end of file diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/FixedLenBytesColumnReader.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/FixedLenBytesColumnReader.java deleted file mode 100644 index 6ebe5f1e6fbf..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/FixedLenBytesColumnReader.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * 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.vector.writable.WritableBytesVector; -import org.apache.flink.table.data.vector.writable.WritableColumnVector; -import org.apache.flink.table.data.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.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java deleted file mode 100644 index 70638a9c4320..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * 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.vector.writable.WritableIntVector; -import org.apache.flink.table.data.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.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java deleted file mode 100644 index 015a867c4f22..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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.vector.ColumnVector; -import org.apache.flink.table.data.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.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java deleted file mode 100644 index 9436305d2955..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java +++ /dev/null @@ -1,390 +0,0 @@ -/* - * 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.data.ColumnarRowData; -import org.apache.hudi.table.data.vector.VectorizedColumnBatch; -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.vector.ColumnVector; -import org.apache.flink.table.data.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.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReader.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReader.java deleted file mode 100644 index e96cf22d29ef..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReader.java +++ /dev/null @@ -1,199 +0,0 @@ -/* - * 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.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReaderFactory.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReaderFactory.java deleted file mode 100644 index 861d5cb00bbe..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReaderFactory.java +++ /dev/null @@ -1,304 +0,0 @@ -/* - * 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.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RowColumnReader.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RowColumnReader.java deleted file mode 100644 index 524c00f402d4..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RowColumnReader.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.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.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RunLengthDecoder.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RunLengthDecoder.java deleted file mode 100644 index 3266f835e4d1..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RunLengthDecoder.java +++ /dev/null @@ -1,304 +0,0 @@ -/* - * 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.vector.writable.WritableColumnVector; -import org.apache.flink.table.data.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.13.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java deleted file mode 100644 index 18686b811c40..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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; - -/** - * Adapter clazz for {@link Output}. - */ -public interface OutputAdapter extends Output { -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java deleted file mode 100644 index 8563d2422b64..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * 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; - -/** - * Adapter clazz for {@link StateInitializationContext}. - */ -public interface StateInitializationContextAdapter extends StateInitializationContext { -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java deleted file mode 100644 index 176783e8108c..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.MetricGroup; -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 MetricGroup getMetricGroup() { - return new UnregisteredMetricsGroup(); - } -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/TestStreamConfigs.java b/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/TestStreamConfigs.java deleted file mode 100644 index 4b62c790b582..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/TestStreamConfigs.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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); - } -} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java b/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java deleted file mode 100644 index e3088356709f..000000000000 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java +++ /dev/null @@ -1,34 +0,0 @@ -package org.apache.hudi.adapter; - -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.internal.TableEnvironmentImpl; - -/* - * 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. - */ - -/** - * TableEnv for test goals. - */ -public class TestTableEnvs { - - public static TableEnvironment getBatchTableEnv() { - EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); - return TableEnvironmentImpl.create(settings); - } -} diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java deleted file mode 100644 index d4c6bc3a8f4d..000000000000 --- a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.14.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java deleted file mode 100644 index 6dcfe71ccfd9..000000000000 --- a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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.14.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java deleted file mode 100644 index 0c836f3db391..000000000000 --- a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.14.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java deleted file mode 100644 index 865c0c81d4d9..000000000000 --- a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.14.x/src/main/java/org/apache/hudi/adapter/Utils.java b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/Utils.java index b5c83936b02c..9fd25f163147 100644 --- a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/Utils.java +++ b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/Utils.java @@ -22,13 +22,6 @@ 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.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.data.RowData; @@ -43,22 +36,6 @@ * 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, diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java b/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java deleted file mode 100644 index c0d83e6096e3..000000000000 --- a/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.14.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java b/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java deleted file mode 100644 index 1f76ad692f33..000000000000 --- a/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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 { - @Override - default OptionalLong getRestoredCheckpointId() { - return OptionalLong.empty(); - } -} diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java b/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java deleted file mode 100644 index 4461c28943d3..000000000000 --- a/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.15.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java b/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java deleted file mode 100644 index d4c6bc3a8f4d..000000000000 --- a/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.15.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java b/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java deleted file mode 100644 index 6dcfe71ccfd9..000000000000 --- a/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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.15.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java b/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java deleted file mode 100644 index 0c836f3db391..000000000000 --- a/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.15.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java b/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java deleted file mode 100644 index 865c0c81d4d9..000000000000 --- a/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.15.x/src/main/java/org/apache/hudi/adapter/Utils.java b/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/Utils.java index 7c8366dd381b..89ae23f6b649 100644 --- a/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/Utils.java +++ b/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/Utils.java @@ -22,13 +22,6 @@ 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.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.data.RowData; @@ -45,22 +38,6 @@ * 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, diff --git a/hudi-flink-datasource/hudi-flink1.15.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java b/hudi-flink-datasource/hudi-flink1.15.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java deleted file mode 100644 index c0d83e6096e3..000000000000 --- a/hudi-flink-datasource/hudi-flink1.15.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.15.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java b/hudi-flink-datasource/hudi-flink1.15.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java deleted file mode 100644 index c903ec2ed408..000000000000 --- a/hudi-flink-datasource/hudi-flink1.15.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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.15.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java b/hudi-flink-datasource/hudi-flink1.15.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java deleted file mode 100644 index 4461c28943d3..000000000000 --- a/hudi-flink-datasource/hudi-flink1.15.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.15.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java b/hudi-flink-datasource/hudi-flink1.15.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java deleted file mode 100644 index e65437609a21..000000000000 --- a/hudi-flink-datasource/hudi-flink1.15.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java deleted file mode 100644 index d4c6bc3a8f4d..000000000000 --- a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.16.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java deleted file mode 100644 index 6dcfe71ccfd9..000000000000 --- a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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.16.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java deleted file mode 100644 index 0c836f3db391..000000000000 --- a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.16.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java deleted file mode 100644 index 865c0c81d4d9..000000000000 --- a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.16.x/src/main/java/org/apache/hudi/adapter/Utils.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/Utils.java index 1112b7c7f69e..c418dc3d19db 100644 --- a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/Utils.java +++ b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/Utils.java @@ -22,13 +22,6 @@ 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.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.data.RowData; @@ -45,22 +38,6 @@ * 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, diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java b/hudi-flink-datasource/hudi-flink1.16.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java deleted file mode 100644 index c0d83e6096e3..000000000000 --- a/hudi-flink-datasource/hudi-flink1.16.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.16.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java b/hudi-flink-datasource/hudi-flink1.16.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java deleted file mode 100644 index c903ec2ed408..000000000000 --- a/hudi-flink-datasource/hudi-flink1.16.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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.16.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java b/hudi-flink-datasource/hudi-flink1.16.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java deleted file mode 100644 index 4461c28943d3..000000000000 --- a/hudi-flink-datasource/hudi-flink1.16.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.16.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java b/hudi-flink-datasource/hudi-flink1.16.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java deleted file mode 100644 index e65437609a21..000000000000 --- a/hudi-flink-datasource/hudi-flink1.16.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java b/hudi-flink-datasource/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java deleted file mode 100644 index d4c6bc3a8f4d..000000000000 --- a/hudi-flink-datasource/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.17.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java b/hudi-flink-datasource/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java deleted file mode 100644 index 6dcfe71ccfd9..000000000000 --- a/hudi-flink-datasource/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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.17.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java b/hudi-flink-datasource/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java deleted file mode 100644 index 0c836f3db391..000000000000 --- a/hudi-flink-datasource/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.17.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java b/hudi-flink-datasource/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java deleted file mode 100644 index 865c0c81d4d9..000000000000 --- a/hudi-flink-datasource/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.17.x/src/main/java/org/apache/hudi/adapter/Utils.java b/hudi-flink-datasource/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/Utils.java index 659c65973674..a0c7b36420b9 100644 --- a/hudi-flink-datasource/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/Utils.java +++ b/hudi-flink-datasource/hudi-flink1.17.x/src/main/java/org/apache/hudi/adapter/Utils.java @@ -22,13 +22,6 @@ 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; @@ -46,22 +39,6 @@ * 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, diff --git a/hudi-flink-datasource/hudi-flink1.17.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java b/hudi-flink-datasource/hudi-flink1.17.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java deleted file mode 100644 index c0d83e6096e3..000000000000 --- a/hudi-flink-datasource/hudi-flink1.17.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.17.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java b/hudi-flink-datasource/hudi-flink1.17.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java deleted file mode 100644 index c903ec2ed408..000000000000 --- a/hudi-flink-datasource/hudi-flink1.17.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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.17.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java b/hudi-flink-datasource/hudi-flink1.17.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java deleted file mode 100644 index 4461c28943d3..000000000000 --- a/hudi-flink-datasource/hudi-flink1.17.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.17.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java b/hudi-flink-datasource/hudi-flink1.17.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java deleted file mode 100644 index e65437609a21..000000000000 --- a/hudi-flink-datasource/hudi-flink1.17.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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/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 deleted file mode 100644 index d4c6bc3a8f4d..000000000000 --- a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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 deleted file mode 100644 index 6dcfe71ccfd9..000000000000 --- a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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/MailboxExecutorAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java deleted file mode 100644 index 0c836f3db391..000000000000 --- a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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/RateLimiterAdapter.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java deleted file mode 100644 index 865c0c81d4d9..000000000000 --- a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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/Utils.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/adapter/Utils.java index 659c65973674..fe0351af4310 100644 --- 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 @@ -22,13 +22,6 @@ 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; @@ -46,22 +39,6 @@ * 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, @@ -70,7 +47,7 @@ public static FactoryUtil.DefaultDynamicTableContext getTableContext( Collections.emptyMap(), conf, Thread.currentThread().getContextClassLoader(), false); } - public static BinaryExternalSorter getBinaryExternalSorter( + public static BinaryExternalSorter getBinaryExternalSorter( final Object owner, MemoryManager memoryManager, long reservedMemorySize, 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 deleted file mode 100644 index c0d83e6096e3..000000000000 --- a/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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 deleted file mode 100644 index c903ec2ed408..000000000000 --- a/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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 deleted file mode 100644 index 4461c28943d3..000000000000 --- a/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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/TestTableEnvs.java b/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java deleted file mode 100644 index e65437609a21..000000000000 --- a/hudi-flink-datasource/hudi-flink1.18.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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 671b1eb0ae7c..615536657e00 100644 --- a/hudi-flink-datasource/pom.xml +++ b/hudi-flink-datasource/pom.xml @@ -33,7 +33,6 @@ - hudi-flink1.13.x hudi-flink1.14.x hudi-flink1.15.x hudi-flink1.16.x diff --git a/hudi-gcp/pom.xml b/hudi-gcp/pom.xml index b1cfb8076a6d..2c308fbf4244 100644 --- a/hudi-gcp/pom.xml +++ b/hudi-gcp/pom.xml @@ -36,7 +36,7 @@ See https://github.com/GoogleCloudPlatform/cloud-opensource-java/wiki/The-Google com.google.cloud libraries-bom - 25.1.0 + 26.15.0 pom import @@ -70,7 +70,6 @@ See https://github.com/GoogleCloudPlatform/cloud-opensource-java/wiki/The-Google com.google.cloud google-cloud-pubsub - ${google.cloud.pubsub.version} diff --git a/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncConfig.java b/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncConfig.java index 94510ca8dfa3..ed8895ca217c 100644 --- a/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncConfig.java +++ b/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncConfig.java @@ -122,6 +122,20 @@ public class BigQuerySyncConfig extends HoodieSyncConfig implements Serializable .markAdvanced() .withDocumentation("Fetch file listing from Hudi's metadata"); + public static final ConfigProperty BIGQUERY_SYNC_REQUIRE_PARTITION_FILTER = ConfigProperty + .key("hoodie.gcp.bigquery.sync.require_partition_filter") + .defaultValue(false) + .sinceVersion("0.14.1") + .markAdvanced() + .withDocumentation("If true, configure table to require a partition filter to be specified when querying the table"); + + public static final ConfigProperty BIGQUERY_SYNC_BIG_LAKE_CONNECTION_ID = ConfigProperty + .key("hoodie.gcp.bigquery.sync.big_lake_connection_id") + .noDefaultValue() + .sinceVersion("0.14.1") + .markAdvanced() + .withDocumentation("The Big Lake connection ID to use"); + public BigQuerySyncConfig(Properties props) { super(props); setDefaults(BigQuerySyncConfig.class.getName()); @@ -147,6 +161,10 @@ public static class BigQuerySyncConfigParams { public String sourceUri; @Parameter(names = {"--source-uri-prefix"}, description = "Name of the source uri gcs path prefix of the table", required = false) public String sourceUriPrefix; + @Parameter(names = {"--big-lake-connection-id"}, description = "The Big Lake connection ID to use when creating the table if using the manifest file approach.") + public String bigLakeConnectionId; + @Parameter(names = {"--require-partition-filter"}, description = "If true, configure table to require a partition filter to be specified when querying the table") + public Boolean requirePartitionFilter; public boolean isHelp() { return hoodieSyncConfigParams.isHelp(); @@ -164,6 +182,8 @@ public TypedProperties toProps() { props.setPropertyIfNonNull(BIGQUERY_SYNC_SYNC_BASE_PATH.key(), hoodieSyncConfigParams.basePath); props.setPropertyIfNonNull(BIGQUERY_SYNC_PARTITION_FIELDS.key(), StringUtils.join(",", hoodieSyncConfigParams.partitionFields)); props.setPropertyIfNonNull(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA.key(), hoodieSyncConfigParams.useFileListingFromMetadata); + props.setPropertyIfNonNull(BIGQUERY_SYNC_BIG_LAKE_CONNECTION_ID.key(), bigLakeConnectionId); + props.setPropertyIfNonNull(BIGQUERY_SYNC_REQUIRE_PARTITION_FILTER.key(), requirePartitionFilter); return props; } } diff --git a/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncTool.java b/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncTool.java index 19c8449f8fa1..28c071e52315 100644 --- a/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncTool.java +++ b/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncTool.java @@ -122,7 +122,7 @@ private boolean tableExists(HoodieBigQuerySyncClient bqSyncClient, String tableN } private void syncTable(HoodieBigQuerySyncClient bqSyncClient) { - LOG.info("Sync hoodie table " + snapshotViewName + " at base path " + bqSyncClient.getBasePath()); + LOG.info("Sync hoodie table {} at base path {}", snapshotViewName, bqSyncClient.getBasePath()); if (!bqSyncClient.datasetExists()) { throw new HoodieBigQuerySyncException("Dataset not found: " + config.getString(BIGQUERY_SYNC_DATASET_NAME)); @@ -132,19 +132,21 @@ private void syncTable(HoodieBigQuerySyncClient bqSyncClient) { Schema latestSchema = bqSchemaResolver.getTableSchema(metaClient, partitionFields); if (config.getBoolean(BIGQUERY_SYNC_USE_BQ_MANIFEST_FILE)) { manifestFileWriter.writeManifestFile(true); - if (!tableExists(bqSyncClient, tableName)) { - bqSyncClient.createTableUsingBqManifestFile( + // if table does not exist, create it using the manifest file + // if table exists but is not yet using manifest file or needs to be recreated with the big-lake connection ID, update it to use manifest file + if (bqSyncClient.tableNotExistsOrDoesNotMatchSpecification(tableName)) { + bqSyncClient.createOrUpdateTableUsingBqManifestFile( tableName, manifestFileWriter.getManifestSourceUri(true), config.getString(BIGQUERY_SYNC_SOURCE_URI_PREFIX), latestSchema); - LOG.info("Completed table " + tableName + " creation using the manifest file"); + LOG.info("Completed table {} creation using the manifest file", tableName); } else { bqSyncClient.updateTableSchema(tableName, latestSchema, partitionFields); - LOG.info("Synced schema for " + tableName); + LOG.info("Synced schema for {}", tableName); } - LOG.info("Sync table complete for " + tableName); + LOG.info("Sync table complete for {}", tableName); return; } @@ -152,7 +154,7 @@ private void syncTable(HoodieBigQuerySyncClient bqSyncClient) { if (!tableExists(bqSyncClient, manifestTableName)) { bqSyncClient.createManifestTable(manifestTableName, manifestFileWriter.getManifestSourceUri(false)); - LOG.info("Manifest table creation complete for " + manifestTableName); + LOG.info("Manifest table creation complete for {}", manifestTableName); } if (!tableExists(bqSyncClient, versionsTableName)) { @@ -161,16 +163,15 @@ private void syncTable(HoodieBigQuerySyncClient bqSyncClient) { config.getString(BIGQUERY_SYNC_SOURCE_URI), config.getString(BIGQUERY_SYNC_SOURCE_URI_PREFIX), config.getSplitStrings(BIGQUERY_SYNC_PARTITION_FIELDS)); - LOG.info("Versions table creation complete for " + versionsTableName); + LOG.info("Versions table creation complete for {}", versionsTableName); } if (!tableExists(bqSyncClient, snapshotViewName)) { bqSyncClient.createSnapshotView(snapshotViewName, versionsTableName, manifestTableName); - LOG.info("Snapshot view creation complete for " + snapshotViewName); + LOG.info("Snapshot view creation complete for {}", snapshotViewName); } - // TODO: Implement automatic schema evolution when you add a new column. - LOG.info("Sync table complete for " + snapshotViewName); + LOG.info("Sync table complete for {}", snapshotViewName); } @Override diff --git a/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/HoodieBigQuerySyncClient.java b/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/HoodieBigQuerySyncClient.java index a5462b5669e2..af56194214df 100644 --- a/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/HoodieBigQuerySyncClient.java +++ b/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/HoodieBigQuerySyncClient.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.sync.common.HoodieSyncClient; +import org.apache.hudi.sync.common.util.ManifestFileWriter; import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.BigQueryException; @@ -51,9 +52,11 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_BIG_LAKE_CONNECTION_ID; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_LOCATION; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_NAME; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_PROJECT_ID; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_REQUIRE_PARTITION_FILTER; public class HoodieBigQuerySyncClient extends HoodieSyncClient { @@ -61,14 +64,18 @@ public class HoodieBigQuerySyncClient extends HoodieSyncClient { protected final BigQuerySyncConfig config; private final String projectId; + private final String bigLakeConnectionId; private final String datasetName; + private final boolean requirePartitionFilter; private transient BigQuery bigquery; public HoodieBigQuerySyncClient(final BigQuerySyncConfig config) { super(config); this.config = config; this.projectId = config.getString(BIGQUERY_SYNC_PROJECT_ID); + this.bigLakeConnectionId = config.getString(BIGQUERY_SYNC_BIG_LAKE_CONNECTION_ID); this.datasetName = config.getString(BIGQUERY_SYNC_DATASET_NAME); + this.requirePartitionFilter = config.getBoolean(BIGQUERY_SYNC_REQUIRE_PARTITION_FILTER); this.createBigQueryConnection(); } @@ -78,7 +85,9 @@ public HoodieBigQuerySyncClient(final BigQuerySyncConfig config) { this.config = config; this.projectId = config.getString(BIGQUERY_SYNC_PROJECT_ID); this.datasetName = config.getString(BIGQUERY_SYNC_DATASET_NAME); + this.requirePartitionFilter = config.getBoolean(BIGQUERY_SYNC_REQUIRE_PARTITION_FILTER); this.bigquery = bigquery; + this.bigLakeConnectionId = config.getString(BIGQUERY_SYNC_BIG_LAKE_CONNECTION_ID); } private void createBigQueryConnection() { @@ -94,19 +103,22 @@ private void createBigQueryConnection() { } } - public void createTableUsingBqManifestFile(String tableName, String bqManifestFileUri, String sourceUriPrefix, Schema schema) { + public void createOrUpdateTableUsingBqManifestFile(String tableName, String bqManifestFileUri, String sourceUriPrefix, Schema schema) { try { String withClauses = String.format("( %s )", BigQuerySchemaResolver.schemaToSqlString(schema)); String extraOptions = "enable_list_inference=true,"; if (!StringUtils.isNullOrEmpty(sourceUriPrefix)) { withClauses += " WITH PARTITION COLUMNS"; - extraOptions += String.format(" hive_partition_uri_prefix=\"%s\",", sourceUriPrefix); + extraOptions += String.format(" hive_partition_uri_prefix=\"%s\", require_hive_partition_filter=%s,", sourceUriPrefix, requirePartitionFilter); + } + if (!StringUtils.isNullOrEmpty(bigLakeConnectionId)) { + withClauses += String.format(" WITH CONNECTION `%s`", bigLakeConnectionId); } String query = String.format( - "CREATE EXTERNAL TABLE `%s.%s.%s` %s OPTIONS (%s " - + "uris=[\"%s\"], format=\"PARQUET\", file_set_spec_type=\"NEW_LINE_DELIMITED_MANIFEST\")", + "CREATE OR REPLACE EXTERNAL TABLE `%s.%s.%s` %s OPTIONS (%s " + + "uris=[\"%s\"], format=\"PARQUET\", file_set_spec_type=\"NEW_LINE_DELIMITED_MANIFEST\")", projectId, datasetName, tableName, @@ -125,7 +137,7 @@ public void createTableUsingBqManifestFile(String tableName, String bqManifestFi if (queryJob == null) { LOG.error("Job for table creation no longer exists"); } else if (queryJob.getStatus().getError() != null) { - LOG.error("Job for table creation failed: " + queryJob.getStatus().getError().toString()); + LOG.error("Job for table creation failed: {}", queryJob.getStatus().getError().toString()); } else { LOG.info("External table created using manifest file."); } @@ -176,13 +188,21 @@ public void updateTableSchema(String tableName, Schema schema, List part .collect(Collectors.toList()); updatedTableFields.addAll(schema.getFields()); Schema finalSchema = Schema.of(updatedTableFields); - if (definition.getSchema() != null && definition.getSchema().equals(finalSchema)) { + boolean sameSchema = definition.getSchema() != null && definition.getSchema().equals(finalSchema); + boolean samePartitionFilter = partitionFields.isEmpty() + || (requirePartitionFilter == (definition.getHivePartitioningOptions().getRequirePartitionFilter() != null && definition.getHivePartitioningOptions().getRequirePartitionFilter())); + if (sameSchema && samePartitionFilter) { return; // No need to update schema. } + ExternalTableDefinition.Builder builder = definition.toBuilder(); + builder.setSchema(finalSchema); + builder.setAutodetect(false); + if (definition.getHivePartitioningOptions() != null) { + builder.setHivePartitioningOptions(definition.getHivePartitioningOptions().toBuilder().setRequirePartitionFilter(requirePartitionFilter).build()); + } Table updatedTable = existingTable.toBuilder() - .setDefinition(definition.toBuilder().setSchema(finalSchema).setAutodetect(false).build()) + .setDefinition(builder.build()) .build(); - bigquery.update(updatedTable); } @@ -264,6 +284,28 @@ public boolean tableExists(String tableName) { return table != null && table.exists(); } + /** + * Checks for the existence of a table that uses the manifest file approach and matches other requirements. + * @param tableName name of the table + * @return Returns true if the table does not exist or if the table does exist but does not use the manifest file. False otherwise. + */ + public boolean tableNotExistsOrDoesNotMatchSpecification(String tableName) { + TableId tableId = TableId.of(projectId, datasetName, tableName); + Table table = bigquery.getTable(tableId); + if (table == null || !table.exists()) { + return true; + } + ExternalTableDefinition externalTableDefinition = table.getDefinition(); + boolean manifestDoesNotExist = + externalTableDefinition.getSourceUris() == null + || externalTableDefinition.getSourceUris().stream().noneMatch(uri -> uri.contains(ManifestFileWriter.ABSOLUTE_PATH_MANIFEST_FOLDER_NAME)); + if (!StringUtils.isNullOrEmpty(config.getString(BIGQUERY_SYNC_BIG_LAKE_CONNECTION_ID))) { + // If bigLakeConnectionId is present and connectionId is not present in table definition, we need to replace the table. + return manifestDoesNotExist || externalTableDefinition.getConnectionId() == null; + } + return manifestDoesNotExist; + } + @Override public void close() { bigquery = null; diff --git a/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncConfig.java b/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncConfig.java index b5d812cce10c..2c17749158f8 100644 --- a/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncConfig.java +++ b/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncConfig.java @@ -33,11 +33,11 @@ import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_NAME; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_PARTITION_FIELDS; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_PROJECT_ID; -import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_USE_BQ_MANIFEST_FILE; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_SOURCE_URI; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_SOURCE_URI_PREFIX; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_SYNC_BASE_PATH; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_TABLE_NAME; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_USE_BQ_MANIFEST_FILE; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; diff --git a/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncTool.java b/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncTool.java index 5edbdac1c2e8..ff7abdb68703 100644 --- a/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncTool.java +++ b/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncTool.java @@ -76,13 +76,13 @@ void useBQManifestFile_newTablePartitioned() { properties.setProperty(BigQuerySyncConfig.BIGQUERY_SYNC_PARTITION_FIELDS.key(), "datestr,type"); when(mockBqSyncClient.getTableType()).thenReturn(HoodieTableType.COPY_ON_WRITE); when(mockBqSyncClient.datasetExists()).thenReturn(true); - when(mockBqSyncClient.tableExists(TEST_TABLE)).thenReturn(false); + when(mockBqSyncClient.tableNotExistsOrDoesNotMatchSpecification(TEST_TABLE)).thenReturn(true); Path manifestPath = new Path("file:///local/path"); when(mockManifestFileWriter.getManifestSourceUri(true)).thenReturn(manifestPath.toUri().getPath()); when(mockBqSchemaResolver.getTableSchema(any(), eq(Arrays.asList("datestr", "type")))).thenReturn(schema); BigQuerySyncTool tool = new BigQuerySyncTool(properties, mockManifestFileWriter, mockBqSyncClient, mockMetaClient, mockBqSchemaResolver); tool.syncHoodieTable(); - verify(mockBqSyncClient).createTableUsingBqManifestFile(TEST_TABLE, manifestPath.toUri().getPath(), prefix, schema); + verify(mockBqSyncClient).createOrUpdateTableUsingBqManifestFile(TEST_TABLE, manifestPath.toUri().getPath(), prefix, schema); verify(mockManifestFileWriter).writeManifestFile(true); } @@ -91,13 +91,13 @@ void useBQManifestFile_newTableNonPartitioned() { properties.setProperty(BigQuerySyncConfig.BIGQUERY_SYNC_USE_BQ_MANIFEST_FILE.key(), "true"); when(mockBqSyncClient.getTableType()).thenReturn(HoodieTableType.COPY_ON_WRITE); when(mockBqSyncClient.datasetExists()).thenReturn(true); - when(mockBqSyncClient.tableExists(TEST_TABLE)).thenReturn(false); + when(mockBqSyncClient.tableNotExistsOrDoesNotMatchSpecification(TEST_TABLE)).thenReturn(true); Path manifestPath = new Path("file:///local/path"); when(mockManifestFileWriter.getManifestSourceUri(true)).thenReturn(manifestPath.toUri().getPath()); when(mockBqSchemaResolver.getTableSchema(any(), eq(Collections.emptyList()))).thenReturn(schema); BigQuerySyncTool tool = new BigQuerySyncTool(properties, mockManifestFileWriter, mockBqSyncClient, mockMetaClient, mockBqSchemaResolver); tool.syncHoodieTable(); - verify(mockBqSyncClient).createTableUsingBqManifestFile(TEST_TABLE, manifestPath.toUri().getPath(), null, schema); + verify(mockBqSyncClient).createOrUpdateTableUsingBqManifestFile(TEST_TABLE, manifestPath.toUri().getPath(), null, schema); verify(mockManifestFileWriter).writeManifestFile(true); } @@ -109,7 +109,7 @@ void useBQManifestFile_existingPartitionedTable() { properties.setProperty(BigQuerySyncConfig.BIGQUERY_SYNC_PARTITION_FIELDS.key(), "datestr,type"); when(mockBqSyncClient.getTableType()).thenReturn(HoodieTableType.COPY_ON_WRITE); when(mockBqSyncClient.datasetExists()).thenReturn(true); - when(mockBqSyncClient.tableExists(TEST_TABLE)).thenReturn(true); + when(mockBqSyncClient.tableNotExistsOrDoesNotMatchSpecification(TEST_TABLE)).thenReturn(false); Path manifestPath = new Path("file:///local/path"); when(mockManifestFileWriter.getManifestSourceUri(true)).thenReturn(manifestPath.toUri().getPath()); List partitionFields = Arrays.asList("datestr", "type"); @@ -125,7 +125,7 @@ void useBQManifestFile_existingNonPartitionedTable() { properties.setProperty(BigQuerySyncConfig.BIGQUERY_SYNC_USE_BQ_MANIFEST_FILE.key(), "true"); when(mockBqSyncClient.getTableType()).thenReturn(HoodieTableType.COPY_ON_WRITE); when(mockBqSyncClient.datasetExists()).thenReturn(true); - when(mockBqSyncClient.tableExists(TEST_TABLE)).thenReturn(true); + when(mockBqSyncClient.tableNotExistsOrDoesNotMatchSpecification(TEST_TABLE)).thenReturn(false); Path manifestPath = new Path("file:///local/path"); when(mockManifestFileWriter.getManifestSourceUri(true)).thenReturn(manifestPath.toUri().getPath()); when(mockBqSchemaResolver.getTableSchema(any(), eq(Collections.emptyList()))).thenReturn(schema); diff --git a/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncToolArgs.java b/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncToolArgs.java index 0a4ba6fd61a8..3f8ee6b9966d 100644 --- a/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncToolArgs.java +++ b/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncToolArgs.java @@ -23,10 +23,12 @@ import java.util.Properties; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_BIG_LAKE_CONNECTION_ID; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_LOCATION; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_NAME; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_PARTITION_FIELDS; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_PROJECT_ID; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_REQUIRE_PARTITION_FILTER; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_USE_BQ_MANIFEST_FILE; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_SOURCE_URI; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_SOURCE_URI_PREFIX; @@ -50,8 +52,10 @@ public void testArgsParse() { "--source-uri-prefix", "gs://foobartable/", "--base-path", "gs://foobartable", "--partitioned-by", "year,month,day", + "--big-lake-connection-id", "connection-id", "--use-bq-manifest-file", - "--use-file-listing-from-metadata" + "--use-file-listing-from-metadata", + "--require-partition-filter" }; cmd.parse(args); @@ -66,5 +70,7 @@ public void testArgsParse() { assertEquals("year,month,day", props.getProperty(BIGQUERY_SYNC_PARTITION_FIELDS.key())); assertEquals("true", props.getProperty(BIGQUERY_SYNC_USE_BQ_MANIFEST_FILE.key())); assertEquals("true", props.getProperty(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA.key())); + assertEquals("true", props.getProperty(BIGQUERY_SYNC_REQUIRE_PARTITION_FILTER.key())); + assertEquals("connection-id", props.getProperty(BIGQUERY_SYNC_BIG_LAKE_CONNECTION_ID.key())); } } diff --git a/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestHoodieBigQuerySyncClient.java b/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestHoodieBigQuerySyncClient.java index af2167f0f160..37b2800b563d 100644 --- a/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestHoodieBigQuerySyncClient.java +++ b/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestHoodieBigQuerySyncClient.java @@ -58,6 +58,7 @@ public class TestHoodieBigQuerySyncClient { private static String basePath; private final BigQuery mockBigQuery = mock(BigQuery.class); private HoodieBigQuerySyncClient client; + private Properties properties; @BeforeAll static void setupOnce() throws Exception { @@ -71,16 +72,19 @@ static void setupOnce() throws Exception { @BeforeEach void setup() { - Properties properties = new Properties(); + properties = new Properties(); properties.setProperty(BigQuerySyncConfig.BIGQUERY_SYNC_PROJECT_ID.key(), PROJECT_ID); properties.setProperty(BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_NAME.key(), TEST_DATASET); properties.setProperty(HoodieSyncConfig.META_SYNC_BASE_PATH.key(), tempDir.toString()); - BigQuerySyncConfig config = new BigQuerySyncConfig(properties); - client = new HoodieBigQuerySyncClient(config, mockBigQuery); + properties.setProperty(BigQuerySyncConfig.BIGQUERY_SYNC_REQUIRE_PARTITION_FILTER.key(), "true"); } @Test void createTableWithManifestFile_partitioned() throws Exception { + properties.setProperty(BigQuerySyncConfig.BIGQUERY_SYNC_BIG_LAKE_CONNECTION_ID.key(), "my-project.us.bl_connection"); + BigQuerySyncConfig config = new BigQuerySyncConfig(properties); + client = new HoodieBigQuerySyncClient(config, mockBigQuery); + Schema schema = Schema.of(Field.of("field", StandardSQLTypeName.STRING)); ArgumentCaptor jobInfoCaptor = ArgumentCaptor.forClass(JobInfo.class); Job mockJob = mock(Job.class); @@ -90,17 +94,21 @@ void createTableWithManifestFile_partitioned() throws Exception { JobStatus mockJobStatus = mock(JobStatus.class); when(mockJobFinished.getStatus()).thenReturn(mockJobStatus); when(mockJobStatus.getError()).thenReturn(null); - client.createTableUsingBqManifestFile(TEST_TABLE, MANIFEST_FILE_URI, SOURCE_PREFIX, schema); + client.createOrUpdateTableUsingBqManifestFile(TEST_TABLE, MANIFEST_FILE_URI, SOURCE_PREFIX, schema); QueryJobConfiguration configuration = jobInfoCaptor.getValue().getConfiguration(); assertEquals(configuration.getQuery(), - String.format("CREATE EXTERNAL TABLE `%s.%s.%s` ( `field` STRING ) WITH PARTITION COLUMNS OPTIONS (enable_list_inference=true, " - + "hive_partition_uri_prefix=\"%s\", uris=[\"%s\"], format=\"PARQUET\", " - + "file_set_spec_type=\"NEW_LINE_DELIMITED_MANIFEST\")", PROJECT_ID, TEST_DATASET, TEST_TABLE, SOURCE_PREFIX, MANIFEST_FILE_URI)); + String.format("CREATE OR REPLACE EXTERNAL TABLE `%s.%s.%s` ( `field` STRING ) WITH PARTITION COLUMNS WITH CONNECTION `my-project.us.bl_connection` " + + "OPTIONS (enable_list_inference=true, hive_partition_uri_prefix=\"%s\", " + + "require_hive_partition_filter=true, uris=[\"%s\"], format=\"PARQUET\", file_set_spec_type=\"NEW_LINE_DELIMITED_MANIFEST\")", + PROJECT_ID, TEST_DATASET, TEST_TABLE, SOURCE_PREFIX, MANIFEST_FILE_URI)); } @Test void createTableWithManifestFile_nonPartitioned() throws Exception { + BigQuerySyncConfig config = new BigQuerySyncConfig(properties); + client = new HoodieBigQuerySyncClient(config, mockBigQuery); + Schema schema = Schema.of(Field.of("field", StandardSQLTypeName.STRING)); ArgumentCaptor jobInfoCaptor = ArgumentCaptor.forClass(JobInfo.class); Job mockJob = mock(Job.class); @@ -110,11 +118,11 @@ void createTableWithManifestFile_nonPartitioned() throws Exception { JobStatus mockJobStatus = mock(JobStatus.class); when(mockJobFinished.getStatus()).thenReturn(mockJobStatus); when(mockJobStatus.getError()).thenReturn(null); - client.createTableUsingBqManifestFile(TEST_TABLE, MANIFEST_FILE_URI, "", schema); + client.createOrUpdateTableUsingBqManifestFile(TEST_TABLE, MANIFEST_FILE_URI, "", schema); QueryJobConfiguration configuration = jobInfoCaptor.getValue().getConfiguration(); assertEquals(configuration.getQuery(), - String.format("CREATE EXTERNAL TABLE `%s.%s.%s` ( `field` STRING ) OPTIONS (enable_list_inference=true, uris=[\"%s\"], format=\"PARQUET\", " + String.format("CREATE OR REPLACE EXTERNAL TABLE `%s.%s.%s` ( `field` STRING ) OPTIONS (enable_list_inference=true, uris=[\"%s\"], format=\"PARQUET\", " + "file_set_spec_type=\"NEW_LINE_DELIMITED_MANIFEST\")", PROJECT_ID, TEST_DATASET, TEST_TABLE, MANIFEST_FILE_URI)); } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieTableFileIndex.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieTableFileIndex.java index 7cfa624c764c..e8953450d5f0 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieTableFileIndex.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieTableFileIndex.java @@ -58,7 +58,9 @@ public HiveHoodieTableFileIndex(HoodieEngineContext engineContext, shouldIncludePendingCommits, true, new NoopCache(), - false); + false, + Option.empty(), + Option.empty()); } /** diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java index 75504cdd132d..04b65d8878aa 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java @@ -20,6 +20,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieLogFile; @@ -27,11 +28,13 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.table.view.FileSystemViewManager; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -52,6 +55,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; @@ -59,7 +63,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.hudi.common.util.ValidationUtils.checkState; +import static org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE; /** * Base implementation of the Hive's {@link FileInputFormat} allowing for reading of Hudi's @@ -195,7 +199,8 @@ protected List listStatusForIncrementalMode(JobConf job, return HoodieInputFormatUtils.filterIncrementalFileStatus(jobContext, tableMetaClient, timeline.get(), fileStatuses, commitsToCheck.get()); } - protected FileStatus createFileStatusUnchecked(FileSlice fileSlice, HiveHoodieTableFileIndex fileIndex, HoodieTableMetaClient metaClient) { + protected FileStatus createFileStatusUnchecked(FileSlice fileSlice, Option latestCompletedInstantOpt, + String tableBasePath, HoodieTableMetaClient metaClient) { Option baseFileOpt = fileSlice.getBaseFile(); if (baseFileOpt.isPresent()) { @@ -241,31 +246,86 @@ private List listStatusForSnapshotMode(JobConf job, boolean shouldIncludePendingCommits = HoodieHiveUtils.shouldIncludePendingCommits(job, tableMetaClient.getTableConfig().getTableName()); - HiveHoodieTableFileIndex fileIndex = - new HiveHoodieTableFileIndex( - engineContext, - tableMetaClient, - props, - HoodieTableQueryType.SNAPSHOT, - partitionPaths, - queryCommitInstant, - shouldIncludePendingCommits); - - Map> partitionedFileSlices = fileIndex.listFileSlices(); - - targetFiles.addAll( - partitionedFileSlices.values() - .stream() - .flatMap(Collection::stream) - .filter(fileSlice -> checkIfValidFileSlice(fileSlice)) - .map(fileSlice -> createFileStatusUnchecked(fileSlice, fileIndex, tableMetaClient)) - .collect(Collectors.toList()) - ); + if (HoodieTableMetadataUtil.isFilesPartitionAvailable(tableMetaClient) || conf.getBoolean(ENABLE.key(), ENABLE.defaultValue())) { + HiveHoodieTableFileIndex fileIndex = + new HiveHoodieTableFileIndex( + engineContext, + tableMetaClient, + props, + HoodieTableQueryType.SNAPSHOT, + partitionPaths, + queryCommitInstant, + shouldIncludePendingCommits); + + Map> partitionedFileSlices = fileIndex.listFileSlices(); + + targetFiles.addAll( + partitionedFileSlices.values() + .stream() + .flatMap(Collection::stream) + .filter(fileSlice -> checkIfValidFileSlice(fileSlice)) + .map(fileSlice -> createFileStatusUnchecked(fileSlice, fileIndex.getLatestCompletedInstant(), + fileIndex.getBasePath().toString(), tableMetaClient)) + .collect(Collectors.toList()) + ); + } else { + // If hoodie.metadata.enabled is set to false and the table doesn't have the metadata, + // read the table using fs view cache instead of file index. + // This is because there's no file index in non-metadata table. + String basePath = tableMetaClient.getBasePathV2().toString(); + Map fsViewCache = new HashMap<>(); + HoodieTimeline timeline = getActiveTimeline(tableMetaClient, shouldIncludePendingCommits); + Option queryInstant = queryCommitInstant.or(() -> timeline.lastInstant().map(HoodieInstant::getTimestamp)); + validateInstant(timeline, queryInstant); + + try { + HoodieTableFileSystemView fsView = fsViewCache.computeIfAbsent(tableMetaClient, hoodieTableMetaClient -> + FileSystemViewManager.createInMemoryFileSystemViewWithTimeline(engineContext, hoodieTableMetaClient, + HoodieInputFormatUtils.buildMetadataConfig(job), timeline)); + + List filteredFileSlices = new ArrayList<>(); + + for (Path p : entry.getValue()) { + String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(basePath), p); + + List fileSlices = queryInstant.map( + instant -> fsView.getLatestMergedFileSlicesBeforeOrOn(relativePartitionPath, instant)) + .orElse(fsView.getLatestFileSlices(relativePartitionPath)) + .collect(Collectors.toList()); + + filteredFileSlices.addAll(fileSlices); + } + + targetFiles.addAll( + filteredFileSlices.stream() + .filter(fileSlice -> checkIfValidFileSlice(fileSlice)) + .map(fileSlice -> createFileStatusUnchecked(fileSlice, timeline.filterCompletedInstants().lastInstant(), + basePath, tableMetaClient)) + .collect(Collectors.toList())); + } finally { + fsViewCache.forEach(((metaClient, fsView) -> fsView.close())); + } + } } return targetFiles; } + private static HoodieTimeline getActiveTimeline(HoodieTableMetaClient metaClient, boolean shouldIncludePendingCommits) { + HoodieTimeline timeline = metaClient.getCommitsAndCompactionTimeline(); + if (shouldIncludePendingCommits) { + return timeline; + } else { + return timeline.filterCompletedAndCompactionInstants(); + } + } + + private static void validateInstant(HoodieTimeline activeTimeline, Option queryInstant) { + if (queryInstant.isPresent() && !activeTimeline.containsInstant(queryInstant.get())) { + throw new HoodieIOException(String.format("Query instant (%s) not found in the timeline", queryInstant.get())); + } + } + protected boolean checkIfValidFileSlice(FileSlice fileSlice) { Option baseFileOpt = fileSlice.getBaseFile(); Option latestLogFileOpt = fileSlice.getLatestLogFile(); @@ -280,11 +340,6 @@ protected boolean checkIfValidFileSlice(FileSlice fileSlice) { } } - private void validate(List targetFiles, List legacyFileStatuses) { - List diff = CollectionUtils.diff(targetFiles, legacyFileStatuses); - checkState(diff.isEmpty(), "Should be empty"); - } - @Nonnull protected static FileStatus getFileStatusUnchecked(HoodieBaseFile baseFile) { try { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index 3719718e95aa..0cfe0d0a1940 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -36,7 +36,6 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.BootstrapBaseFileSplit; import org.apache.hudi.hadoop.FileStatusWithBootstrapBaseFile; -import org.apache.hudi.hadoop.HiveHoodieTableFileIndex; import org.apache.hudi.hadoop.HoodieCopyOnWriteTableInputFormat; import org.apache.hudi.hadoop.LocatedFileStatusWithBootstrapBaseFile; import org.apache.hudi.hadoop.RealtimeFileStatus; @@ -92,14 +91,12 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { } @Override - protected FileStatus createFileStatusUnchecked(FileSlice fileSlice, HiveHoodieTableFileIndex fileIndex, HoodieTableMetaClient metaClient) { + protected FileStatus createFileStatusUnchecked(FileSlice fileSlice, Option latestCompletedInstantOpt, + String tableBasePath, HoodieTableMetaClient metaClient) { Option baseFileOpt = fileSlice.getBaseFile(); Option latestLogFileOpt = fileSlice.getLatestLogFile(); Stream logFiles = fileSlice.getLogFiles(); - Option latestCompletedInstantOpt = fileIndex.getLatestCompletedInstant(); - String tableBasePath = fileIndex.getBasePath().toString(); - // Check if we're reading a MOR table if (baseFileOpt.isPresent()) { return createRealtimeFileStatusUnchecked(baseFileOpt.get(), logFiles, tableBasePath, latestCompletedInstantOpt, getHoodieVirtualKeyInfo(metaClient)); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java index d4665282be12..8809213582ac 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java @@ -87,7 +87,7 @@ public Pair>> fetchSource() t .setBasePath(service.getCfg().targetBasePath) .build(); String instantTime = InProcessTimeGenerator.createNewInstantTime(); - InputBatch inputBatch = service.readFromSource(instantTime, metaClient).getLeft(); + InputBatch inputBatch = service.readFromSource(instantTime, metaClient); return Pair.of(inputBatch.getSchemaProvider(), Pair.of(inputBatch.getCheckpointForNextBatch(), (JavaRDD) inputBatch.getBatch().get())); } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/HoodieStreamerDatasetBulkInsertCommitActionExecutor.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/HoodieStreamerDatasetBulkInsertCommitActionExecutor.java index 5593a95ca393..2a5113538e4d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/HoodieStreamerDatasetBulkInsertCommitActionExecutor.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/HoodieStreamerDatasetBulkInsertCommitActionExecutor.java @@ -26,9 +26,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -44,12 +42,8 @@ public HoodieStreamerDatasetBulkInsertCommitActionExecutor(HoodieWriteConfig con @Override protected void preExecute() { - // no op - } - - @Override - protected void afterExecute(HoodieWriteMetadata> result) { - // no op + table.validateInsertSchema(); + writeClient.preWrite(instantTime, getWriteOperationType(), table.getMetaClient()); } @Override diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala index 65bb8881455a..aa1209e67fa1 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala @@ -77,14 +77,6 @@ case class BaseFileOnlyRelation(override val sqlContext: SQLContext, override def updatePrunedDataSchema(prunedSchema: StructType): Relation = this.copy(prunedDataSchema = Some(prunedSchema)) - override def imbueConfigs(sqlContext: SQLContext): Unit = { - super.imbueConfigs(sqlContext) - // TODO Issue with setting this to true in spark 332 - if (HoodieSparkUtils.gteqSpark3_4 || !HoodieSparkUtils.gteqSpark3_3_2) { - sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.enableVectorizedReader", "true") - } - } - protected override def composeRDD(fileSplits: Seq[HoodieBaseFileSplit], tableSchema: HoodieTableSchema, requiredSchema: HoodieTableSchema, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala index dd76aee2f187..9cdb15092b08 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala @@ -309,7 +309,7 @@ class ColumnStatsIndexSupport(spark: SparkSession, colStatsDF.select(targetColumnStatsIndexColumns.map(col): _*) } - private def loadColumnStatsIndexRecords(targetColumns: Seq[String], shouldReadInMemory: Boolean): HoodieData[HoodieMetadataColumnStats] = { + def loadColumnStatsIndexRecords(targetColumns: Seq[String], shouldReadInMemory: Boolean): HoodieData[HoodieMetadataColumnStats] = { // Read Metadata Table's Column Stats Index records into [[HoodieData]] container by // - Fetching the records from CSI by key-prefixes (encoded column names) // - Extracting [[HoodieMetadataColumnStats]] records diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 80f7cebd3a2f..ac60606a4bc4 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -209,7 +209,7 @@ object DataSourceReadOptions { " by carefully analyzing provided partition-column predicates and deducing corresponding partition-path prefix from " + " them (if possible).") - val INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES: ConfigProperty[String] = ConfigProperty + val INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN: ConfigProperty[String] = ConfigProperty .key("hoodie.datasource.read.incr.fallback.fulltablescan.enable") .defaultValue("false") .markAdvanced() diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapMORRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapMORRelation.scala index 94f8cbfc433f..d7d24577ba53 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapMORRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapMORRelation.scala @@ -67,12 +67,6 @@ case class HoodieBootstrapMORRelation(override val sqlContext: SQLContext, override lazy val mandatoryFields: Seq[String] = mandatoryFieldsForMerging - override def imbueConfigs(sqlContext: SQLContext): Unit = { - super.imbueConfigs(sqlContext) - sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.enableVectorizedReader", "true") - } - - protected override def getFileSlices(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[FileSlice] = { if (globPaths.isEmpty) { fileIndex.listFileSlices(HoodieFileIndex. diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index 580a03dae709..9961e46b5014 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -85,7 +85,9 @@ case class HoodieFileIndex(spark: SparkSession, configProperties = getConfigProperties(spark, options), queryPaths = HoodieFileIndex.getQueryPaths(options), specifiedQueryInstant = options.get(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key).map(HoodieSqlCommonUtils.formatQueryInstant), - fileStatusCache = fileStatusCache + fileStatusCache = fileStatusCache, + beginInstantTime = options.get(DataSourceReadOptions.BEGIN_INSTANTTIME.key), + endInstantTime = options.get(DataSourceReadOptions.END_INSTANTTIME.key) ) with FileIndex { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodiePartitionCDCFileGroupMapping.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodiePartitionCDCFileGroupMapping.scala index bd052c086ffb..6ff9bd036e8e 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodiePartitionCDCFileGroupMapping.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodiePartitionCDCFileGroupMapping.scala @@ -22,11 +22,6 @@ package org.apache.hudi import org.apache.hudi.common.model.HoodieFileGroupId import org.apache.hudi.common.table.cdc.HoodieCDCFileSplit import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} -import org.apache.spark.sql.types.{DataType, Decimal} -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} - -import java.util class HoodiePartitionCDCFileGroupMapping(partitionValues: InternalRow, fileGroups: Map[HoodieFileGroupId, List[HoodieCDCFileSplit]] diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index cbde026adeb4..7f28e190e950 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -27,12 +27,12 @@ import org.apache.hudi.DataSourceOptionsHelper.fetchMissingWriteConfigsFromTable import org.apache.hudi.DataSourceUtils.tryOverrideParquetWriteLegacyFormatProperty import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.HoodieConversionUtils.{toProperties, toScalaOption} -import org.apache.hudi.HoodieSparkSqlWriter.{CANONICALIZE_SCHEMA, SQL_MERGE_INTO_WRITES, StreamingWriteParams} +import org.apache.hudi.HoodieSparkSqlWriter.StreamingWriteParams import org.apache.hudi.HoodieWriterUtils._ -import org.apache.hudi.avro.AvroSchemaUtils.{isCompatibleProjectionOf, isSchemaCompatible, isValidEvolutionOf, resolveNullableSchema} +import org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema import org.apache.hudi.avro.HoodieAvroUtils -import org.apache.hudi.avro.HoodieAvroUtils.removeMetadataFields import org.apache.hudi.client.common.HoodieSparkEngineContext +import org.apache.hudi.client.embedded.EmbeddedTimelineService import org.apache.hudi.client.{HoodieWriteResult, SparkRDDWriteClient} import org.apache.hudi.commit.{DatasetBulkInsertCommitActionExecutor, DatasetBulkInsertOverwriteCommitActionExecutor, DatasetBulkInsertOverwriteTableCommitActionExecutor} import org.apache.hudi.common.config._ @@ -49,12 +49,11 @@ import org.apache.hudi.common.util.{CommitUtils, StringUtils, Option => HOption} import org.apache.hudi.config.HoodieBootstrapConfig.{BASE_PATH, INDEX_CLASS_NAME} import org.apache.hudi.config.HoodieWriteConfig.SPARK_SQL_MERGE_INTO_PREPPED_KEY import org.apache.hudi.config.{HoodieCompactionConfig, HoodieInternalConfig, HoodieWriteConfig} -import org.apache.hudi.exception.{HoodieException, HoodieWriteConflictException, SchemaCompatibilityException} +import org.apache.hudi.exception.{HoodieException, HoodieWriteConflictException} import org.apache.hudi.hive.{HiveSyncConfigHolder, HiveSyncTool} import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter -import org.apache.hudi.internal.schema.utils.AvroSchemaEvolutionUtils.reconcileSchemaRequirements -import org.apache.hudi.internal.schema.utils.{AvroSchemaEvolutionUtils, SerDeHelper} +import org.apache.hudi.internal.schema.utils.SerDeHelper import org.apache.hudi.keygen.constant.KeyGeneratorType import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory.getKeyGeneratorClassName @@ -270,14 +269,14 @@ class HoodieSparkSqlWriterInternal { } } + val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(hoodieConfig.getProps)) if (mode == SaveMode.Ignore && tableExists) { log.warn(s"hoodie table at $basePath already exists. Ignoring & not performing actual writes.") (false, common.util.Option.empty(), common.util.Option.empty(), common.util.Option.empty(), hoodieWriteClient.orNull, tableConfig) } else { // Handle various save modes handleSaveModes(sqlContext.sparkSession, mode, basePath, tableConfig, tblName, operation, fs) - val partitionColumns = SparkKeyGenUtils.getPartitionColumns(getKeyGeneratorClassName(new TypedProperties(hoodieConfig.getProps)), - toProperties(parameters)) + val partitionColumns = SparkKeyGenUtils.getPartitionColumns(keyGenerator, toProperties(parameters)) val timelineTimeZone = HoodieTimelineTimeZone.valueOf(hoodieConfig.getStringOrDefault(HoodieTableConfig.TIMELINE_TIMEZONE)) val tableMetaClient = if (tableExists) { HoodieInstantTimeGenerator.setCommitTimeZone(timelineTimeZone) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index e2c5ad88d7f2..133f641d280b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -203,10 +203,11 @@ object HoodieWriterUtils { } val datasourcePartitionFields = params.getOrElse(PARTITIONPATH_FIELD.key(), null) + val currentPartitionFields = if (datasourcePartitionFields == null) null else SparkKeyGenUtils.getPartitionColumns(TypedProperties.fromMap(params)) val tableConfigPartitionFields = tableConfig.getString(HoodieTableConfig.PARTITION_FIELDS) if (null != datasourcePartitionFields && null != tableConfigPartitionFields - && datasourcePartitionFields != tableConfigPartitionFields) { - diffConfigs.append(s"PartitionPath:\t$datasourcePartitionFields\t$tableConfigPartitionFields\n") + && currentPartitionFields != tableConfigPartitionFields) { + diffConfigs.append(s"PartitionPath:\t$currentPartitionFields\t$tableConfigPartitionFields\n") } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala index b02fecff3276..227b585c9ef2 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala @@ -212,8 +212,8 @@ class IncrementalRelation(val sqlContext: SQLContext, // 1. the start commit is archived // 2. the end commit is archived // 3. there are files in metadata be deleted - val fallbackToFullTableScan = optParams.getOrElse(DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES.key, - DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES.defaultValue).toBoolean + val fallbackToFullTableScan = optParams.getOrElse(DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN.key, + DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN.defaultValue).toBoolean val sOpts = optParams.filter(p => !p._1.equalsIgnoreCase("path")) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala index e37a1737d27a..7b25fd9a8c73 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala @@ -33,7 +33,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.execution.datasources.PartitionDirectory import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType @@ -196,8 +195,8 @@ trait HoodieIncrementalRelationTrait extends HoodieBaseRelation { // 2. the end commit is archived // 3. there are files in metadata be deleted protected lazy val fullTableScan: Boolean = { - val fallbackToFullTableScan = optParams.getOrElse(DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES.key, - DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES.defaultValue).toBoolean + val fallbackToFullTableScan = optParams.getOrElse(DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN.key, + DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN.defaultValue).toBoolean fallbackToFullTableScan && (startInstantArchived || endInstantArchived || affectedFilesInCommits.exists(fileStatus => !metaClient.getFs.exists(fileStatus.getPath))) } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index 828c8df41deb..f77a6cbe546b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -66,7 +66,9 @@ class SparkHoodieTableFileIndex(spark: SparkSession, configProperties: TypedProperties, queryPaths: Seq[Path], specifiedQueryInstant: Option[String] = None, - @transient fileStatusCache: FileStatusCache = NoopCache) + @transient fileStatusCache: FileStatusCache = NoopCache, + beginInstantTime: Option[String] = None, + endInstantTime: Option[String] = None) extends BaseHoodieTableFileIndex( new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext)), metaClient, @@ -77,7 +79,9 @@ class SparkHoodieTableFileIndex(spark: SparkSession, false, false, SparkHoodieTableFileIndex.adapt(fileStatusCache), - shouldListLazily(configProperties) + shouldListLazily(configProperties), + toJavaOption(beginInstantTime), + toJavaOption(endInstantTime) ) with SparkAdapterSupport with Logging { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DeleteMarkerProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DeleteMarkerProcedure.scala index 87d58fa6ed09..8d73a753cf4c 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DeleteMarkerProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DeleteMarkerProcedure.scala @@ -48,6 +48,8 @@ class DeleteMarkerProcedure extends BaseProcedure with ProcedureBuilder with Log val tableName = getArgValueOrDefault(args, PARAMETERS(0)) val instantTime = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String] val basePath = getBasePath(tableName) + val instantTimes = instantTime.split(",") + var currentInstant = "" var client: SparkRDDWriteClient[_] = null val result = Try { @@ -56,14 +58,17 @@ class DeleteMarkerProcedure extends BaseProcedure with ProcedureBuilder with Log val config = client.getConfig val context = client.getEngineContext val table = HoodieSparkTable.create(config, context) - WriteMarkersFactory.get(config.getMarkersType, table, instantTime) - .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism) + for (it <- instantTimes) { + currentInstant = it + WriteMarkersFactory.get(config.getMarkersType, table, it) + .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism) + } } match { case Success(_) => logInfo(s"Marker $instantTime deleted.") true case Failure(e) => - logWarning(s"Failed: Could not clean marker instantTime: $instantTime.", e) + logWarning(s"Failed: Could not clean marker instantTime: $currentInstant.", e) false } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DeleteSavepointProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DeleteSavepointProcedure.scala index 0e92abc49776..d568566e5546 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DeleteSavepointProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DeleteSavepointProcedure.scala @@ -58,25 +58,28 @@ class DeleteSavepointProcedure extends BaseProcedure with ProcedureBuilder with if (StringUtils.isNullOrEmpty(instantTime)) { instantTime = completedInstants.lastInstant.get.getTimestamp } - val savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, instantTime) - - if (!completedInstants.containsInstant(savePoint)) { - throw new HoodieException("Commit " + instantTime + " not found in Commits " + completedInstants) - } - + val instantTimes = instantTime.split(",") val client = HoodieCLIUtils.createHoodieWriteClient(sparkSession, basePath, Map.empty, tableName.asInstanceOf[Option[String]]) - var result = false - - try { - client.deleteSavepoint(instantTime) - logInfo(s"The commit $instantTime has been deleted savepoint.") - result = true - } catch { - case _: HoodieSavepointException => - logWarning(s"Failed: Could not delete savepoint $instantTime.") - } finally { - client.close() + var result = true + var currentInstant = "" + for (it <- instantTimes) { + val savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, it) + currentInstant = it + if (!completedInstants.containsInstant(savePoint)) { + throw new HoodieException("Commit " + it + " not found in Commits " + completedInstants) + } + + try { + client.deleteSavepoint(it) + logInfo(s"The commit $instantTime has been deleted savepoint.") + } catch { + case _: HoodieSavepointException => + logWarning(s"Failed: Could not delete savepoint $currentInstant.") + result = false + } finally { + client.close() + } } Seq(Row(result)) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala index ad63ddbb29ee..1a960ecb8fd6 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala @@ -66,6 +66,7 @@ object HoodieProcedures { ,(ShowBootstrapPartitionsProcedure.NAME, ShowBootstrapPartitionsProcedure.builder) ,(UpgradeTableProcedure.NAME, UpgradeTableProcedure.builder) ,(DowngradeTableProcedure.NAME, DowngradeTableProcedure.builder) + ,(ShowMetadataTableColumnStatsProcedure.NAME, ShowMetadataTableColumnStatsProcedure.builder) ,(ShowMetadataTableFilesProcedure.NAME, ShowMetadataTableFilesProcedure.builder) ,(ShowMetadataTablePartitionsProcedure.NAME, ShowMetadataTablePartitionsProcedure.builder) ,(CreateMetadataTableProcedure.NAME, CreateMetadataTableProcedure.builder) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableColumnStatsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableColumnStatsProcedure.scala new file mode 100644 index 000000000000..e90dc14ec9ee --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableColumnStatsProcedure.scala @@ -0,0 +1,165 @@ +/* + * 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.spark.sql.hudi.command.procedures + +import org.apache.avro.generic.IndexedRecord +import org.apache.hadoop.fs.FileStatus +import org.apache.hudi.avro.model._ +import org.apache.hudi.client.common.HoodieSparkEngineContext +import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.common.data.HoodieData +import org.apache.hudi.common.model.FileSlice +import org.apache.hudi.common.table.timeline.{HoodieDefaultTimeline, HoodieInstant} +import org.apache.hudi.common.table.view.HoodieTableFileSystemView +import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} +import org.apache.hudi.common.util.{Option => HOption} +import org.apache.hudi.exception.HoodieException +import org.apache.hudi.metadata.HoodieTableMetadata +import org.apache.hudi.{AvroConversionUtils, ColumnStatsIndexSupport} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} + +import java.util +import java.util.function.{Function, Supplier} +import scala.collection.{JavaConversions, mutable} +import scala.jdk.CollectionConverters.{asScalaBufferConverter, asScalaIteratorConverter} + +class ShowMetadataTableColumnStatsProcedure extends BaseProcedure with ProcedureBuilder with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType), + ProcedureParameter.optional(1, "partition", DataTypes.StringType), + ProcedureParameter.optional(2, "targetColumns", DataTypes.StringType) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("file_name", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("column_name", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("min_value", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("max_value", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("null_number", DataTypes.LongType, nullable = true, Metadata.empty) + )) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val table = getArgValueOrDefault(args, PARAMETERS(0)) + val partitions = getArgValueOrDefault(args, PARAMETERS(1)).getOrElse("").toString + val partitionsSeq = partitions.split(",").filter(_.nonEmpty).toSeq + + val targetColumns = getArgValueOrDefault(args, PARAMETERS(2)).getOrElse("").toString + val targetColumnsSeq = targetColumns.split(",").toSeq + val basePath = getBasePath(table) + val metadataConfig = HoodieMetadataConfig.newBuilder + .enable(true) + .build + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build + val schemaUtil = new TableSchemaResolver(metaClient) + val schema = AvroConversionUtils.convertAvroSchemaToStructType(schemaUtil.getTableAvroSchema) + val columnStatsIndex = new ColumnStatsIndexSupport(spark, schema, metadataConfig, metaClient) + val colStatsRecords: HoodieData[HoodieMetadataColumnStats] = columnStatsIndex.loadColumnStatsIndexRecords(targetColumnsSeq, shouldReadInMemory = false) + val fsView = buildFileSystemView(table) + val allFileSlices: Set[FileSlice] = { + if (partitionsSeq.isEmpty) { + val engineCtx = new HoodieSparkEngineContext(jsc) + val metaTable = HoodieTableMetadata.create(engineCtx, metadataConfig, basePath) + metaTable.getAllPartitionPaths + .asScala + .flatMap(path => fsView.getLatestFileSlices(path).iterator().asScala) + .toSet + } else { + partitionsSeq + .flatMap(partition => fsView.getLatestFileSlices(partition).iterator().asScala) + .toSet + } + } + + val allFileNames: Set[String] = allFileSlices.map(_.getBaseFile.get().getFileName) + + val rows = mutable.ListBuffer[Row]() + colStatsRecords.collectAsList().asScala + .filter(c => allFileNames.contains(c.getFileName)) + .foreach { c => + rows += Row(c.getFileName, c.getColumnName, getColumnStatsValue(c.getMinValue), + getColumnStatsValue(c.getMaxValue), c.getNullCount.longValue()) + } + + rows.toList.sortBy(r => r.getString(1)) + } + + private def getColumnStatsValue(stats_value: Any): String = { + stats_value match { + case null => "null" + case _: IntWrapper | + _: BooleanWrapper | + _: DateWrapper | + _: DoubleWrapper | + _: FloatWrapper | + _: LongWrapper | + _: StringWrapper | + _: TimeMicrosWrapper | + _: TimestampMicrosWrapper => + String.valueOf(stats_value.asInstanceOf[IndexedRecord].get(0)) + case _: BytesWrapper => + val bytes_value = stats_value.asInstanceOf[BytesWrapper].getValue + util.Arrays.toString(bytes_value.array()) + case _: DecimalWrapper => + val decimal_value = stats_value.asInstanceOf[DecimalWrapper].getValue + util.Arrays.toString(decimal_value.array()) + case _ => + throw new HoodieException(s"Unsupported type: ${stats_value.getClass.getSimpleName}") + } + } + + def buildFileSystemView(table: Option[Any]): HoodieTableFileSystemView = { + val basePath = getBasePath(table) + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build + + val timeline = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants() + + val maxInstant = metaClient.createNewInstantTime() + val instants = timeline.getInstants.iterator().asScala.filter(_.getTimestamp < maxInstant) + + val details = new Function[HoodieInstant, org.apache.hudi.common.util.Option[Array[Byte]]] + with java.io.Serializable { + override def apply(instant: HoodieInstant): HOption[Array[Byte]] = { + metaClient.getActiveTimeline.getInstantDetails(instant) + } + } + + val filteredTimeline = new HoodieDefaultTimeline( + new java.util.ArrayList[HoodieInstant](JavaConversions.asJavaCollection(instants.toList)).stream(), details) + + new HoodieTableFileSystemView(metaClient, filteredTimeline, new Array[FileStatus](0)) + } + + override def build: Procedure = new ShowMetadataTableColumnStatsProcedure() +} + +object ShowMetadataTableColumnStatsProcedure { + val NAME = "show_metadata_table_column_stats" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new ShowMetadataTableColumnStatsProcedure() + } +} + diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/UpgradeOrDowngradeProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/UpgradeOrDowngradeProcedure.scala index 0ae413040bc1..b94f09665750 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/UpgradeOrDowngradeProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/UpgradeOrDowngradeProcedure.scala @@ -20,16 +20,18 @@ package org.apache.spark.sql.hudi.command.procedures import org.apache.hudi.client.common.HoodieSparkEngineContext import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion -import org.apache.hudi.common.table.{HoodieTableMetaClient, HoodieTableVersion} +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, HoodieTableVersion} import org.apache.hudi.common.util.Option import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig, HoodieCleanConfig} import org.apache.hudi.index.HoodieIndex import org.apache.hudi.table.upgrade.{SparkUpgradeDowngradeHelper, UpgradeDowngrade} +import org.apache.hudi.HoodieCLIUtils import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} import java.util.function.Supplier +import scala.collection.JavaConverters._ import scala.util.{Failure, Success, Try} class UpgradeOrDowngradeProcedure extends BaseProcedure with ProcedureBuilder with Logging { @@ -51,9 +53,8 @@ class UpgradeOrDowngradeProcedure extends BaseProcedure with ProcedureBuilder wi val tableName = getArgValueOrDefault(args, PARAMETERS(0)) val toVersion = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String] - val basePath = getBasePath(tableName) - - val config = getWriteConfigWithTrue(basePath) + val config = getWriteConfigWithTrue(tableName) + val basePath = config.getBasePath val metaClient = HoodieTableMetaClient.builder .setConf(jsc.hadoopConfiguration) .setBasePath(config.getBasePath) @@ -78,12 +79,16 @@ class UpgradeOrDowngradeProcedure extends BaseProcedure with ProcedureBuilder wi Seq(Row(result)) } - private def getWriteConfigWithTrue(basePath: String) = { + private def getWriteConfigWithTrue(tableOpt: scala.Option[Any]) = { + val basePath = getBasePath(tableOpt) + val (tableName, database) = HoodieCLIUtils.getTableIdentifier(tableOpt.get.asInstanceOf[String]) HoodieWriteConfig.newBuilder + .forTable(tableName) .withPath(basePath) .withRollbackUsingMarkers(true) .withCleanConfig(HoodieCleanConfig.newBuilder.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER).build) .withIndexConfig(HoodieIndexConfig.newBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build) + .withProps(Map(HoodieTableConfig.DATABASE_NAME.key -> database.getOrElse(sparkSession.sessionState.catalog.getCurrentDatabase)).asJava) .build } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java index ee59100caa69..9847918adc1a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java @@ -40,6 +40,7 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; @@ -86,6 +87,8 @@ public void setUp() throws IOException { HoodieTableConfig.BASE_FILE_FORMAT.key(), HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().toString()); properties.setProperty(HoodieTableConfig.PRECOMBINE_FIELD.key(), "record_key"); + properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(),"partition_path"); + properties.setProperty(HoodieTableConfig.PARTITION_FIELDS.key(),"partition_path"); metaClient = getHoodieMetaClient(hadoopConf(), basePath(), HoodieTableType.MERGE_ON_READ, properties); } @@ -170,6 +173,7 @@ public HoodieWriteConfig getWriteConfig(Schema avroSchema) { extraProperties.setProperty( WRITE_RECORD_POSITIONS.key(), "true"); + extraProperties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(),"partition_path"); return getConfigBuilder(true) .withPath(basePath()) @@ -246,7 +250,7 @@ public void checkDataEquality(int numRecords) { .read() .options(properties) .format("org.apache.hudi") - .load(basePath() + "/" + getPartitionPath()); + .load(basePath()); List result = rows.collectAsList(); assertEquals(numRecords, result.size()); } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java index 8df34768909e..254c1768a62b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java @@ -92,6 +92,7 @@ import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -201,9 +202,9 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec String keyGeneratorClass = partitioned ? SimpleKeyGenerator.class.getCanonicalName() : NonpartitionedKeyGenerator.class.getCanonicalName(); if (deltaCommit) { - metaClient = HoodieTestUtils.init(basePath, HoodieTableType.MERGE_ON_READ, bootstrapBasePath, true, keyGeneratorClass); + metaClient = HoodieTestUtils.init(basePath, HoodieTableType.MERGE_ON_READ, bootstrapBasePath, true, keyGeneratorClass, "partition_path"); } else { - metaClient = HoodieTestUtils.init(basePath, HoodieTableType.COPY_ON_WRITE, bootstrapBasePath, true, keyGeneratorClass); + metaClient = HoodieTestUtils.init(basePath, HoodieTableType.COPY_ON_WRITE, bootstrapBasePath, true, keyGeneratorClass, "partition_path"); } int totalRecords = 100; @@ -240,7 +241,7 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS); break; } - List partitions = Arrays.asList("2020/04/01", "2020/04/02", "2020/04/03"); + List partitions = partitioned ? Arrays.asList("2020/04/01", "2020/04/02", "2020/04/03") : Collections.EMPTY_LIST; long timestamp = Instant.now().toEpochMilli(); Schema schema = generateNewDataSetAndReturnSchema(timestamp, totalRecords, partitions, bootstrapBasePath); HoodieWriteConfig config = getConfigBuilder(schema.toString()) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala index 70d74f0a0cea..5cd6ac3954ee 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala @@ -30,146 +30,150 @@ import org.scalatest.{FunSuite, Matchers} class TestAvroConversionUtils extends FunSuite with Matchers { - test("test convertStructTypeToAvroSchema") { - val mapType = DataTypes.createMapType(StringType, new StructType().add("mapKey", "string", false).add("mapVal", "integer", true)) - val arrayType = ArrayType(new StructType().add("arrayKey", "string", false).add("arrayVal", "integer", true)) - val innerStruct = new StructType().add("innerKey","string",false).add("value", "long", true) - - val struct = new StructType().add("key", "string", false).add("version", "string", true) - .add("data1",innerStruct,false).add("data2",innerStruct,true) - .add("nullableMap", mapType, true).add("map",mapType,false) - .add("nullableArray", arrayType, true).add("array",arrayType,false) - - val avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(struct, "SchemaName", "SchemaNS") - - val expectedSchemaStr = s""" - { - "type" : "record", - "name" : "SchemaName", - "namespace" : "SchemaNS", - "fields" : [ { - "name" : "key", - "type" : "string" - }, { - "name" : "version", - "type" : [ "null", "string" ], - "default" : null - }, { + val complexSchemaStr = + s""" + { + "type" : "record", + "name" : "SchemaName", + "namespace" : "SchemaNS", + "fields" : [ { + "name" : "key", + "type" : "string" + }, { + "name" : "version", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "data1", + "type" : { + "type" : "record", "name" : "data1", - "type" : { + "namespace" : "SchemaNS.SchemaName", + "fields" : [ { + "name" : "innerKey", + "type" : "string" + }, { + "name" : "value", + "type" : [ "null", "long" ], + "default" : null + } ] + } + }, { + "name" : "data2", + "type" : [ "null", { + "type" : "record", + "name" : "data2", + "namespace" : "SchemaNS.SchemaName", + "fields" : [ { + "name" : "innerKey", + "type" : "string" + }, { + "name" : "value", + "type" : [ "null", "long" ], + "default" : null + } ] + } ], + "default" : null + }, { + "name" : "nullableMap", + "type" : [ "null", { + "type" : "map", + "values" : [ + "null", + { "type" : "record", - "name" : "data1", + "name" : "nullableMap", "namespace" : "SchemaNS.SchemaName", "fields" : [ { - "name" : "innerKey", + "name" : "mapKey", "type" : "string" }, { - "name" : "value", - "type" : [ "null", "long" ], + "name" : "mapVal", + "type" : [ "null", "int" ], "default" : null } ] - } - }, { - "name" : "data2", - "type" : [ "null", { + } ] + } ], + "default" : null + }, { + "name" : "map", + "type" : { + "type" : "map", + "values" : [ + "null", + { "type" : "record", - "name" : "data2", + "name" : "map", "namespace" : "SchemaNS.SchemaName", "fields" : [ { - "name" : "innerKey", + "name" : "mapKey", "type" : "string" }, { - "name" : "value", - "type" : [ "null", "long" ], + "name" : "mapVal", + "type" : [ "null", "int" ], "default" : null } ] - } ], - "default" : null - }, { - "name" : "nullableMap", - "type" : [ "null", { - "type" : "map", - "values" : [ - "null", - { - "type" : "record", - "name" : "nullableMap", - "namespace" : "SchemaNS.SchemaName", - "fields" : [ { - "name" : "mapKey", - "type" : "string" - }, { - "name" : "mapVal", - "type" : [ "null", "int" ], - "default" : null - } ] - } ] - } ], - "default" : null - }, { - "name" : "map", - "type" : { - "type" : "map", - "values" : [ - "null", - { - "type" : "record", - "name" : "map", - "namespace" : "SchemaNS.SchemaName", - "fields" : [ { - "name" : "mapKey", - "type" : "string" - }, { - "name" : "mapVal", - "type" : [ "null", "int" ], - "default" : null - } ] - } ] - } - }, { - "name" : "nullableArray", - "type" : [ "null", { - "type" : "array", - "items" : [ - "null", - { - "type" : "record", - "name" : "nullableArray", - "namespace" : "SchemaNS.SchemaName", - "fields" : [ { - "name" : "arrayKey", - "type" : "string" - }, { - "name" : "arrayVal", - "type" : [ "null", "int" ], - "default" : null - } ] + } ] + } + }, { + "name" : "nullableArray", + "type" : [ "null", { + "type" : "array", + "items" : [ + "null", + { + "type" : "record", + "name" : "nullableArray", + "namespace" : "SchemaNS.SchemaName", + "fields" : [ { + "name" : "arrayKey", + "type" : "string" + }, { + "name" : "arrayVal", + "type" : [ "null", "int" ], + "default" : null } ] - } ], - "default" : null - }, { - "name" : "array", - "type" : { - "type" : "array", - "items" : [ - "null", - { - "type" : "record", - "name" : "array", - "namespace" : "SchemaNS.SchemaName", - "fields" : [ { - "name" : "arrayKey", - "type" : "string" - }, { - "name" : "arrayVal", - "type" : [ "null", "int" ], - "default" : null - } ] + } ] + } ], + "default" : null + }, { + "name" : "array", + "type" : { + "type" : "array", + "items" : [ + "null", + { + "type" : "record", + "name" : "array", + "namespace" : "SchemaNS.SchemaName", + "fields" : [ { + "name" : "arrayKey", + "type" : "string" + }, { + "name" : "arrayVal", + "type" : [ "null", "int" ], + "default" : null } ] - } - } ] - } + } ] + } + } ] + } """ + + + test("test convertStructTypeToAvroSchema_orig") { + val mapType = DataTypes.createMapType(StringType, new StructType().add("mapKey", "string", false).add("mapVal", "integer", true)) + val arrayType = ArrayType(new StructType().add("arrayKey", "string", false).add("arrayVal", "integer", true)) + val innerStruct = new StructType().add("innerKey", "string", false).add("value", "long", true) + + val struct = new StructType().add("key", "string", false).add("version", "string", true) + .add("data1", innerStruct, false).add("data2", innerStruct, true) + .add("nullableMap", mapType, true).add("map", mapType, false) + .add("nullableArray", arrayType, true).add("array", arrayType, false) + + val avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(struct, "SchemaName", "SchemaNS") + + val expectedSchemaStr = complexSchemaStr val expectedAvroSchema = new Schema.Parser().parse(expectedSchemaStr) assert(avroSchema.equals(expectedAvroSchema)) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala index 69248fc2c237..803702addb48 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala @@ -29,7 +29,7 @@ import org.apache.hudi.common.config.TimestampKeyGeneratorConfig.{TIMESTAMP_INPU import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieStorageConfig} import org.apache.hudi.common.engine.EngineType import org.apache.hudi.common.fs.FSUtils -import org.apache.hudi.common.model.{HoodieRecord, HoodieTableType} +import org.apache.hudi.common.model.{HoodieBaseFile, HoodieRecord, HoodieTableType} import org.apache.hudi.common.table.view.HoodieTableFileSystemView import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime @@ -86,13 +86,9 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS @BeforeEach override def setUp() { setTableName("hoodie_test") + super.setUp() initPath() - initSparkContexts() spark = sqlContext.sparkSession - initTestDataGenerator() - initFileSystem() - initMetaClient() - queryOpts = queryOpts ++ Map("path" -> basePath) } @@ -244,6 +240,67 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS assertEquals(List("2021/03/08", "2021/03/09"), prunedPartitions) } + @ParameterizedTest + @CsvSource(value = Array("lazy,true", "lazy,false", + "eager,true", "eager,false")) + def testIndexRefreshesFileSlices(listingModeOverride: String, + useMetadataTable: Boolean): Unit = { + def getDistinctCommitTimeFromAllFilesInIndex(files: Seq[PartitionDirectory]): Seq[String] = { + files.flatMap(_.files).map(fileStatus => new HoodieBaseFile(fileStatus.getPath.toString)).map(_.getCommitTime).distinct + } + + val r = new Random(0xDEED) + // partition column values are [0, 5) + val tuples = for (i <- 1 to 1000) yield (r.nextString(1000), r.nextInt(5), r.nextString(1000)) + + val writeOpts = commonOpts ++ Map(HoodieMetadataConfig.ENABLE.key -> useMetadataTable.toString) + val _spark = spark + import _spark.implicits._ + val inputDF = tuples.toDF("_row_key", "partition", "timestamp") + inputDF + .write + .format("hudi") + .options(writeOpts) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Overwrite) + .save(basePath) + + val readOpts = queryOpts ++ Map( + HoodieMetadataConfig.ENABLE.key -> useMetadataTable.toString, + DataSourceReadOptions.FILE_INDEX_LISTING_MODE_OVERRIDE.key -> listingModeOverride + ) + + metaClient = HoodieTableMetaClient.reload(metaClient) + val fileIndexFirstWrite = HoodieFileIndex(spark, metaClient, None, readOpts) + + val listFilesAfterFirstWrite = fileIndexFirstWrite.listFiles(Nil, Nil) + val distinctListOfCommitTimesAfterFirstWrite = getDistinctCommitTimeFromAllFilesInIndex(listFilesAfterFirstWrite) + val firstWriteCommitTime = metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get().getTimestamp + assertEquals(1, distinctListOfCommitTimesAfterFirstWrite.size, "Should have only one commit") + assertEquals(firstWriteCommitTime, distinctListOfCommitTimesAfterFirstWrite.head, "All files should belong to the first existing commit") + + val nextBatch = for ( + i <- 0 to 4 + ) yield(r.nextString(1000), i, r.nextString(1000)) + + nextBatch.toDF("_row_key", "partition", "timestamp") + .write + .format("hudi") + .options(writeOpts) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + + fileIndexFirstWrite.refresh() + val fileSlicesAfterSecondWrite = fileIndexFirstWrite.listFiles(Nil, Nil) + val distinctListOfCommitTimesAfterSecondWrite = getDistinctCommitTimeFromAllFilesInIndex(fileSlicesAfterSecondWrite) + metaClient = HoodieTableMetaClient.reload(metaClient) + val lastCommitTime = metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get().getTimestamp + + assertEquals(1, distinctListOfCommitTimesAfterSecondWrite.size, "All basefiles affected so all have same commit time") + assertEquals(lastCommitTime, distinctListOfCommitTimesAfterSecondWrite.head, "All files should be of second commit after index refresh") + } + @ParameterizedTest @CsvSource(value = Array("lazy,true,true", "lazy,true,false", "lazy,false,true", "lazy,false,false", "eager,true,true", "eager,true,false", "eager,false,true", "eager,false,false")) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieParquetBloom.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieParquetBloom.scala index 2e5e30362bb9..a6f3a0e7368b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieParquetBloom.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieParquetBloom.scala @@ -19,53 +19,29 @@ package org.apache.hudi import org.apache.spark.sql._ import org.apache.spark.sql.hudi.HoodieSparkSessionExtension -import org.apache.spark.util.{AccumulatorV2} +import org.apache.spark.util.AccumulatorV2 import org.apache.spark.SparkContext - import org.apache.hudi.testutils.HoodieClientTestUtils.getSparkConfForTest import org.apache.hudi.DataSourceWriteOptions import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.common.model.{HoodieTableType, WriteOperationType} - - -import org.junit.jupiter.api.Assertions.{assertEquals} -import org.junit.jupiter.api.{BeforeEach} +import org.apache.hudi.testutils.HoodieSparkClientTestBase +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{EnumSource} - -class TestHoodieParquetBloomFilter { - - var spark: SparkSession = _ - var sqlContext: SQLContext = _ - var sc: SparkContext = _ +import org.junit.jupiter.params.provider.EnumSource - def initSparkContext(): Unit = { - val sparkConf = getSparkConfForTest(getClass.getSimpleName) - - spark = SparkSession.builder() - .withExtensions(new HoodieSparkSessionExtension) - .config(sparkConf) - .getOrCreate() - - sc = spark.sparkContext - sc.setLogLevel("ERROR") - sqlContext = spark.sqlContext - } - - @BeforeEach - def setUp() { - initSparkContext() - } +class TestHoodieParquetBloomFilter extends HoodieSparkClientTestBase with ScalaAssertionSupport { @ParameterizedTest @EnumSource(value = classOf[WriteOperationType], names = Array("BULK_INSERT", "INSERT", "UPSERT", "INSERT_OVERWRITE")) def testBloomFilter(operation: WriteOperationType): Unit = { // setup hadoop conf with bloom col enabled - spark.sparkContext.hadoopConfiguration.set("parquet.bloom.filter.enabled#bloom_col", "true") - spark.sparkContext.hadoopConfiguration.set("parquet.bloom.filter.expected.ndv#bloom_col", "2") + jsc.hadoopConfiguration.set("parquet.bloom.filter.enabled#bloom_col", "true") + jsc.hadoopConfiguration.set("parquet.bloom.filter.expected.ndv#bloom_col", "2") // ensure nothing but bloom can trigger read skip - spark.sql("set parquet.filter.columnindex.enabled=false") - spark.sql("set parquet.filter.stats.enabled=false") + sparkSession.sql("set parquet.filter.columnindex.enabled=false") + sparkSession.sql("set parquet.filter.stats.enabled=false") val basePath = java.nio.file.Files.createTempDirectory("hoodie_bloom_source_path").toAbsolutePath.toString val opts = Map( @@ -75,7 +51,7 @@ class TestHoodieParquetBloomFilter { DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key", DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition" ) - val inputDF = spark.sql( + val inputDF = sparkSession.sql( """select '0' as _row_key, '1' as bloom_col, '2' as partition, '3' as ts |union |select '1', '2', '3', '4' @@ -86,19 +62,19 @@ class TestHoodieParquetBloomFilter { .save(basePath) val accu = new NumRowGroupsAcc - spark.sparkContext.register(accu) + sparkSession.sparkContext.register(accu) // this one shall skip partition scanning thanks to bloom when spark >=3 - spark.read.format("hudi").load(basePath).filter("bloom_col = '3'").foreachPartition((it: Iterator[Row]) => it.foreach(_ => accu.add(0))) + sparkSession.read.format("hudi").load(basePath).filter("bloom_col = '3'").foreachPartition((it: Iterator[Row]) => it.foreach(_ => accu.add(0))) assertEquals(if (currentSparkSupportParquetBloom()) 0 else 1, accu.value) // this one will trigger one partition scan - spark.read.format("hudi").load(basePath).filter("bloom_col = '2'").foreachPartition((it: Iterator[Row]) => it.foreach(_ => accu.add(0))) + sparkSession.read.format("hudi").load(basePath).filter("bloom_col = '2'").foreachPartition((it: Iterator[Row]) => it.foreach(_ => accu.add(0))) assertEquals(1, accu.value) } def currentSparkSupportParquetBloom(): Boolean = { - Integer.valueOf(spark.version.charAt(0)) >= 3 + Integer.valueOf(sparkSession.version.charAt(0)) >= 3 } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala index 7f89817a7f8c..865ca147eb05 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala @@ -470,7 +470,7 @@ class TestHoodieSparkSqlWriter { val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType) // try write to Hudi - assertThrows[IllegalArgumentException] { + assertThrows[IOException] { HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, tableOpts - DataSourceWriteOptions.PARTITIONPATH_FIELD.key, df) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index 02c9b90e75ad..e2c719e87820 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -1001,8 +1001,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup writer.save(basePath) fail("should fail when invalid PartitionKeyType is provided!") } catch { - case e: Exception => - assertTrue(e.getCause.getMessage.contains("No enum constant org.apache.hudi.keygen.CustomAvroKeyGenerator.PartitionKeyType.DUMMY")) + case e: Exception => assertTrue(e.getCause.getMessage.contains("Unable to instantiate class org.apache.hudi.keygen.CustomKeyGenerator")) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala index e6aa2b39fb01..f30a143de322 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala @@ -252,7 +252,7 @@ class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase { // verify incremental query verifySQLQueries(numRecordsForFirstQuery, numRecordsForSecondQuery, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL, commonOpts, isTableDataSameAsAfterSecondInstant) - commonOpts = commonOpts + (DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES.key -> "true") + commonOpts = commonOpts + (DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN.key -> "true") // TODO: https://issues.apache.org/jira/browse/HUDI-6657 - Investigate why below assertions fail with full table scan enabled. //verifySQLQueries(numRecordsForFirstQuery, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL, commonOpts, isTableDataSameAsAfterSecondInstant) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadWithFullTableScan.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadWithFullTableScan.scala index 673468dfec8b..75a7026ad3ce 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadWithFullTableScan.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadWithFullTableScan.scala @@ -164,7 +164,7 @@ class TestIncrementalReadWithFullTableScan extends HoodieSparkClientTestBase { .option(DataSourceReadOptions.QUERY_TYPE.key(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key(), startTs) .option(DataSourceReadOptions.END_INSTANTTIME.key(), endTs) - .option(DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES.key(), fallBackFullTableScan) + .option(DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN.key(), fallBackFullTableScan) .load(basePath) assertEquals(perBatchSize * batchNum, hoodieIncViewDF.count()) } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallProcedure.scala index 5b90e2668197..30bec0f8a9ce 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallProcedure.scala @@ -209,6 +209,50 @@ class TestCallProcedure extends HoodieSparkProcedureTestBase { } } + test("Test Call delete_marker Procedure with batch mode") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = s"${tmp.getCanonicalPath}/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // Check required fields + checkExceptionContain(s"""call delete_marker(table => '$tableName')""")( + s"Argument: instant_time is required") + + var instantTime = "101" + FileCreateUtils.createMarkerFile(tablePath, "", instantTime, "f0", IOType.APPEND) + assertResult(1) { + FileCreateUtils.getTotalMarkerFileCount(tablePath, "", instantTime, IOType.APPEND) + } + instantTime = "102" + FileCreateUtils.createMarkerFile(tablePath, "", instantTime, "f0", IOType.APPEND) + assertResult(1) { + FileCreateUtils.getTotalMarkerFileCount(tablePath, "", instantTime, IOType.APPEND) + } + + instantTime = "101,102" + checkAnswer(s"""call delete_marker(table => '$tableName', instant_time => '$instantTime')""")(Seq(true)) + + assertResult(0) { + FileCreateUtils.getTotalMarkerFileCount(tablePath, "", instantTime, IOType.APPEND) + } + } + } + test("Test Call show_rollbacks Procedure") { withTempDir { tmp => val tableName = generateTableName diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestMetadataProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestMetadataProcedure.scala index c618c227ce1d..b3ce71c70eb9 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestMetadataProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestMetadataProcedure.scala @@ -91,6 +91,72 @@ class TestMetadataProcedure extends HoodieSparkProcedureTestBase { } } + test("Test Call show_metadata_table_column_stats Procedure") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | c1 int, + | c2 boolean, + | c3 binary, + | c4 date, + | c5 decimal(10,1), + | c6 double, + | c7 float, + | c8 long, + | c9 string, + | c10 timestamp + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'c1', + | preCombineField = 'c8', + | hoodie.metadata.enable="true", + | hoodie.metadata.index.column.stats.enable="true" + | ) + """.stripMargin) + // insert data to table + + spark.sql( + s""" + |insert into table $tableName + |values (1, true, CAST('binary data' AS BINARY), CAST('2021-01-01' AS DATE), CAST(10.5 AS DECIMAL(10,1)), CAST(3.14 AS DOUBLE), CAST(2.5 AS FLOAT), 1000, 'example string', CAST('2021-01-01 00:00:00' AS TIMESTAMP)) + |""".stripMargin) + spark.sql( + s""" + |insert into table $tableName + |values (10, false, CAST('binary data' AS BINARY), CAST('2022-02-02' AS DATE), CAST(20.5 AS DECIMAL(10,1)), CAST(6.28 AS DOUBLE), CAST(3.14 AS FLOAT), 2000, 'another string', CAST('2022-02-02 00:00:00' AS TIMESTAMP)) + |""".stripMargin) + + // Only numerical and string types are compared for clarity on min/max values. + val expectedValues = Map( + 1 -> ("1", "10"), + 2 -> ("false", "true"), + 6 -> ("3.14", "6.28"), + 7 -> ("2.5", "3.14"), + 8 -> ("1000", "2000"), + 9 -> ("another string", "example string") + ) + + for (i <- 1 to 10) { + val columnName = s"c$i" + val metadataStats = spark.sql(s"""call show_metadata_table_column_stats(table => '$tableName', targetColumns => '$columnName')""").collect() + assertResult(1)(metadataStats.length) + val minVal: String = metadataStats(0).getAs[String]("min_value") + val maxVal: String = metadataStats(0).getAs[String]("max_value") + + expectedValues.get(i) match { + case Some((expectedMin, expectedMax)) => + assertResult(expectedMin)(minVal) + assertResult(expectedMax)(maxVal) + case None => // Do nothing if no expected values found + } + } + } + } + test("Test Call show_metadata_table_stats Procedure") { withTempDir { tmp => val tableName = generateTableName diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestSavepointsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestSavepointsProcedure.scala index c8fa10bde2c6..af31cd4bb2c4 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestSavepointsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestSavepointsProcedure.scala @@ -214,6 +214,77 @@ class TestSavepointsProcedure extends HoodieSparkProcedureTestBase { } } + test("Test Call delete_savepoint Procedure with batch mode") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = tmp.getCanonicalPath + "/" + tableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + spark.sql(s"insert into $tableName select 3, 'a3', 30, 2000") + spark.sql(s"insert into $tableName select 4, 'a4', 40, 2500") + + val commits = spark.sql(s"""call show_commits(table => '$tableName')""").collect() + assertResult(4) { + commits.length + } + + // create 4 savepoints + commits.foreach(r => { + checkAnswer(s"""call create_savepoint('$tableName', '${r.getString(0)}')""")(Seq(true)) + }) + + // Delete 2 savepoint with table name and instant time + val toDeleteInstant = s"${commits.apply(1).getString(0)},${commits.apply(0).getString(0)}" + checkAnswer(s"""call delete_savepoint('$tableName', '${toDeleteInstant}')""")(Seq(true)) + + // show_savepoints should return two savepoint + var savepoints = spark.sql(s"""call show_savepoints(table => '$tableName')""").collect() + assertResult(2) { + savepoints.length + } + + assertResult(commits(2).getString(0))(savepoints(0).getString(0)) + assertResult(commits(3).getString(0))(savepoints(1).getString(0)) + + // Delete a savepoint with table name and latest savepoint time + checkAnswer(s"""call delete_savepoint('$tableName', '')""")(Seq(true)) + + // show_savepoints should return one savepoint + savepoints = spark.sql(s"""call show_savepoints(table => '$tableName')""").collect() + assertResult(1) { + savepoints.length + } + + assertResult(commits(3).getString(0))(savepoints(0).getString(0)) + + // Delete a savepoint with table base path and latest savepoint time + checkAnswer(s"""call delete_savepoint(path => '$tablePath')""".stripMargin)(Seq(true)) + + // show_savepoints should return zero savepoint + savepoints = spark.sql(s"""call show_savepoints(table => '$tableName')""").collect() + assertResult(0) { + savepoints.length + } + } + } + test("Test Call rollback_to_savepoint Procedure") { withTempDir { tmp => val tableName = generateTableName diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestUpgradeOrDowngradeProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestUpgradeOrDowngradeProcedure.scala index ff4b5aa92ead..1bd29cabc400 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestUpgradeOrDowngradeProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestUpgradeOrDowngradeProcedure.scala @@ -82,6 +82,33 @@ class TestUpgradeOrDowngradeProcedure extends HoodieSparkProcedureTestBase { } } + test("Test Call upgrade_table from version three") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = s"${tmp.getCanonicalPath}/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // downgrade table to THREE + checkAnswer(s"""call downgrade_table(table => '$tableName', to_version => 'THREE')""")(Seq(true)) + // upgrade table to FOUR + checkAnswer(s"""call upgrade_table(table => '$tableName', to_version => 'FOUR')""")(Seq(true)) + } + } + @throws[IOException] private def assertTableVersionFromPropertyFile(metaClient: HoodieTableMetaClient, versionCode: Int): Unit = { val propertyFile = new Path(metaClient.getMetaPath + "/" + HoodieTableConfig.HOODIE_PROPERTIES_FILE) diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java index 5150e0debbe3..9d44bbdc07ef 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java @@ -46,7 +46,6 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty; import static org.apache.hudi.common.util.StringUtils.nonEmpty; import static org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.getInputFormatClassName; import static org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.getOutputFormatClassName; @@ -103,15 +102,19 @@ public class HiveSyncTool extends HoodieSyncTool implements AutoCloseable { public HiveSyncTool(Properties props, Configuration hadoopConf) { super(props, hadoopConf); - String metastoreUris = props.getProperty(METASTORE_URIS.key()); - // Give precedence to HiveConf.ConfVars.METASTOREURIS if it is set. - // Else if user has provided HiveSyncConfigHolder.METASTORE_URIS, then set that in hadoop conf. - if (isNullOrEmpty(hadoopConf.get(HiveConf.ConfVars.METASTOREURIS.varname)) && nonEmpty(metastoreUris)) { - LOG.info(String.format("Setting %s = %s", HiveConf.ConfVars.METASTOREURIS.varname, metastoreUris)); - hadoopConf.set(HiveConf.ConfVars.METASTOREURIS.varname, metastoreUris); + String configuredMetastoreUris = props.getProperty(METASTORE_URIS.key()); + + final Configuration hadoopConfForSync; // the configuration to use for this instance of the sync tool + if (nonEmpty(configuredMetastoreUris)) { + // if metastore uri is configured, we can create a new configuration with the value set + hadoopConfForSync = new Configuration(hadoopConf); + hadoopConfForSync.set(HiveConf.ConfVars.METASTOREURIS.varname, configuredMetastoreUris); + } else { + // if the user did not provide any URIs, then we can use the provided configuration + hadoopConfForSync = hadoopConf; } - HiveSyncConfig config = new HiveSyncConfig(props, hadoopConf); - this.config = config; + + this.config = new HiveSyncConfig(props, hadoopConfForSync); this.databaseName = config.getStringOrDefault(META_SYNC_DATABASE_NAME); this.tableName = config.getStringOrDefault(META_SYNC_TABLE_NAME); initSyncClient(config); diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java index 345070fbe5c0..c72491341fe4 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java @@ -24,12 +24,12 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.dto.BaseFileDTO; -import org.apache.hudi.common.table.timeline.dto.InstantStateDTO; import org.apache.hudi.common.table.timeline.dto.ClusteringOpDTO; import org.apache.hudi.common.table.timeline.dto.CompactionOpDTO; import org.apache.hudi.common.table.timeline.dto.FileGroupDTO; import org.apache.hudi.common.table.timeline.dto.FileSliceDTO; import org.apache.hudi.common.table.timeline.dto.InstantDTO; +import org.apache.hudi.common.table.timeline.dto.InstantStateDTO; import org.apache.hudi.common.table.timeline.dto.TimelineDTO; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView; @@ -45,6 +45,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.module.afterburner.AfterburnerModule; import io.javalin.Javalin; import io.javalin.http.BadRequestResponse; import io.javalin.http.Context; @@ -69,7 +70,7 @@ */ public class RequestHandler { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper().registerModule(new AfterburnerModule()); private static final Logger LOG = LoggerFactory.getLogger(RequestHandler.class); private final TimelineService.Config timelineServiceConfig; @@ -336,6 +337,14 @@ private void registerFileSlicesAPI() { writeValueAsString(ctx, dtos); }, true)); + app.get(RemoteHoodieTableFileSystemView.LATEST_PARTITION_SLICES_STATELESS_URL, new ViewHandler(ctx -> { + metricsRegistry.add("LATEST_PARTITION_SLICES_STATELESS", 1); + List dtos = sliceHandler.getLatestFileSlicesStateless( + ctx.queryParamAsClass(RemoteHoodieTableFileSystemView.BASEPATH_PARAM, String.class).getOrThrow(e -> new HoodieException("Basepath is invalid")), + ctx.queryParamAsClass(RemoteHoodieTableFileSystemView.PARTITION_PARAM, String.class).getOrDefault("")); + writeValueAsString(ctx, dtos); + }, true)); + app.get(RemoteHoodieTableFileSystemView.LATEST_PARTITION_SLICE_URL, new ViewHandler(ctx -> { metricsRegistry.add("LATEST_PARTITION_SLICE", 1); List dtos = sliceHandler.getLatestFileSlice( @@ -420,6 +429,14 @@ private void registerFileSlicesAPI() { writeValueAsString(ctx, dtos); }, true)); + app.get(RemoteHoodieTableFileSystemView.ALL_FILEGROUPS_FOR_PARTITION_STATELESS_URL, new ViewHandler(ctx -> { + metricsRegistry.add("ALL_FILEGROUPS_FOR_PARTITION_STATELESS", 1); + List dtos = sliceHandler.getAllFileGroupsStateless( + ctx.queryParamAsClass(RemoteHoodieTableFileSystemView.BASEPATH_PARAM, String.class).getOrThrow(e -> new HoodieException("Basepath is invalid")), + ctx.queryParamAsClass(RemoteHoodieTableFileSystemView.PARTITION_PARAM, String.class).getOrDefault("")); + writeValueAsString(ctx, dtos); + }, true)); + app.post(RemoteHoodieTableFileSystemView.REFRESH_TABLE, new ViewHandler(ctx -> { metricsRegistry.add("REFRESH_TABLE", 1); boolean success = sliceHandler diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java index 790d476311ff..f215c26bc151 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java @@ -52,8 +52,8 @@ public class TimelineService { private static final int DEFAULT_NUM_THREADS = 250; private int serverPort; - private Config timelineServerConf; - private Configuration conf; + private final Config timelineServerConf; + private final Configuration conf; private transient HoodieEngineContext context; private transient FileSystem fs; private transient Javalin app = null; @@ -434,6 +434,10 @@ public void close() { LOG.info("Closed Timeline Service"); } + public void unregisterBasePath(String basePath) { + fsViewsManager.clearFileSystemView(basePath); + } + public Configuration getConf() { return conf; } diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/BaseFileHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/BaseFileHandler.java index a34b49843fac..5a5fa00b0de9 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/BaseFileHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/BaseFileHandler.java @@ -26,8 +26,7 @@ import org.apache.hadoop.fs.FileSystem; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -49,7 +48,7 @@ public List getLatestDataFiles(String basePath, String partitionPat public List getLatestDataFile(String basePath, String partitionPath, String fileId) { return viewManager.getFileSystemView(basePath).getLatestBaseFile(partitionPath, fileId) - .map(BaseFileDTO::fromHoodieBaseFile).map(Arrays::asList).orElse(new ArrayList<>()); + .map(BaseFileDTO::fromHoodieBaseFile).map(Collections::singletonList).orElse(Collections.emptyList()); } public List getLatestDataFiles(String basePath) { @@ -74,10 +73,8 @@ public Map> getAllLatestDataFilesBeforeOrOn(String bas public List getLatestDataFileOn(String basePath, String partitionPath, String instantTime, String fileId) { - List result = new ArrayList<>(); - viewManager.getFileSystemView(basePath).getBaseFileOn(partitionPath, instantTime, fileId) - .map(BaseFileDTO::fromHoodieBaseFile).ifPresent(result::add); - return result; + return viewManager.getFileSystemView(basePath).getBaseFileOn(partitionPath, instantTime, fileId) + .map(BaseFileDTO::fromHoodieBaseFile).map(Collections::singletonList).orElse(Collections.emptyList()); } public List getLatestDataFilesInRange(String basePath, List instants) { diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java index e8af55e69b38..c2b739c9f8bb 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java @@ -18,9 +18,6 @@ package org.apache.hudi.timeline.service.handlers; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; - import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.table.timeline.dto.ClusteringOpDTO; import org.apache.hudi.common.table.timeline.dto.CompactionOpDTO; @@ -30,6 +27,9 @@ import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.timeline.service.TimelineService; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; + import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -90,6 +90,11 @@ public List getLatestFileSlices(String basePath, String partitionP .collect(Collectors.toList()); } + public List getLatestFileSlicesStateless(String basePath, String partitionPath) { + return viewManager.getFileSystemView(basePath).getLatestFileSlicesStateless(partitionPath).map(FileSliceDTO::fromFileSlice) + .collect(Collectors.toList()); + } + public List getLatestFileSlice(String basePath, String partitionPath, String fileId) { return viewManager.getFileSystemView(basePath).getLatestFileSlice(partitionPath, fileId) .map(FileSliceDTO::fromFileSlice).map(Arrays::asList).orElse(new ArrayList<>()); @@ -113,6 +118,12 @@ public List getAllFileGroups(String basePath, String partitionPath return DTOUtils.fileGroupDTOsfromFileGroups(fileGroups); } + public List getAllFileGroupsStateless(String basePath, String partitionPath) { + List fileGroups = viewManager.getFileSystemView(basePath).getAllFileGroupsStateless(partitionPath) + .collect(Collectors.toList()); + return DTOUtils.fileGroupDTOsfromFileGroups(fileGroups); + } + public List getReplacedFileGroupsBeforeOrOn(String basePath, String maxCommitTime, String partitionPath) { List fileGroups = viewManager.getFileSystemView(basePath).getReplacedFileGroupsBeforeOrOn(maxCommitTime, partitionPath) .collect(Collectors.toList()); diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java index 9f8ed5d84cfe..05551dc42dde 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java @@ -32,6 +32,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.module.afterburner.AfterburnerModule; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -64,7 +65,7 @@ */ public class MarkerDirState implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(MarkerDirState.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper().registerModule(new AfterburnerModule()); // Marker directory private final String markerDirPath; private final FileSystem fileSystem; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/config/CloudSourceConfig.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/config/CloudSourceConfig.java index 16d9b73c70e2..81533d940a8c 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/config/CloudSourceConfig.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/config/CloudSourceConfig.java @@ -53,7 +53,17 @@ public class CloudSourceConfig extends HoodieConfig { .defaultValue(10) .withAlternatives(DELTA_STREAMER_CONFIG_PREFIX + "source.cloud.meta.batch.size") .markAdvanced() - .withDocumentation("Number of metadata messages to pull at a time"); + .withDocumentation("Number of metadata messages to pull in one API call to the cloud events queue. " + + "Multiple API calls with this batch size are sent to cloud events queue, until we consume hoodie.streamer.source.cloud.meta.max.num.messages.per.sync" + + "from the queue or hoodie.streamer.source.cloud.meta.max.fetch.time.per.sync.ms amount of time has passed or queue is empty. "); + + public static final ConfigProperty MAX_NUM_MESSAGES_PER_SYNC = ConfigProperty + .key(STREAMER_CONFIG_PREFIX + "source.cloud.meta.max.num.messages.per.sync") + .defaultValue(1000) + .markAdvanced() + .sinceVersion("0.14.1") + .withDocumentation("Maximum number of messages to consume per sync round. Multiple rounds of " + + BATCH_SIZE_CONF.key() + " could be invoked to reach max messages as configured by this config"); public static final ConfigProperty ACK_MESSAGES = ConfigProperty .key(STREAMER_CONFIG_PREFIX + "source.cloud.meta.ack") @@ -108,7 +118,7 @@ public class CloudSourceConfig extends HoodieConfig { public static final ConfigProperty DATAFILE_FORMAT = ConfigProperty .key(STREAMER_CONFIG_PREFIX + "source.cloud.data.datafile.format") - .defaultValue("parquet") + .defaultValue(HoodieIncrSourceConfig.SOURCE_FILE_FORMAT.defaultValue()) .withAlternatives(DELTA_STREAMER_CONFIG_PREFIX + "source.cloud.data.datafile.format") .markAdvanced() .withDocumentation("Format of the data file. By default, this will be the same as hoodie.streamer.source.hoodieincr.file.format"); @@ -121,4 +131,28 @@ public class CloudSourceConfig extends HoodieConfig { .sinceVersion("0.14.0") .withDocumentation("A comma delimited list of path-based partition fields in the source file structure."); + public static final ConfigProperty SPARK_DATASOURCE_READER_COMMA_SEPARATED_PATH_FORMAT = ConfigProperty + .key(STREAMER_CONFIG_PREFIX + "source.cloud.data.reader.comma.separated.path.format") + .defaultValue(false) + .markAdvanced() + .sinceVersion("0.14.1") + .withDocumentation("Boolean value for specifying path format in load args of spark.read.format(\"..\").load(\"a.xml,b.xml,c.xml\"),\n" + + " * set true if path format needs to be comma separated string value, if false it's passed as array of strings like\n" + + " * spark.read.format(\"..\").load(new String[]{a.xml,b.xml,c.xml})"); + + public static final ConfigProperty SOURCE_MAX_BYTES_PER_PARTITION = ConfigProperty + .key(STREAMER_CONFIG_PREFIX + "source.cloud.data.partition.max.size") + .noDefaultValue() + .markAdvanced() + .sinceVersion("0.14.1") + .withDocumentation("specify this value in bytes, to coalesce partitions of source dataset not greater than specified limit"); + + public static final ConfigProperty MAX_FETCH_TIME_PER_SYNC_MS = ConfigProperty + .key(STREAMER_CONFIG_PREFIX + "source.cloud.meta.max.fetch.time.per.sync.ms") + .defaultValue(1) + .markAdvanced() + .sinceVersion("0.14.1") + .withDocumentation("Max time in millis to consume " + MAX_NUM_MESSAGES_PER_SYNC.key() + " messages from cloud queue. Cloud event queues like SQS, " + + "PubSub can return empty responses even when messages are available the queue, this config ensures we don't wait forever " + + "to consume MAX_MESSAGES_CONF messages, but time out and move on further."); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SchemaRegistryProvider.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SchemaRegistryProvider.java index 780fbb9dc0ad..110c8cc2fb13 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SchemaRegistryProvider.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SchemaRegistryProvider.java @@ -195,7 +195,10 @@ public Schema getSourceSchema() { try { return parseSchemaFromRegistry(registryUrl); } catch (Exception e) { - throw new HoodieSchemaFetchException("Error reading source schema from registry :" + registryUrl, e); + throw new HoodieSchemaFetchException(String.format( + "Error reading source schema from registry. Please check %s is configured correctly. Truncated URL: %s", + Config.SRC_SCHEMA_REGISTRY_URL_PROP, + StringUtils.truncate(registryUrl, 10, 10)), e); } } @@ -207,7 +210,11 @@ public Schema getTargetSchema() { try { return parseSchemaFromRegistry(targetRegistryUrl); } catch (Exception e) { - throw new HoodieSchemaFetchException("Error reading target schema from registry :" + targetRegistryUrl, e); + throw new HoodieSchemaFetchException(String.format( + "Error reading target schema from registry. Please check %s is configured correctly. If that is not configured then check %s. Truncated URL: %s", + Config.SRC_SCHEMA_REGISTRY_URL_PROP, + Config.TARGET_SCHEMA_REGISTRY_URL_PROP, + StringUtils.truncate(targetRegistryUrl, 10, 10)), e); } } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java index e9353bb26660..2bf92280faf5 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java @@ -19,6 +19,7 @@ package org.apache.hudi.utilities.sources; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.Option; import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.deser.KafkaAvroSchemaDeserializer; import org.apache.hudi.utilities.exception.HoodieReadFromSourceException; @@ -78,18 +79,25 @@ public AvroKafkaSource(TypedProperties props, JavaSparkContext sparkContext, Spa try { props.put(NATIVE_KAFKA_VALUE_DESERIALIZER_PROP, Class.forName(deserializerClassName).getName()); - if (deserializerClassName.equals(KafkaAvroSchemaDeserializer.class.getName())) { - if (schemaProvider == null) { - throw new HoodieReadFromSourceException("SchemaProvider has to be set to use KafkaAvroSchemaDeserializer"); - } - props.put(KAFKA_VALUE_DESERIALIZER_SCHEMA.key(), schemaProvider.getSourceSchema().toString()); - } } catch (ClassNotFoundException e) { String error = "Could not load custom avro kafka deserializer: " + deserializerClassName; LOG.error(error); throw new HoodieReadFromSourceException(error, e); } - this.offsetGen = new KafkaOffsetGen(props); + + if (deserializerClassName.equals(KafkaAvroSchemaDeserializer.class.getName())) { + configureSchemaDeserializer(); + } + offsetGen = new KafkaOffsetGen(props); + } + + @Override + protected InputBatch> fetchNewData(Option lastCheckpointStr, long sourceLimit) { + if (deserializerClassName.equals(KafkaAvroSchemaDeserializer.class.getName())) { + configureSchemaDeserializer(); + offsetGen = new KafkaOffsetGen(props); + } + return super.fetchNewData(lastCheckpointStr, sourceLimit); } @Override @@ -121,4 +129,11 @@ protected JavaRDD maybeAppendKafkaOffsets(JavaRDD (GenericRecord) consumerRecord.value()); } } + + private void configureSchemaDeserializer() { + if (schemaProvider == null) { + throw new HoodieReadFromSourceException("SchemaProvider has to be set to use KafkaAvroSchemaDeserializer"); + } + props.put(KAFKA_VALUE_DESERIALIZER_SCHEMA.key(), schemaProvider.getSourceSchema().toString()); + } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/GcsEventsHoodieIncrSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/GcsEventsHoodieIncrSource.java index d09bad719167..a06130d39728 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/GcsEventsHoodieIncrSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/GcsEventsHoodieIncrSource.java @@ -114,6 +114,7 @@ public class GcsEventsHoodieIncrSource extends HoodieIncrSource { private final CloudDataFetcher gcsObjectDataFetcher; private final QueryRunner queryRunner; private final Option schemaProvider; + private final Option snapshotLoadQuerySplitter; public static final String GCS_OBJECT_KEY = "name"; @@ -145,6 +146,7 @@ public GcsEventsHoodieIncrSource(TypedProperties props, JavaSparkContext jsc, Sp this.gcsObjectDataFetcher = gcsObjectDataFetcher; this.queryRunner = queryRunner; this.schemaProvider = Option.ofNullable(schemaProvider); + this.snapshotLoadQuerySplitter = SnapshotLoadQuerySplitter.getInstance(props); LOG.info("srcPath: " + srcPath); LOG.info("missingCheckpointStrategy: " + missingCheckpointStrategy); @@ -171,8 +173,9 @@ public Pair>, String> fetchNextBatch(Option lastChec return Pair.of(Option.empty(), queryInfo.getStartInstant()); } - Dataset cloudObjectMetadataDF = queryRunner.run(queryInfo); - Dataset filteredSourceData = gcsObjectMetadataFetcher.applyFilter(cloudObjectMetadataDF); + Pair> queryInfoDatasetPair = queryRunner.run(queryInfo, snapshotLoadQuerySplitter); + Dataset filteredSourceData = gcsObjectMetadataFetcher.applyFilter(queryInfoDatasetPair.getRight()); + queryInfo = queryInfoDatasetPair.getLeft(); LOG.info("Adjusting end checkpoint:" + queryInfo.getEndInstant() + " based on sourceLimit :" + sourceLimit); Pair>> checkPointAndDataset = IncrSourceHelper.filterAndGenerateCheckpointBasedOnSourceLimit( diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/GcsEventsSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/GcsEventsSource.java index f934f2794989..897771168edf 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/GcsEventsSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/GcsEventsSource.java @@ -49,6 +49,8 @@ import static org.apache.hudi.common.util.ConfigUtils.getStringWithAltKeys; import static org.apache.hudi.utilities.config.CloudSourceConfig.ACK_MESSAGES; import static org.apache.hudi.utilities.config.CloudSourceConfig.BATCH_SIZE_CONF; +import static org.apache.hudi.utilities.config.CloudSourceConfig.MAX_FETCH_TIME_PER_SYNC_MS; +import static org.apache.hudi.utilities.config.CloudSourceConfig.MAX_NUM_MESSAGES_PER_SYNC; import static org.apache.hudi.utilities.config.GCSEventsSourceConfig.GOOGLE_PROJECT_ID; import static org.apache.hudi.utilities.config.GCSEventsSourceConfig.PUBSUB_SUBSCRIPTION_ID; import static org.apache.hudi.utilities.sources.helpers.gcs.MessageValidity.ProcessingDecision.DO_SKIP; @@ -117,8 +119,9 @@ public GcsEventsSource(TypedProperties props, JavaSparkContext jsc, SparkSession new PubsubMessagesFetcher( getStringWithAltKeys(props, GOOGLE_PROJECT_ID), getStringWithAltKeys(props, PUBSUB_SUBSCRIPTION_ID), - getIntWithAltKeys(props, BATCH_SIZE_CONF) - ) + getIntWithAltKeys(props, BATCH_SIZE_CONF), + getIntWithAltKeys(props, MAX_NUM_MESSAGES_PER_SYNC), + getIntWithAltKeys(props, MAX_FETCH_TIME_PER_SYNC_MS)) ); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java index aafd4c9e3b56..f87e5c231bf3 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.table.timeline.TimelineUtils.HollowCommitHandling; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.utilities.config.HoodieIncrSourceConfig; import org.apache.hudi.utilities.schema.SchemaProvider; @@ -46,7 +45,7 @@ import static org.apache.hudi.DataSourceReadOptions.BEGIN_INSTANTTIME; import static org.apache.hudi.DataSourceReadOptions.END_INSTANTTIME; -import static org.apache.hudi.DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES; +import static org.apache.hudi.DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN; import static org.apache.hudi.DataSourceReadOptions.INCREMENTAL_READ_HANDLE_HOLLOW_COMMIT; import static org.apache.hudi.DataSourceReadOptions.QUERY_TYPE; import static org.apache.hudi.DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL; @@ -56,7 +55,6 @@ import static org.apache.hudi.common.util.ConfigUtils.getIntWithAltKeys; import static org.apache.hudi.common.util.ConfigUtils.getStringWithAltKeys; import static org.apache.hudi.utilities.UtilHelpers.createRecordMerger; -import static org.apache.hudi.utilities.sources.SnapshotLoadQuerySplitter.Config.SNAPSHOT_LOAD_QUERY_SPLITTER_CLASS_NAME; import static org.apache.hudi.utilities.sources.helpers.IncrSourceHelper.generateQueryInfo; import static org.apache.hudi.utilities.sources.helpers.IncrSourceHelper.getHollowCommitHandleMode; @@ -150,9 +148,7 @@ public HoodieIncrSource(TypedProperties props, JavaSparkContext sparkContext, Sp } } - this.snapshotLoadQuerySplitter = Option.ofNullable(props.getString(SNAPSHOT_LOAD_QUERY_SPLITTER_CLASS_NAME, null)) - .map(className -> (SnapshotLoadQuerySplitter) ReflectionUtils.loadClass(className, - new Class[] {TypedProperties.class}, props)); + this.snapshotLoadQuerySplitter = SnapshotLoadQuerySplitter.getInstance(props); } @Override @@ -203,9 +199,9 @@ public Pair>, String> fetchNextBatch(Option lastCkpt .option(QUERY_TYPE().key(), QUERY_TYPE_INCREMENTAL_OPT_VAL()) .option(BEGIN_INSTANTTIME().key(), queryInfo.getStartInstant()) .option(END_INSTANTTIME().key(), queryInfo.getEndInstant()) - .option(INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES().key(), - props.getString(INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES().key(), - INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES().defaultValue())) + .option(INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN().key(), + props.getString(INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN().key(), + INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN().defaultValue())) .option(INCREMENTAL_READ_HANDLE_HOLLOW_COMMIT().key(), handlingMode.name()) .load(srcPath); } else { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java index 4b9be847c756..61ed02da106f 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java @@ -35,6 +35,7 @@ import org.apache.hudi.utilities.sources.helpers.QueryInfo; import org.apache.hudi.utilities.sources.helpers.QueryRunner; +import org.apache.parquet.Strings; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; @@ -51,6 +52,7 @@ import static org.apache.hudi.common.util.ConfigUtils.getIntWithAltKeys; import static org.apache.hudi.common.util.ConfigUtils.getStringWithAltKeys; import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty; +import static org.apache.hudi.utilities.config.CloudSourceConfig.DATAFILE_FORMAT; import static org.apache.hudi.utilities.config.CloudSourceConfig.ENABLE_EXISTS_CHECK; import static org.apache.hudi.utilities.config.HoodieIncrSourceConfig.HOODIE_SRC_BASE_PATH; import static org.apache.hudi.utilities.config.HoodieIncrSourceConfig.NUM_INSTANTS_PER_FETCH; @@ -70,6 +72,7 @@ public class S3EventsHoodieIncrSource extends HoodieIncrSource { private static final Logger LOG = LoggerFactory.getLogger(S3EventsHoodieIncrSource.class); + private static final String EMPTY_STRING = ""; private final String srcPath; private final int numInstantsPerFetch; private final boolean checkIfFileExists; @@ -80,6 +83,8 @@ public class S3EventsHoodieIncrSource extends HoodieIncrSource { private final Option schemaProvider; + private final Option snapshotLoadQuerySplitter; + public static class Config { // control whether we do existence check for files before consuming them @Deprecated @@ -133,11 +138,18 @@ public S3EventsHoodieIncrSource( this.srcPath = getStringWithAltKeys(props, HOODIE_SRC_BASE_PATH); this.numInstantsPerFetch = getIntWithAltKeys(props, NUM_INSTANTS_PER_FETCH); this.checkIfFileExists = getBooleanWithAltKeys(props, ENABLE_EXISTS_CHECK); - this.fileFormat = getStringWithAltKeys(props, SOURCE_FILE_FORMAT, true); + + // This is to ensure backward compatibility where we were using the + // config SOURCE_FILE_FORMAT for file format in previous versions. + this.fileFormat = Strings.isNullOrEmpty(getStringWithAltKeys(props, DATAFILE_FORMAT, EMPTY_STRING)) + ? getStringWithAltKeys(props, SOURCE_FILE_FORMAT, true) + : getStringWithAltKeys(props, DATAFILE_FORMAT, EMPTY_STRING); + this.missingCheckpointStrategy = getMissingCheckpointStrategy(props); this.queryRunner = queryRunner; this.cloudDataFetcher = cloudDataFetcher; this.schemaProvider = Option.ofNullable(schemaProvider); + this.snapshotLoadQuerySplitter = SnapshotLoadQuerySplitter.getInstance(props); } @Override @@ -158,9 +170,9 @@ public Pair>, String> fetchNextBatch(Option lastChec LOG.warn("Already caught up. No new data to process"); return Pair.of(Option.empty(), queryInfo.getEndInstant()); } - - Dataset source = queryRunner.run(queryInfo); - Dataset filteredSourceData = applyFilter(source, fileFormat); + Pair> queryInfoDatasetPair = queryRunner.run(queryInfo, snapshotLoadQuerySplitter); + queryInfo = queryInfoDatasetPair.getLeft(); + Dataset filteredSourceData = applyFilter(queryInfoDatasetPair.getRight(), fileFormat); LOG.info("Adjusting end checkpoint:" + queryInfo.getEndInstant() + " based on sourceLimit :" + sourceLimit); Pair>> checkPointAndDataset = diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/SnapshotLoadQuerySplitter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/SnapshotLoadQuerySplitter.java index 6a13607b1d5e..ca299122ec72 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/SnapshotLoadQuerySplitter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/SnapshotLoadQuerySplitter.java @@ -20,10 +20,13 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.utilities.sources.helpers.QueryInfo; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import static org.apache.hudi.utilities.sources.SnapshotLoadQuerySplitter.Config.SNAPSHOT_LOAD_QUERY_SPLITTER_CLASS_NAME; + /** * Abstract splitter responsible for managing the snapshot load query operations. */ @@ -75,4 +78,10 @@ public QueryInfo getNextCheckpoint(Dataset df, QueryInfo queryInfo) { .map(checkpoint -> queryInfo.withUpdatedEndInstant(checkpoint)) .orElse(queryInfo); } + + public static Option getInstance(TypedProperties props) { + return props.getNonEmptyStringOpt(SNAPSHOT_LOAD_QUERY_SPLITTER_CLASS_NAME, null) + .map(className -> (SnapshotLoadQuerySplitter) ReflectionUtils.loadClass(className, + new Class[] {TypedProperties.class}, props)); + } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/CloudObjectsSelector.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/CloudObjectsSelector.java index efe2913255f3..8c447d93a0ff 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/CloudObjectsSelector.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/CloudObjectsSelector.java @@ -200,9 +200,12 @@ protected List> createListPartitions(List singleList, int * Delete batch of messages from queue. */ protected void deleteBatchOfMessages(SqsClient sqs, String queueUrl, List messagesToBeDeleted) { - DeleteMessageBatchRequest deleteBatchReq = - DeleteMessageBatchRequest.builder().queueUrl(queueUrl).build(); - List deleteEntries = new ArrayList<>(deleteBatchReq.entries()); + if (messagesToBeDeleted.isEmpty()) { + return; + } + DeleteMessageBatchRequest.Builder builder = DeleteMessageBatchRequest.builder().queueUrl(queueUrl); + List deleteEntries = new ArrayList<>(); + for (Message message : messagesToBeDeleted) { deleteEntries.add( DeleteMessageBatchRequestEntry.builder() @@ -210,7 +213,8 @@ protected void deleteBatchOfMessages(SqsClient sqs, String queueUrl, List deleteFailures = deleteResponse.failed().stream() .map(BatchResultErrorEntry::id) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/CloudObjectsSelectorCommon.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/CloudObjectsSelectorCommon.java index 19da6aada9bd..4098448b7936 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/CloudObjectsSelectorCommon.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/CloudObjectsSelectorCommon.java @@ -57,7 +57,8 @@ import static org.apache.hudi.common.util.ConfigUtils.containsConfigProperty; import static org.apache.hudi.common.util.ConfigUtils.getStringWithAltKeys; import static org.apache.hudi.utilities.config.CloudSourceConfig.PATH_BASED_PARTITION_FIELDS; -import static org.apache.hudi.utilities.sources.helpers.CloudStoreIngestionConfig.SPARK_DATASOURCE_READER_COMMA_SEPARATED_PATH_FORMAT; +import static org.apache.hudi.utilities.config.CloudSourceConfig.SOURCE_MAX_BYTES_PER_PARTITION; +import static org.apache.hudi.utilities.config.CloudSourceConfig.SPARK_DATASOURCE_READER_COMMA_SEPARATED_PATH_FORMAT; import static org.apache.spark.sql.functions.input_file_name; import static org.apache.spark.sql.functions.split; @@ -191,9 +192,11 @@ public static Option> loadAsDataset(SparkSession spark, List dataset; if (isCommaSeparatedPathFormat) { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/CloudStoreIngestionConfig.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/CloudStoreIngestionConfig.java index 66b94177b7b0..8a1c15c88869 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/CloudStoreIngestionConfig.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/CloudStoreIngestionConfig.java @@ -107,11 +107,4 @@ public class CloudStoreIngestionConfig { * A comma delimited list of path-based partition fields in the source file structure */ public static final String PATH_BASED_PARTITION_FIELDS = "hoodie.deltastreamer.source.cloud.data.partition.fields.from.path"; - - /** - * boolean value for specifying path format in load args of spark.read.format("..").load("a.xml,b.xml,c.xml"), - * set true if path format needs to be comma separated string value, if false it's passed as array of strings like - * spark.read.format("..").load(new String[]{a.xml,b.xml,c.xml}) - */ - public static final String SPARK_DATASOURCE_READER_COMMA_SEPARATED_PATH_FORMAT = "hoodie.deltastreamer.source.cloud.data.reader.comma.separated.path.format"; } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/QueryRunner.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/QueryRunner.java index 761e942549c1..2f0a8bf488e8 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/QueryRunner.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/QueryRunner.java @@ -21,9 +21,12 @@ import org.apache.hudi.DataSourceReadOptions; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.utilities.config.HoodieIncrSourceConfig; +import org.apache.hudi.utilities.sources.SnapshotLoadQuerySplitter; import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -44,12 +47,14 @@ */ public class QueryRunner { private final SparkSession sparkSession; + private final TypedProperties props; private final String sourcePath; private static final Logger LOG = LoggerFactory.getLogger(QueryRunner.class); public QueryRunner(SparkSession sparkSession, TypedProperties props) { this.sparkSession = sparkSession; + this.props = props; checkRequiredConfigProperties(props, Collections.singletonList(HoodieIncrSourceConfig.HOODIE_SRC_BASE_PATH)); this.sourcePath = getStringWithAltKeys(props, HoodieIncrSourceConfig.HOODIE_SRC_BASE_PATH); } @@ -60,16 +65,14 @@ public QueryRunner(SparkSession sparkSession, TypedProperties props) { * @param queryInfo all meta info about the query to be executed. * @return the output of the query as Dataset < Row >. */ - public Dataset run(QueryInfo queryInfo) { - Dataset dataset = null; + public Pair> run(QueryInfo queryInfo, Option snapshotLoadQuerySplitterOption) { if (queryInfo.isIncremental()) { - dataset = runIncrementalQuery(queryInfo); + return runIncrementalQuery(queryInfo); } else if (queryInfo.isSnapshot()) { - dataset = runSnapshotQuery(queryInfo); + return runSnapshotQuery(queryInfo, snapshotLoadQuerySplitterOption); } else { throw new HoodieException("Unknown query type " + queryInfo.getQueryType()); } - return dataset; } public static Dataset applyOrdering(Dataset dataset, List orderByColumns) { @@ -80,22 +83,34 @@ public static Dataset applyOrdering(Dataset dataset, List orde return dataset; } - public Dataset runIncrementalQuery(QueryInfo queryInfo) { + public Pair> runIncrementalQuery(QueryInfo queryInfo) { LOG.info("Running incremental query"); - return sparkSession.read().format("org.apache.hudi") + return Pair.of(queryInfo, sparkSession.read().format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE().key(), queryInfo.getQueryType()) .option(DataSourceReadOptions.BEGIN_INSTANTTIME().key(), queryInfo.getPreviousInstant()) - .option(DataSourceReadOptions.END_INSTANTTIME().key(), queryInfo.getEndInstant()).load(sourcePath); + .option(DataSourceReadOptions.END_INSTANTTIME().key(), queryInfo.getEndInstant()) + .option(DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN().key(), + props.getString(DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN().key(), + DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN().defaultValue())) + .load(sourcePath)); } - public Dataset runSnapshotQuery(QueryInfo queryInfo) { + public Pair> runSnapshotQuery(QueryInfo queryInfo, Option snapshotLoadQuerySplitterOption) { LOG.info("Running snapshot query"); - return sparkSession.read().format("org.apache.hudi") - .option(DataSourceReadOptions.QUERY_TYPE().key(), queryInfo.getQueryType()).load(sourcePath) + Dataset snapshot = sparkSession.read().format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE().key(), queryInfo.getQueryType()).load(sourcePath); + QueryInfo snapshotQueryInfo = snapshotLoadQuerySplitterOption + .map(snapshotLoadQuerySplitter -> snapshotLoadQuerySplitter.getNextCheckpoint(snapshot, queryInfo)) + .orElse(queryInfo); + return Pair.of(snapshotQueryInfo, applySnapshotQueryFilters(snapshot, snapshotQueryInfo)); + } + + public Dataset applySnapshotQueryFilters(Dataset snapshot, QueryInfo snapshotQueryInfo) { + return snapshot // add filtering so that only interested records are returned. .filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, - queryInfo.getStartInstant())) + snapshotQueryInfo.getStartInstant())) .filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, - queryInfo.getEndInstant())); + snapshotQueryInfo.getEndInstant())); } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/gcs/PubsubMessagesFetcher.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/gcs/PubsubMessagesFetcher.java index 886b60cce7cc..3b574045d7aa 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/gcs/PubsubMessagesFetcher.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/gcs/PubsubMessagesFetcher.java @@ -20,21 +20,25 @@ import org.apache.hudi.exception.HoodieException; -import com.google.cloud.pubsub.v1.stub.GrpcSubscriberStub; import com.google.cloud.pubsub.v1.stub.SubscriberStub; import com.google.cloud.pubsub.v1.stub.SubscriberStubSettings; -import com.google.pubsub.v1.AcknowledgeRequest; import com.google.pubsub.v1.ProjectSubscriptionName; -import com.google.pubsub.v1.PullRequest; import com.google.pubsub.v1.PullResponse; import com.google.pubsub.v1.ReceivedMessage; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.ArrayList; import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.IntStream; -import static com.google.cloud.pubsub.v1.stub.GrpcSubscriberStub.create; import static org.apache.hudi.utilities.sources.helpers.gcs.GcsIngestionConfig.DEFAULT_MAX_INBOUND_MESSAGE_SIZE; /** @@ -42,18 +46,31 @@ */ public class PubsubMessagesFetcher { + private static final int DEFAULT_BATCH_SIZE_ACK_API = 10; + private static final long MAX_WAIT_TIME_TO_ACK_MESSAGES = TimeUnit.MINUTES.toMillis(1); + private static final int ACK_PRODUCER_THREAD_POOL_SIZE = 3; + + private final ExecutorService threadPool = Executors.newFixedThreadPool(ACK_PRODUCER_THREAD_POOL_SIZE); private final String googleProjectId; private final String pubsubSubscriptionId; private final int batchSize; + private final int maxMessagesPerSync; + private final long maxFetchTimePerSync; private final SubscriberStubSettings subscriberStubSettings; + private final PubsubQueueClient pubsubQueueClient; private static final Logger LOG = LoggerFactory.getLogger(PubsubMessagesFetcher.class); - public PubsubMessagesFetcher(String googleProjectId, String pubsubSubscriptionId, int batchSize) { + public PubsubMessagesFetcher(String googleProjectId, String pubsubSubscriptionId, int batchSize, + int maxMessagesPerSync, + long maxFetchTimePerSync, + PubsubQueueClient pubsubQueueClient) { this.googleProjectId = googleProjectId; this.pubsubSubscriptionId = pubsubSubscriptionId; this.batchSize = batchSize; + this.maxMessagesPerSync = maxMessagesPerSync; + this.maxFetchTimePerSync = maxFetchTimePerSync; try { /** For details of timeout and retry configs, @@ -69,49 +86,60 @@ public PubsubMessagesFetcher(String googleProjectId, String pubsubSubscriptionId } catch (IOException e) { throw new HoodieException("Error creating subscriber stub settings", e); } + this.pubsubQueueClient = pubsubQueueClient; + } + + public PubsubMessagesFetcher( + String googleProjectId, + String pubsubSubscriptionId, + int batchSize, + int maxMessagesPerSync, + long maxFetchTimePerSync) { + this( + googleProjectId, + pubsubSubscriptionId, + batchSize, + maxMessagesPerSync, + maxFetchTimePerSync, + new PubsubQueueClient() + ); } public List fetchMessages() { - try { - try (SubscriberStub subscriber = createSubscriber()) { - String subscriptionName = getSubscriptionName(); - PullResponse pullResponse = makePullRequest(subscriber, subscriptionName); - return pullResponse.getReceivedMessagesList(); + List messageList = new ArrayList<>(); + try (SubscriberStub subscriber = pubsubQueueClient.getSubscriber(subscriberStubSettings)) { + String subscriptionName = ProjectSubscriptionName.format(googleProjectId, pubsubSubscriptionId); + long startTime = System.currentTimeMillis(); + long unAckedMessages = pubsubQueueClient.getNumUnAckedMessages(this.pubsubSubscriptionId); + LOG.info("Found unacked messages " + unAckedMessages); + while (messageList.size() < unAckedMessages && messageList.size() < maxMessagesPerSync && (System.currentTimeMillis() - startTime < maxFetchTimePerSync)) { + PullResponse pullResponse = pubsubQueueClient.makePullRequest(subscriber, subscriptionName, batchSize); + messageList.addAll(pullResponse.getReceivedMessagesList()); } - } catch (IOException e) { + return messageList; + } catch (Exception e) { throw new HoodieException("Error when fetching metadata", e); } } public void sendAcks(List messagesToAck) throws IOException { - String subscriptionName = getSubscriptionName(); - try (SubscriberStub subscriber = createSubscriber()) { - - AcknowledgeRequest acknowledgeRequest = AcknowledgeRequest.newBuilder() - .setSubscription(subscriptionName) - .addAllAckIds(messagesToAck) - .build(); - - subscriber.acknowledgeCallable().call(acknowledgeRequest); - - LOG.info("Acknowledged messages: " + messagesToAck); + try (SubscriberStub subscriber = pubsubQueueClient.getSubscriber(subscriberStubSettings)) { + int numberOfBatches = (int) Math.ceil((double) messagesToAck.size() / DEFAULT_BATCH_SIZE_ACK_API); + CompletableFuture.allOf(IntStream.range(0, numberOfBatches) + .parallel() + .boxed() + .map(batchIndex -> getTask(subscriber, messagesToAck, batchIndex)).toArray(CompletableFuture[]::new)) + .get(MAX_WAIT_TIME_TO_ACK_MESSAGES, TimeUnit.MILLISECONDS); + LOG.debug("Flushed out all outstanding acknowledged messages: " + messagesToAck.size()); + } catch (ExecutionException | InterruptedException | TimeoutException e) { + throw new IOException("Failed to ack messages from PubSub", e); } } - private PullResponse makePullRequest(SubscriberStub subscriber, String subscriptionName) { - PullRequest pullRequest = PullRequest.newBuilder() - .setMaxMessages(batchSize) - .setSubscription(subscriptionName) - .build(); - - return subscriber.pullCallable().call(pullRequest); - } - - private GrpcSubscriberStub createSubscriber() throws IOException { - return create(subscriberStubSettings); - } - - private String getSubscriptionName() { - return ProjectSubscriptionName.format(googleProjectId, pubsubSubscriptionId); + private CompletableFuture getTask(SubscriberStub subscriber, List messagesToAck, int batchIndex) { + String subscriptionName = ProjectSubscriptionName.format(googleProjectId, pubsubSubscriptionId); + List messages = messagesToAck.subList(batchIndex, Math.min(batchIndex + DEFAULT_BATCH_SIZE_ACK_API, messagesToAck.size())); + return CompletableFuture.runAsync(() -> pubsubQueueClient.makeAckRequest(subscriber, subscriptionName, messages), threadPool); } } + diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/gcs/PubsubQueueClient.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/gcs/PubsubQueueClient.java new file mode 100644 index 000000000000..7f93d32b6068 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/gcs/PubsubQueueClient.java @@ -0,0 +1,80 @@ +/* + * 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.utilities.sources.helpers.gcs; + +import com.google.cloud.ServiceOptions; +import com.google.cloud.monitoring.v3.MetricServiceClient; +import com.google.cloud.pubsub.v1.stub.GrpcSubscriberStub; +import com.google.cloud.pubsub.v1.stub.SubscriberStub; +import com.google.cloud.pubsub.v1.stub.SubscriberStubSettings; +import com.google.monitoring.v3.ListTimeSeriesRequest; +import com.google.monitoring.v3.Point; +import com.google.monitoring.v3.ProjectName; +import com.google.monitoring.v3.TimeInterval; +import com.google.protobuf.util.Timestamps; +import com.google.pubsub.v1.AcknowledgeRequest; +import com.google.pubsub.v1.PullRequest; +import com.google.pubsub.v1.PullResponse; + +import java.io.IOException; +import java.time.Instant; +import java.util.List; +import java.util.concurrent.TimeUnit; + +public class PubsubQueueClient { + private static final String METRIC_FILTER_PATTERN = "metric.type=\"pubsub.googleapis.com/subscription/%s\" AND resource.label.subscription_id=\"%s\""; + private static final String NUM_UNDELIVERED_MESSAGES = "num_undelivered_messages"; + + public SubscriberStub getSubscriber(SubscriberStubSettings subscriberStubSettings) throws IOException { + return GrpcSubscriberStub.create(subscriberStubSettings); + } + + public PullResponse makePullRequest(SubscriberStub subscriber, String subscriptionName, int batchSize) throws IOException { + PullRequest pullRequest = PullRequest.newBuilder() + .setMaxMessages(batchSize) + .setSubscription(subscriptionName) + .build(); + return subscriber.pullCallable().call(pullRequest); + } + + public void makeAckRequest(SubscriberStub subscriber, String subscriptionName, List messages) { + AcknowledgeRequest acknowledgeRequest = AcknowledgeRequest.newBuilder() + .setSubscription(subscriptionName) + .addAllAckIds(messages) + .build(); + subscriber.acknowledgeCallable().call(acknowledgeRequest); + } + + public long getNumUnAckedMessages(String subscriptionId) throws IOException { + try (MetricServiceClient metricServiceClient = MetricServiceClient.create()) { + MetricServiceClient.ListTimeSeriesPagedResponse response = metricServiceClient.listTimeSeries( + ListTimeSeriesRequest.newBuilder() + .setName(ProjectName.of(ServiceOptions.getDefaultProjectId()).toString()) + .setFilter(String.format(METRIC_FILTER_PATTERN, NUM_UNDELIVERED_MESSAGES, subscriptionId)) + .setInterval(TimeInterval.newBuilder() + .setStartTime(Timestamps.fromSeconds(Instant.now().getEpochSecond() - TimeUnit.MINUTES.toSeconds(2))) + .setEndTime(Timestamps.fromSeconds(Instant.now().getEpochSecond())) + .build()) + .build()); + // use the latest value from the window + List pointList = response.getPage().getValues().iterator().next().getPointsList(); + return pointList.stream().findFirst().map(point -> point.getValue().getInt64Value()).orElse(Long.MAX_VALUE); + } + } +} \ No newline at end of file diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/ErrorTableUtils.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/ErrorTableUtils.java index 694990cf1fa0..8907a1b66478 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/ErrorTableUtils.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/ErrorTableUtils.java @@ -64,7 +64,7 @@ public static Option getErrorTableWriter(HoodieStreamer.Co public static HoodieErrorTableConfig.ErrorWriteFailureStrategy getErrorWriteFailureStrategy( TypedProperties props) { - String writeFailureStrategy = props.getString(ERROR_TABLE_WRITE_FAILURE_STRATEGY.key()); + String writeFailureStrategy = props.getString(ERROR_TABLE_WRITE_FAILURE_STRATEGY.key(), ERROR_TABLE_WRITE_FAILURE_STRATEGY.defaultValue()); return HoodieErrorTableConfig.ErrorWriteFailureStrategy.valueOf(writeFailureStrategy); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java index 5604a6240c71..264b6a5cc0e2 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java @@ -747,6 +747,8 @@ protected Pair startService() { while (!isShutdownRequested()) { try { long start = System.currentTimeMillis(); + // Send a heartbeat metrics event to track the active ingestion job for this table. + streamSync.getMetrics().updateStreamerHeartbeatTimestamp(start); // check if deltastreamer need to update the configuration before the sync if (configurationHotUpdateStrategyOpt.isPresent()) { Option newProps = configurationHotUpdateStrategyOpt.get().updateProperties(props); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamerUtils.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamerUtils.java index ad1de230f414..a6f9513a14e3 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamerUtils.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamerUtils.java @@ -70,63 +70,63 @@ public class HoodieStreamerUtils { * Takes care of dropping columns, precombine, auto key generation. * Both AVRO and SPARK record types are supported. */ - static JavaRDD createHoodieRecords(HoodieStreamer.Config cfg, TypedProperties props, Option> avroRDDOptional, + static Option> createHoodieRecords(HoodieStreamer.Config cfg, TypedProperties props, Option> avroRDDOptional, SchemaProvider schemaProvider, HoodieRecord.HoodieRecordType recordType, boolean autoGenerateRecordKeys, String instantTime) { boolean shouldCombine = cfg.filterDupes || cfg.operation.equals(WriteOperationType.UPSERT); Set partitionColumns = getPartitionColumns(props); - JavaRDD avroRDD = avroRDDOptional.get(); + return avroRDDOptional.map(avroRDD -> { + JavaRDD records; + SerializableSchema avroSchema = new SerializableSchema(schemaProvider.getTargetSchema()); + SerializableSchema processedAvroSchema = new SerializableSchema(isDropPartitionColumns(props) ? HoodieAvroUtils.removeMetadataFields(avroSchema.get()) : avroSchema.get()); + if (recordType == HoodieRecord.HoodieRecordType.AVRO) { + records = avroRDD.mapPartitions( + (FlatMapFunction, HoodieRecord>) genericRecordIterator -> { + if (autoGenerateRecordKeys) { + props.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG, String.valueOf(TaskContext.getPartitionId())); + props.setProperty(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG, instantTime); + } + BuiltinKeyGenerator builtinKeyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); + List avroRecords = new ArrayList<>(); + while (genericRecordIterator.hasNext()) { + GenericRecord genRec = genericRecordIterator.next(); + HoodieKey hoodieKey = new HoodieKey(builtinKeyGenerator.getRecordKey(genRec), builtinKeyGenerator.getPartitionPath(genRec)); + GenericRecord gr = isDropPartitionColumns(props) ? HoodieAvroUtils.removeFields(genRec, partitionColumns) : genRec; + HoodieRecordPayload payload = shouldCombine ? DataSourceUtils.createPayload(cfg.payloadClassName, gr, + (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, cfg.sourceOrderingField, false, props.getBoolean( + KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), + Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue())))) + : DataSourceUtils.createPayload(cfg.payloadClassName, gr); + avroRecords.add(new HoodieAvroRecord<>(hoodieKey, payload)); + } + return avroRecords.iterator(); + }); + } else if (recordType == HoodieRecord.HoodieRecordType.SPARK) { + // TODO we should remove it if we can read InternalRow from source. + records = avroRDD.mapPartitions(itr -> { + if (autoGenerateRecordKeys) { + props.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG, String.valueOf(TaskContext.getPartitionId())); + props.setProperty(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG, instantTime); + } + BuiltinKeyGenerator builtinKeyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); + StructType baseStructType = AvroConversionUtils.convertAvroSchemaToStructType(processedAvroSchema.get()); + StructType targetStructType = isDropPartitionColumns(props) ? AvroConversionUtils + .convertAvroSchemaToStructType(HoodieAvroUtils.removeFields(processedAvroSchema.get(), partitionColumns)) : baseStructType; + HoodieAvroDeserializer deserializer = SparkAdapterSupport$.MODULE$.sparkAdapter().createAvroDeserializer(processedAvroSchema.get(), baseStructType); - JavaRDD records; - SerializableSchema avroSchema = new SerializableSchema(schemaProvider.getTargetSchema()); - SerializableSchema processedAvroSchema = new SerializableSchema(isDropPartitionColumns(props) ? HoodieAvroUtils.removeMetadataFields(avroSchema.get()) : avroSchema.get()); - if (recordType == HoodieRecord.HoodieRecordType.AVRO) { - records = avroRDD.mapPartitions( - (FlatMapFunction, HoodieRecord>) genericRecordIterator -> { - if (autoGenerateRecordKeys) { - props.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG, String.valueOf(TaskContext.getPartitionId())); - props.setProperty(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG, instantTime); - } - BuiltinKeyGenerator builtinKeyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); - List avroRecords = new ArrayList<>(); - while (genericRecordIterator.hasNext()) { - GenericRecord genRec = genericRecordIterator.next(); - HoodieKey hoodieKey = new HoodieKey(builtinKeyGenerator.getRecordKey(genRec), builtinKeyGenerator.getPartitionPath(genRec)); - GenericRecord gr = isDropPartitionColumns(props) ? HoodieAvroUtils.removeFields(genRec, partitionColumns) : genRec; - HoodieRecordPayload payload = shouldCombine ? DataSourceUtils.createPayload(cfg.payloadClassName, gr, - (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, cfg.sourceOrderingField, false, props.getBoolean( - KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), - Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue())))) - : DataSourceUtils.createPayload(cfg.payloadClassName, gr); - avroRecords.add(new HoodieAvroRecord<>(hoodieKey, payload)); - } - return avroRecords.iterator(); + return new CloseableMappingIterator<>(ClosableIterator.wrap(itr), rec -> { + InternalRow row = (InternalRow) deserializer.deserialize(rec).get(); + String recordKey = builtinKeyGenerator.getRecordKey(row, baseStructType).toString(); + String partitionPath = builtinKeyGenerator.getPartitionPath(row, baseStructType).toString(); + return new HoodieSparkRecord(new HoodieKey(recordKey, partitionPath), + HoodieInternalRowUtils.getCachedUnsafeProjection(baseStructType, targetStructType).apply(row), targetStructType, false); }); - } else if (recordType == HoodieRecord.HoodieRecordType.SPARK) { - // TODO we should remove it if we can read InternalRow from source. - records = avroRDD.mapPartitions(itr -> { - if (autoGenerateRecordKeys) { - props.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG, String.valueOf(TaskContext.getPartitionId())); - props.setProperty(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG, instantTime); - } - BuiltinKeyGenerator builtinKeyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); - StructType baseStructType = AvroConversionUtils.convertAvroSchemaToStructType(processedAvroSchema.get()); - StructType targetStructType = isDropPartitionColumns(props) ? AvroConversionUtils - .convertAvroSchemaToStructType(HoodieAvroUtils.removeFields(processedAvroSchema.get(), partitionColumns)) : baseStructType; - HoodieAvroDeserializer deserializer = SparkAdapterSupport$.MODULE$.sparkAdapter().createAvroDeserializer(processedAvroSchema.get(), baseStructType); - - return new CloseableMappingIterator<>(ClosableIterator.wrap(itr), rec -> { - InternalRow row = (InternalRow) deserializer.deserialize(rec).get(); - String recordKey = builtinKeyGenerator.getRecordKey(row, baseStructType).toString(); - String partitionPath = builtinKeyGenerator.getPartitionPath(row, baseStructType).toString(); - return new HoodieSparkRecord(new HoodieKey(recordKey, partitionPath), - HoodieInternalRowUtils.getCachedUnsafeProjection(baseStructType, targetStructType).apply(row), targetStructType, false); }); - }); - } else { - throw new UnsupportedOperationException(recordType.name()); - } - return records; + } else { + throw new UnsupportedOperationException(recordType.name()); + } + return records; + }); } /** diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SourceFormatAdapter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SourceFormatAdapter.java index 9f1b087900d9..f29404701db9 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SourceFormatAdapter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SourceFormatAdapter.java @@ -229,8 +229,11 @@ public InputBatch> fetchNewDataInRowFormat(Option lastCkptS // configured via this option. The column is then used to trigger error events. StructType dataType = AvroConversionUtils.convertAvroSchemaToStructType(sourceSchema) .add(new StructField(ERROR_TABLE_CURRUPT_RECORD_COL_NAME, DataTypes.StringType, true, Metadata.empty())); + StructType nullableStruct = dataType.asNullable(); Option> dataset = r.getBatch().map(rdd -> source.getSparkSession().read() - .option("columnNameOfCorruptRecord", ERROR_TABLE_CURRUPT_RECORD_COL_NAME).schema(dataType.asNullable()) + .option("columnNameOfCorruptRecord", ERROR_TABLE_CURRUPT_RECORD_COL_NAME) + .schema(nullableStruct) + .option("mode", "PERMISSIVE") .json(rdd)); Option> eventsDataset = processErrorEvents(dataset, ErrorEvent.ErrorReason.JSON_ROW_DESERIALIZATION_FAILURE); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java index 6c87f53a5652..0fd7a41ab556 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java @@ -64,7 +64,7 @@ public class SparkSampleWritesUtils { private static final Logger LOG = LoggerFactory.getLogger(SparkSampleWritesUtils.class); - public static Option getWriteConfigWithRecordSizeEstimate(JavaSparkContext jsc, JavaRDD records, HoodieWriteConfig writeConfig) { + public static Option getWriteConfigWithRecordSizeEstimate(JavaSparkContext jsc, Option> recordsOpt, HoodieWriteConfig writeConfig) { if (!writeConfig.getBoolean(SAMPLE_WRITES_ENABLED)) { LOG.debug("Skip overwriting record size estimate as it's disabled."); return Option.empty(); @@ -76,7 +76,7 @@ public static Option getWriteConfigWithRecordSizeEstimate(Jav } try { String instantTime = getInstantFromTemporalAccessor(Instant.now().atZone(ZoneId.systemDefault())); - Pair result = doSampleWrites(jsc, records, writeConfig, instantTime); + Pair result = doSampleWrites(jsc, recordsOpt, writeConfig, instantTime); if (result.getLeft()) { long avgSize = getAvgSizeFromSampleWrites(jsc, result.getRight()); LOG.info("Overwriting record size estimate to " + avgSize); @@ -90,7 +90,7 @@ public static Option getWriteConfigWithRecordSizeEstimate(Jav return Option.empty(); } - private static Pair doSampleWrites(JavaSparkContext jsc, JavaRDD records, HoodieWriteConfig writeConfig, String instantTime) + private static Pair doSampleWrites(JavaSparkContext jsc, Option> recordsOpt, HoodieWriteConfig writeConfig, String instantTime) throws IOException { final String sampleWritesBasePath = getSampleWritesBasePath(jsc, writeConfig, instantTime); HoodieTableMetaClient.withPropertyBuilder() @@ -109,25 +109,31 @@ private static Pair doSampleWrites(JavaSparkContext jsc, JavaRD .withAutoCommit(true) .withPath(sampleWritesBasePath) .build(); + Pair emptyRes = Pair.of(false, null); try (SparkRDDWriteClient sampleWriteClient = new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), sampleWriteConfig, Option.empty())) { int size = writeConfig.getIntOrDefault(SAMPLE_WRITES_SIZE); - List samples = records.coalesce(1).take(size); - sampleWriteClient.startCommitWithTime(instantTime); - JavaRDD writeStatusRDD = sampleWriteClient.bulkInsert(jsc.parallelize(samples, 1), instantTime); - if (writeStatusRDD.filter(WriteStatus::hasErrors).count() > 0) { - LOG.error(String.format("sample writes for table %s failed with errors.", writeConfig.getTableName())); - if (LOG.isTraceEnabled()) { - LOG.trace("Printing out the top 100 errors"); - writeStatusRDD.filter(WriteStatus::hasErrors).take(100).forEach(ws -> { - LOG.trace("Global error :", ws.getGlobalError()); - ws.getErrors().forEach((key, throwable) -> - LOG.trace(String.format("Error for key: %s", key), throwable)); - }); + return recordsOpt.map(records -> { + List samples = records.coalesce(1).take(size); + if (samples.isEmpty()) { + return emptyRes; } - return Pair.of(false, null); - } else { - return Pair.of(true, sampleWritesBasePath); - } + sampleWriteClient.startCommitWithTime(instantTime); + JavaRDD writeStatusRDD = sampleWriteClient.bulkInsert(jsc.parallelize(samples, 1), instantTime); + if (writeStatusRDD.filter(WriteStatus::hasErrors).count() > 0) { + LOG.error(String.format("sample writes for table %s failed with errors.", writeConfig.getTableName())); + if (LOG.isTraceEnabled()) { + LOG.trace("Printing out the top 100 errors"); + writeStatusRDD.filter(WriteStatus::hasErrors).take(100).forEach(ws -> { + LOG.trace("Global error :", ws.getGlobalError()); + ws.getErrors().forEach((key, throwable) -> + LOG.trace(String.format("Error for key: %s", key), throwable)); + }); + } + return emptyRes; + } else { + return Pair.of(true, sampleWritesBasePath); + } + }).orElse(emptyRes); } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java index 6626084aa2bd..19289e650c4e 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java @@ -119,12 +119,11 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; import java.util.function.Function; +import java.util.stream.Collectors; import scala.Tuple2; import scala.collection.JavaConversions; @@ -160,6 +159,7 @@ public class StreamSync implements Serializable, Closeable { private static final long serialVersionUID = 1L; private static final Logger LOG = LoggerFactory.getLogger(StreamSync.class); + private static final String NULL_PLACEHOLDER = "[null]"; /** * Delta Sync Config. @@ -402,33 +402,32 @@ public Pair, JavaRDD> syncOnce() throws IOException .build(); String instantTime = metaClient.createNewInstantTime(); - Pair inputBatchIsEmptyPair = readFromSource(instantTime, metaClient); + InputBatch inputBatch = readFromSource(instantTime, metaClient); - if (inputBatchIsEmptyPair != null) { - final JavaRDD recordsFromSource; - if (useRowWriter) { - recordsFromSource = hoodieSparkContext.emptyRDD(); - } else { - recordsFromSource = (JavaRDD) inputBatchIsEmptyPair.getKey().getBatch().get(); - } + if (inputBatch != null) { // this is the first input batch. If schemaProvider not set, use it and register Avro Schema and start // compactor if (writeClient == null) { - this.schemaProvider = inputBatchIsEmptyPair.getKey().getSchemaProvider(); + this.schemaProvider = inputBatch.getSchemaProvider(); // Setup HoodieWriteClient and compaction now that we decided on schema - setupWriteClient(recordsFromSource); + setupWriteClient(inputBatch.getBatch()); } else { - Schema newSourceSchema = inputBatchIsEmptyPair.getKey().getSchemaProvider().getSourceSchema(); - Schema newTargetSchema = inputBatchIsEmptyPair.getKey().getSchemaProvider().getTargetSchema(); - if (!(processedSchema.isSchemaPresent(newSourceSchema)) - || !(processedSchema.isSchemaPresent(newTargetSchema))) { - LOG.info("Seeing new schema. Source :" + newSourceSchema.toString(true) - + ", Target :" + newTargetSchema.toString(true)); + Schema newSourceSchema = inputBatch.getSchemaProvider().getSourceSchema(); + Schema newTargetSchema = inputBatch.getSchemaProvider().getTargetSchema(); + if ((newSourceSchema != null && !processedSchema.isSchemaPresent(newSourceSchema)) + || (newTargetSchema != null && !processedSchema.isSchemaPresent(newTargetSchema))) { + String sourceStr = newSourceSchema == null ? NULL_PLACEHOLDER : newSourceSchema.toString(true); + String targetStr = newTargetSchema == null ? NULL_PLACEHOLDER : newTargetSchema.toString(true); + LOG.info("Seeing new schema. Source: {0}, Target: {1}", sourceStr, targetStr); // We need to recreate write client with new schema and register them. - reInitWriteClient(newSourceSchema, newTargetSchema, recordsFromSource); - processedSchema.addSchema(newSourceSchema); - processedSchema.addSchema(newTargetSchema); + reInitWriteClient(newSourceSchema, newTargetSchema, inputBatch.getBatch()); + if (newSourceSchema != null) { + processedSchema.addSchema(newSourceSchema); + } + if (newTargetSchema != null) { + processedSchema.addSchema(newTargetSchema); + } } } @@ -449,7 +448,7 @@ public Pair, JavaRDD> syncOnce() throws IOException } } - result = writeToSinkAndDoMetaSync(instantTime, inputBatchIsEmptyPair.getKey(), inputBatchIsEmptyPair.getValue(), metrics, overallTimerContext); + result = writeToSinkAndDoMetaSync(instantTime, inputBatch, metrics, overallTimerContext); } metrics.updateStreamerSyncMetrics(System.currentTimeMillis()); @@ -479,7 +478,7 @@ private Option getLastPendingCompactionInstant(Option co * @throws Exception in case of any Exception */ - public Pair readFromSource(String instantTime, HoodieTableMetaClient metaClient) throws IOException { + public InputBatch readFromSource(String instantTime, HoodieTableMetaClient metaClient) throws IOException { // Retrieve the previous round checkpoints, if any Option resumeCheckpointStr = Option.empty(); if (commitsTimelineOpt.isPresent()) { @@ -494,7 +493,7 @@ public Pair readFromSource(String instantTime, HoodieTableM int maxRetryCount = cfg.retryOnSourceFailures ? cfg.maxRetryCount : 1; int curRetryCount = 0; - Pair sourceDataToSync = null; + InputBatch sourceDataToSync = null; while (curRetryCount++ < maxRetryCount && sourceDataToSync == null) { try { sourceDataToSync = fetchFromSourceAndPrepareRecords(resumeCheckpointStr, instantTime, metaClient); @@ -514,7 +513,7 @@ public Pair readFromSource(String instantTime, HoodieTableM return sourceDataToSync; } - private Pair fetchFromSourceAndPrepareRecords(Option resumeCheckpointStr, String instantTime, + private InputBatch fetchFromSourceAndPrepareRecords(Option resumeCheckpointStr, String instantTime, HoodieTableMetaClient metaClient) { HoodieRecordType recordType = createRecordMerger(props).getRecordType(); if (recordType == HoodieRecordType.SPARK && HoodieTableType.valueOf(cfg.tableType) == HoodieTableType.MERGE_ON_READ @@ -539,17 +538,14 @@ private Pair fetchFromSourceAndPrepareRecords(Option preparedInputBatchIsEmptyPair = handleEmptyBatch(useRowWriter, inputBatch, checkpointStr, schemaProvider); - if (preparedInputBatchIsEmptyPair.getValue()) { // return if empty batch - return preparedInputBatchIsEmptyPair; - } + if (useRowWriter) { // no additional processing required for row writer. - return Pair.of(inputBatch, false); + return inputBatch; } else { - JavaRDD records = HoodieStreamerUtils.createHoodieRecords(cfg, props, inputBatch.getBatch(), schemaProvider, + Option> recordsOpt = HoodieStreamerUtils.createHoodieRecords(cfg, props, inputBatch.getBatch(), schemaProvider, recordType, autoGenerateRecordKeys, instantTime); - return Pair.of(new InputBatch(Option.of(records), checkpointStr, schemaProvider), false); + return new InputBatch(recordsOpt, checkpointStr, schemaProvider); } } @@ -577,7 +573,8 @@ private InputBatch fetchNextBatchFromSource(Option resumeCheckpointStr, ErrorEvent.ErrorReason.CUSTOM_TRANSFORMER_FAILURE); checkpointStr = dataAndCheckpoint.getCheckpointForNextBatch(); - if (this.userProvidedSchemaProvider != null && this.userProvidedSchemaProvider.getTargetSchema() != null) { + if (this.userProvidedSchemaProvider != null && this.userProvidedSchemaProvider.getTargetSchema() != null + && this.userProvidedSchemaProvider.getTargetSchema() != InputBatch.NULL_SCHEMA) { if (useRowWriter) { inputBatchForWriter = new InputBatch(transformed, checkpointStr, this.userProvidedSchemaProvider); } else { @@ -646,33 +643,6 @@ private InputBatch fetchNextBatchFromSource(Option resumeCheckpointStr, } } - /** - * Handles empty batch from input. - * @param useRowWriter true if row write code path. - * @param inputBatch {@link InputBatch} instance to use. - * @param checkpointForNextBatch checkpiont to use for next batch. - * @param schemaProvider {@link SchemaProvider} instance of interest. - * @return a Pair of InputBatch and boolean. boolean value is set to true on empty batch. - */ - private Pair handleEmptyBatch(boolean useRowWriter, InputBatch inputBatch, - String checkpointForNextBatch, SchemaProvider schemaProvider) { - hoodieSparkContext.setJobStatus(this.getClass().getSimpleName(), "Checking if input is empty"); - if (useRowWriter) { - Option> rowDatasetOptional = inputBatch.getBatch(); - if ((!rowDatasetOptional.isPresent()) || (rowDatasetOptional.get().isEmpty())) { - LOG.info("No new data, perform empty commit."); - return Pair.of(new InputBatch<>(Option.of(sparkSession.emptyDataFrame()), checkpointForNextBatch, schemaProvider), true); - } - } else { - Option> avroRDDOptional = inputBatch.getBatch(); - if ((!avroRDDOptional.isPresent()) || (avroRDDOptional.get().isEmpty())) { - LOG.info("No new data, perform empty commit."); - return Pair.of(new InputBatch(Option.of(hoodieSparkContext.emptyRDD()), checkpointForNextBatch, schemaProvider), true); - } - } - return Pair.of(inputBatch, false); - } - /** * Apply schema reconcile and schema evolution rules(schema on read) and generate new target schema provider. * @@ -795,24 +765,28 @@ private HoodieWriteConfig prepareHoodieConfigForRowWriter(Schema writerSchema) { * * @param instantTime instant time to use for ingest. * @param inputBatch input batch that contains the records, checkpoint, and schema provider - * @param inputIsEmpty true if input batch is empty. * @param metrics Metrics * @param overallTimerContext Timer Context * @return Option Compaction instant if one is scheduled */ - private Pair, JavaRDD> writeToSinkAndDoMetaSync(String instantTime, InputBatch inputBatch, boolean inputIsEmpty, + private Pair, JavaRDD> writeToSinkAndDoMetaSync(String instantTime, InputBatch inputBatch, HoodieIngestionMetrics metrics, Timer.Context overallTimerContext) { Option scheduledCompactionInstant = Option.empty(); // write to hudi and fetch result - Pair writeClientWriteResultIsEmptyPair = writeToSink(inputBatch, instantTime, inputIsEmpty); - JavaRDD writeStatusRDD = writeClientWriteResultIsEmptyPair.getKey().getWriteStatusRDD(); - Map> partitionToReplacedFileIds = writeClientWriteResultIsEmptyPair.getKey().getPartitionToReplacedFileIds(); - boolean isEmpty = writeClientWriteResultIsEmptyPair.getRight(); + WriteClientWriteResult writeClientWriteResult = writeToSink(inputBatch, instantTime); + JavaRDD writeStatusRDD = writeClientWriteResult.getWriteStatusRDD(); + Map> partitionToReplacedFileIds = writeClientWriteResult.getPartitionToReplacedFileIds(); // process write status long totalErrorRecords = writeStatusRDD.mapToDouble(WriteStatus::getTotalErrorRecords).sum().longValue(); long totalRecords = writeStatusRDD.mapToDouble(WriteStatus::getTotalRecords).sum().longValue(); + long totalSuccessfulRecords = totalRecords - totalErrorRecords; + LOG.info(String.format("instantTime=%s, totalRecords=%d, totalErrorRecords=%d, totalSuccessfulRecords=%d", + instantTime, totalRecords, totalErrorRecords, totalSuccessfulRecords)); + if (totalRecords == 0) { + LOG.info("No new data, perform empty commit."); + } boolean hasErrors = totalErrorRecords > 0; if (!hasErrors || cfg.commitOnErrors) { HashMap checkpointCommitMetadata = new HashMap<>(); @@ -857,8 +831,10 @@ private Pair, JavaRDD> writeToSinkAndDoMetaSync(Stri scheduledCompactionInstant = writeClient.scheduleCompaction(Option.empty()); } - if (!isEmpty || cfg.forceEmptyMetaSync) { + if ((totalSuccessfulRecords > 0) || cfg.forceEmptyMetaSync) { runMetaSync(); + } else { + LOG.info(String.format("Not running metaSync totalSuccessfulRecords=%d", totalSuccessfulRecords)); } } else { LOG.info("Commit " + instantTime + " failed!"); @@ -918,22 +894,20 @@ private String startCommit(String instantTime, boolean retryEnabled) { throw lastException; } - private Pair writeToSink(InputBatch inputBatch, String instantTime, boolean inputIsEmpty) { + private WriteClientWriteResult writeToSink(InputBatch inputBatch, String instantTime) { WriteClientWriteResult writeClientWriteResult = null; instantTime = startCommit(instantTime, !autoGenerateRecordKeys); - boolean isEmpty = inputIsEmpty; if (useRowWriter) { - Dataset df = (Dataset) inputBatch.getBatch().get(); + Dataset df = (Dataset) inputBatch.getBatch().orElse(hoodieSparkContext.emptyRDD()); HoodieWriteConfig hoodieWriteConfig = prepareHoodieConfigForRowWriter(inputBatch.getSchemaProvider().getTargetSchema()); BaseDatasetBulkInsertCommitActionExecutor executor = new HoodieStreamerDatasetBulkInsertCommitActionExecutor(hoodieWriteConfig, writeClient, instantTime); writeClientWriteResult = new WriteClientWriteResult(executor.execute(df, !HoodieStreamerUtils.getPartitionColumns(props).isEmpty()).getWriteStatuses()); } else { - JavaRDD records = (JavaRDD) inputBatch.getBatch().get(); + JavaRDD records = (JavaRDD) inputBatch.getBatch().orElse(hoodieSparkContext.emptyRDD()); // filter dupes if needed if (cfg.filterDupes) { records = DataSourceUtils.dropDuplicates(hoodieSparkContext.jsc(), records, writeClient.getConfig()); - isEmpty = records.isEmpty(); } HoodieWriteResult writeResult = null; @@ -967,7 +941,7 @@ private Pair writeToSink(InputBatch inputBatch, throw new HoodieStreamerException("Unknown operation : " + cfg.operation); } } - return Pair.of(writeClientWriteResult, isEmpty); + return writeClientWriteResult; } private String getSyncClassShortName(String syncClassName) { @@ -975,7 +949,7 @@ private String getSyncClassShortName(String syncClassName) { } public void runMetaSync() { - Set syncClientToolClasses = new HashSet<>(Arrays.asList(cfg.syncClientToolClassNames.split(","))); + List syncClientToolClasses = Arrays.stream(cfg.syncClientToolClassNames.split(",")).distinct().collect(Collectors.toList()); // for backward compatibility if (cfg.enableHiveSync) { cfg.enableMetaSync = true; @@ -983,6 +957,7 @@ public void runMetaSync() { LOG.info("When set --enable-hive-sync will use HiveSyncTool for backward compatibility"); } if (cfg.enableMetaSync) { + LOG.debug("[MetaSync] Starting sync"); FileSystem fs = FSUtils.getFs(cfg.targetBasePath, hoodieSparkContext.hadoopConfiguration()); TypedProperties metaProps = new TypedProperties(); @@ -996,14 +971,19 @@ public void runMetaSync() { Map failedMetaSyncs = new HashMap<>(); for (String impl : syncClientToolClasses) { Timer.Context syncContext = metrics.getMetaSyncTimerContext(); + boolean success = false; try { SyncUtilHelpers.runHoodieMetaSync(impl.trim(), metaProps, conf, fs, cfg.targetBasePath, cfg.baseFileFormat); + success = true; } catch (HoodieMetaSyncException e) { - LOG.warn("SyncTool class " + impl.trim() + " failed with exception", e); + LOG.error("SyncTool class {0} failed with exception {1}", impl.trim(), e); failedMetaSyncs.put(impl, e); } long metaSyncTimeMs = syncContext != null ? syncContext.stop() : 0; metrics.updateStreamerMetaSyncMetrics(getSyncClassShortName(impl), metaSyncTimeMs); + if (success) { + LOG.info("[MetaSync] SyncTool class {0} completed successfully and took {1} ", impl.trim(), metaSyncTimeMs); + } } if (!failedMetaSyncs.isEmpty()) { throw getHoodieMetaSyncException(failedMetaSyncs); @@ -1016,15 +996,15 @@ public void runMetaSync() { * SchemaProvider creation is a precursor to HoodieWriteClient and AsyncCompactor creation. This method takes care of * this constraint. */ - private void setupWriteClient(JavaRDD records) throws IOException { + private void setupWriteClient(Option> recordsOpt) throws IOException { if ((null != schemaProvider)) { Schema sourceSchema = schemaProvider.getSourceSchema(); Schema targetSchema = schemaProvider.getTargetSchema(); - reInitWriteClient(sourceSchema, targetSchema, records); + reInitWriteClient(sourceSchema, targetSchema, recordsOpt); } } - private void reInitWriteClient(Schema sourceSchema, Schema targetSchema, JavaRDD records) throws IOException { + private void reInitWriteClient(Schema sourceSchema, Schema targetSchema, Option> recordsOpt) throws IOException { LOG.info("Setting up new Hoodie Write Client"); if (HoodieStreamerUtils.isDropPartitionColumns(props)) { targetSchema = HoodieAvroUtils.removeFields(targetSchema, HoodieStreamerUtils.getPartitionColumns(props)); @@ -1032,7 +1012,7 @@ private void reInitWriteClient(Schema sourceSchema, Schema targetSchema, JavaRDD registerAvroSchemas(sourceSchema, targetSchema); final HoodieWriteConfig initialWriteConfig = getHoodieClientConfig(targetSchema); final HoodieWriteConfig writeConfig = SparkSampleWritesUtils - .getWriteConfigWithRecordSizeEstimate(hoodieSparkContext.jsc(), records, initialWriteConfig) + .getWriteConfigWithRecordSizeEstimate(hoodieSparkContext.jsc(), recordsOpt, initialWriteConfig) .orElse(initialWriteConfig); if (writeConfig.isEmbeddedTimelineServerEnabled()) { @@ -1141,10 +1121,11 @@ private Schema getSchemaForWriteConfig(Schema targetSchema) { .build(); int totalCompleted = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants(); if (totalCompleted > 0) { - try { - TableSchemaResolver schemaResolver = new TableSchemaResolver(meta); - newWriteSchema = schemaResolver.getTableAvroSchema(false); - } catch (IllegalArgumentException e) { + TableSchemaResolver schemaResolver = new TableSchemaResolver(meta); + Option tableSchema = schemaResolver.getTableAvroSchemaIfPresent(false); + if (tableSchema.isPresent()) { + newWriteSchema = tableSchema.get(); + } else { LOG.warn("Could not fetch schema from table. Falling back to using target schema from schema provider"); } } @@ -1175,13 +1156,14 @@ private void registerAvroSchemas(SchemaProvider schemaProvider) { */ private void registerAvroSchemas(Schema sourceSchema, Schema targetSchema) { // register the schemas, so that shuffle does not serialize the full schemas - if (null != sourceSchema) { - List schemas = new ArrayList<>(); + List schemas = new ArrayList<>(); + if (sourceSchema != null) { schemas.add(sourceSchema); - if (targetSchema != null) { - schemas.add(targetSchema); - } - + } + if (targetSchema != null) { + schemas.add(targetSchema); + } + if (!schemas.isEmpty()) { if (LOG.isDebugEnabled()) { LOG.debug("Registering Schema: " + schemas); } @@ -1205,7 +1187,7 @@ public void close() { LOG.info("Shutting down embedded timeline server"); if (embeddedTimelineService.isPresent()) { - embeddedTimelineService.get().stop(); + embeddedTimelineService.get().stopForBasePath(cfg.targetBasePath); } if (metrics != null) { diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java index 967ad4521826..62aa7328fbbb 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java @@ -379,7 +379,7 @@ public void testKafkaConnectCheckpointProvider() throws IOException { @Test public void testPropsWithInvalidKeyGenerator() { - Exception e = assertThrows(IllegalArgumentException.class, () -> { + Exception e = assertThrows(IOException.class, () -> { String tableBasePath = basePath + "/test_table_invalid_key_gen"; HoodieDeltaStreamer deltaStreamer = new HoodieDeltaStreamer(TestHelpers.makeConfig(tableBasePath, WriteOperationType.BULK_INSERT, @@ -387,8 +387,8 @@ public void testPropsWithInvalidKeyGenerator() { deltaStreamer.sync(); }, "Should error out when setting the key generator class property to an invalid value"); // expected - LOG.debug("Expected error during getting the key generator", e); - assertTrue(e.getMessage().contains("No KeyGeneratorType found for class name")); + LOG.warn("Expected error during getting the key generator", e); + assertTrue(e.getMessage().contains("Could not load key generator class invalid")); } private static Stream provideInferKeyGenArgs() { @@ -531,7 +531,6 @@ public void testModifiedTableConfigs() throws Exception { List counts = countsPerCommit(tableBasePath, sqlContext); assertEquals(1000, counts.stream().mapToLong(entry -> entry.getLong(1)).sum()); - //perform the upsert and now with the original config, the commit should go through HoodieDeltaStreamer.Config newCfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.BULK_INSERT); newCfg.sourceLimit = 2000; @@ -1378,7 +1377,10 @@ private void testBulkInsertRowWriterMultiBatches(Boolean useSchemaProvider, List if (i == 2 || i == 4) { // this validation reloads the timeline. So, we are validating only for first and last batch. // validate commit metadata for all completed commits to have valid schema in extra metadata. HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(tableBasePath).setConf(jsc.hadoopConfiguration()).build(); - metaClient.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants().getInstants().forEach(entry -> assertValidSchemaInCommitMetadata(entry, metaClient)); + metaClient.reloadActiveTimeline().getCommitsTimeline() + .filterCompletedInstants().getInstants() + .forEach(entry -> assertValidSchemaAndOperationTypeInCommitMetadata( + entry, metaClient, WriteOperationType.BULK_INSERT)); } } } finally { @@ -1744,15 +1746,21 @@ private void testParquetDFSSource(boolean useSchemaProvider, List transf assertRecordCount(parquetRecordsCount + 100, tableBasePath, sqlContext); // validate commit metadata for all completed commits to have valid schema in extra metadata. HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(tableBasePath).setConf(jsc.hadoopConfiguration()).build(); - metaClient.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants().getInstants().forEach(entry -> assertValidSchemaInCommitMetadata(entry, metaClient)); + metaClient.reloadActiveTimeline().getCommitsTimeline() + .filterCompletedInstants().getInstants() + .forEach(entry -> assertValidSchemaAndOperationTypeInCommitMetadata( + entry, metaClient, WriteOperationType.INSERT)); testNum++; } - private void assertValidSchemaInCommitMetadata(HoodieInstant instant, HoodieTableMetaClient metaClient) { + private void assertValidSchemaAndOperationTypeInCommitMetadata(HoodieInstant instant, + HoodieTableMetaClient metaClient, + WriteOperationType operationType) { try { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata .fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); assertFalse(StringUtils.isNullOrEmpty(commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY))); + assertEquals(operationType, commitMetadata.getOperationType()); } catch (IOException ioException) { throw new HoodieException("Failed to parse commit metadata for " + instant.toString()); } @@ -2358,7 +2366,7 @@ public void testHoodieIncrFallback() throws Exception { // Remove source.hoodieincr.num_instants config downstreamCfg.configs.remove(downstreamCfg.configs.size() - 1); - downstreamCfg.configs.add(DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES().key() + "=true"); + downstreamCfg.configs.add(DataSourceReadOptions.INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN().key() + "=true"); //Adding this conf to make testing easier :) downstreamCfg.configs.add("hoodie.deltastreamer.source.hoodieincr.num_instants=10"); downstreamCfg.operation = WriteOperationType.UPSERT; diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionQuick.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionQuick.java index db7cb54fe769..9c985dd0cf59 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionQuick.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionQuick.java @@ -466,8 +466,8 @@ public void testNonNullableColumnDrop(String tableType, .stream().anyMatch(t -> t.getType().equals(Schema.Type.STRING))); assertTrue(metaClient.reloadActiveTimeline().lastInstant().get().compareTo(lastInstant) > 0); } catch (Exception e) { - assertTrue(e.getMessage().contains("java.lang.NullPointerException") - || e.getMessage().contains("Incoming batch schema is not compatible with the table's one")); + assertTrue(containsErrorMessage(e, "java.lang.NullPointerException", + "Incoming batch schema is not compatible with the table's one")); } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestSparkSampleWritesUtils.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestSparkSampleWritesUtils.java index e1676219ca0a..2706a97e5d5c 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestSparkSampleWritesUtils.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestSparkSampleWritesUtils.java @@ -80,7 +80,7 @@ public void skipOverwriteRecordSizeEstimateWhenTimelineNonEmpty() throws Excepti .withPath(basePath()) .build(); JavaRDD records = jsc().parallelize(dataGen.generateInserts(commitTime, 1), 1); - Option writeConfigOpt = SparkSampleWritesUtils.getWriteConfigWithRecordSizeEstimate(jsc(), records, originalWriteConfig); + Option writeConfigOpt = SparkSampleWritesUtils.getWriteConfigWithRecordSizeEstimate(jsc(), Option.of(records), originalWriteConfig); assertFalse(writeConfigOpt.isPresent()); assertEquals(originalRecordSize, originalWriteConfig.getCopyOnWriteRecordSizeEstimate(), "Original record size estimate should not be changed."); } @@ -100,7 +100,7 @@ public void overwriteRecordSizeEstimateForEmptyTable() { String commitTime = HoodieTestDataGenerator.getCommitTimeAtUTC(1); JavaRDD records = jsc().parallelize(dataGen.generateInserts(commitTime, 2000), 2); - Option writeConfigOpt = SparkSampleWritesUtils.getWriteConfigWithRecordSizeEstimate(jsc(), records, originalWriteConfig); + Option writeConfigOpt = SparkSampleWritesUtils.getWriteConfigWithRecordSizeEstimate(jsc(), Option.of(records), originalWriteConfig); assertTrue(writeConfigOpt.isPresent()); assertEquals(779.0, writeConfigOpt.get().getCopyOnWriteRecordSizeEstimate(), 10.0); } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestGcsEventsHoodieIncrSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestGcsEventsHoodieIncrSource.java index 5c31f310800b..bc2906d251fc 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestGcsEventsHoodieIncrSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestGcsEventsHoodieIncrSource.java @@ -40,6 +40,7 @@ import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.sources.helpers.CloudDataFetcher; import org.apache.hudi.utilities.sources.helpers.IncrSourceHelper; +import org.apache.hudi.utilities.sources.helpers.QueryInfo; import org.apache.hudi.utilities.sources.helpers.QueryRunner; import org.apache.hudi.utilities.sources.helpers.gcs.GcsObjectMetadataFetcher; @@ -56,6 +57,8 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; @@ -93,6 +96,8 @@ public class TestGcsEventsHoodieIncrSource extends SparkClientFunctionalTestHarn @Mock QueryRunner queryRunner; + @Mock + QueryInfo queryInfo; protected Option schemaProvider; private HoodieTableMetaClient metaClient; @@ -142,7 +147,7 @@ public void shouldFetchDataIfCommitTimeForReadsLessThanForWrites() throws IOExce filePathSizeAndCommitTime.add(Triple.of("path/to/file3.json", 200L, "1")); Dataset inputDs = generateDataset(filePathSizeAndCommitTime); - when(queryRunner.run(Mockito.any())).thenReturn(inputDs); + setMockQueryRunner(inputDs); readAndAssert(READ_UPTO_LATEST_COMMIT, Option.of(commitTimeForReads), 100L, "1#path/to/file1.json"); } @@ -160,7 +165,8 @@ public void testTwoFilesAndContinueInSameCommit() throws IOException { filePathSizeAndCommitTime.add(Triple.of("path/to/file3.json", 200L, "1")); Dataset inputDs = generateDataset(filePathSizeAndCommitTime); - when(queryRunner.run(Mockito.any())).thenReturn(inputDs); + + setMockQueryRunner(inputDs); readAndAssert(READ_UPTO_LATEST_COMMIT, Option.of(commitTimeForReads), 250L, "1#path/to/file2.json"); readAndAssert(READ_UPTO_LATEST_COMMIT, Option.of("1#path/to/file2.json"), 250L, "1#path/to/file3.json"); @@ -183,7 +189,7 @@ public void largeBootstrapWithFilters() throws IOException { Dataset inputDs = generateDataset(filePathSizeAndCommitTime); - when(queryRunner.run(Mockito.any())).thenReturn(inputDs); + setMockQueryRunner(inputDs); readAndAssert(READ_UPTO_LATEST_COMMIT, Option.of(commitTimeForReads), 250L, "1#path/to/file10006.json"); readAndAssert(READ_UPTO_LATEST_COMMIT, Option.of("1#path/to/file10006.json"), 250L, "1#path/to/file10007.json"); @@ -205,7 +211,7 @@ public void testTwoFilesAndContinueAcrossCommits() throws IOException { Dataset inputDs = generateDataset(filePathSizeAndCommitTime); - when(queryRunner.run(Mockito.any())).thenReturn(inputDs); + setMockQueryRunner(inputDs); readAndAssert(READ_UPTO_LATEST_COMMIT, Option.of(commitTimeForReads), 100L, "1#path/to/file1.json"); readAndAssert(READ_UPTO_LATEST_COMMIT, Option.of("1#path/to/file1.json"), 100L, "1#path/to/file2.json"); @@ -213,10 +219,68 @@ public void testTwoFilesAndContinueAcrossCommits() throws IOException { readAndAssert(READ_UPTO_LATEST_COMMIT, Option.of(commitTimeForReads), 100L, "1#path/to/file1.json"); } + @ParameterizedTest + @CsvSource({ + "1,1#path/to/file2.json,3#path/to/file4.json,1#path/to/file1.json,1", + "2,1#path/to/file2.json,3#path/to/file4.json,1#path/to/file1.json,2", + "3,3#path/to/file5.json,3,1#path/to/file1.json,3" + }) + public void testSplitSnapshotLoad(String snapshotCheckPoint, String exptected1, String exptected2, String exptected3, String exptected4) throws IOException { + + writeGcsMetadataRecords("1"); + writeGcsMetadataRecords("2"); + writeGcsMetadataRecords("3"); + + List> filePathSizeAndCommitTime = new ArrayList<>(); + // Add file paths and sizes to the list + filePathSizeAndCommitTime.add(Triple.of("path/to/file1.json", 50L, "1")); + filePathSizeAndCommitTime.add(Triple.of("path/to/file2.json", 50L, "1")); + filePathSizeAndCommitTime.add(Triple.of("path/to/skip1.json", 50L, "2")); + filePathSizeAndCommitTime.add(Triple.of("path/to/skip2.json", 50L, "2")); + filePathSizeAndCommitTime.add(Triple.of("path/to/file5.json", 50L, "3")); + filePathSizeAndCommitTime.add(Triple.of("path/to/file4.json", 50L, "3")); + + Dataset inputDs = generateDataset(filePathSizeAndCommitTime); + + setMockQueryRunner(inputDs, Option.of(snapshotCheckPoint)); + TypedProperties typedProperties = setProps(READ_UPTO_LATEST_COMMIT); + typedProperties.setProperty("hoodie.deltastreamer.source.cloud.data.ignore.relpath.prefix", "path/to/skip"); + //1. snapshot query, read all records + readAndAssert(READ_UPTO_LATEST_COMMIT, Option.empty(), 50000L, exptected1, typedProperties); + //2. incremental query, as commit is present in timeline + readAndAssert(READ_UPTO_LATEST_COMMIT, Option.of(exptected1), 10L, exptected2, typedProperties); + //3. snapshot query with source limit less than first commit size + readAndAssert(READ_UPTO_LATEST_COMMIT, Option.empty(), 50L, exptected3, typedProperties); + typedProperties.setProperty("hoodie.deltastreamer.source.cloud.data.ignore.relpath.prefix", "path/to"); + //4. As snapshotQuery will return 1 -> same would be return as nextCheckpoint (dataset is empty due to ignore prefix). + readAndAssert(READ_UPTO_LATEST_COMMIT, Option.empty(), 50L, exptected4, typedProperties); + } + + private void setMockQueryRunner(Dataset inputDs) { + setMockQueryRunner(inputDs, Option.empty()); + } + + private void setMockQueryRunner(Dataset inputDs, Option nextCheckPointOpt) { + + when(queryRunner.run(Mockito.any(QueryInfo.class), Mockito.any())).thenAnswer(invocation -> { + QueryInfo queryInfo = invocation.getArgument(0); + QueryInfo updatedQueryInfo = nextCheckPointOpt.map(nextCheckPoint -> + queryInfo.withUpdatedEndInstant(nextCheckPoint)) + .orElse(queryInfo); + if (updatedQueryInfo.isSnapshot()) { + return Pair.of(updatedQueryInfo, + inputDs.filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, + updatedQueryInfo.getStartInstant())) + .filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, + updatedQueryInfo.getEndInstant()))); + } + return Pair.of(updatedQueryInfo, inputDs); + }); + } + private void readAndAssert(IncrSourceHelper.MissingCheckpointStrategy missingCheckpointStrategy, - Option checkpointToPull, long sourceLimit, String expectedCheckpoint) { - TypedProperties typedProperties = setProps(missingCheckpointStrategy); - typedProperties.put("hoodie.deltastreamer.source.hoodieincr.file.format", "json"); + Option checkpointToPull, long sourceLimit, String expectedCheckpoint, + TypedProperties typedProperties) { GcsEventsHoodieIncrSource incrSource = new GcsEventsHoodieIncrSource(typedProperties, jsc(), spark(), schemaProvider.orElse(null), new GcsObjectMetadataFetcher(typedProperties, "json"), gcsObjectDataFetcher, queryRunner); @@ -230,6 +294,13 @@ private void readAndAssert(IncrSourceHelper.MissingCheckpointStrategy missingChe Assertions.assertEquals(expectedCheckpoint, nextCheckPoint); } + private void readAndAssert(IncrSourceHelper.MissingCheckpointStrategy missingCheckpointStrategy, + Option checkpointToPull, long sourceLimit, String expectedCheckpoint) { + TypedProperties typedProperties = setProps(missingCheckpointStrategy); + typedProperties.put("hoodie.deltastreamer.source.hoodieincr.file.format", "json"); + readAndAssert(missingCheckpointStrategy, checkpointToPull, sourceLimit, expectedCheckpoint, typedProperties); + } + private HoodieRecord getGcsMetadataRecord(String commitTime, String filename, String bucketName, String generation) { String partitionPath = bucketName; diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestS3EventsHoodieIncrSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestS3EventsHoodieIncrSource.java index 7d58d21d874f..e0af8d73e269 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestS3EventsHoodieIncrSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestS3EventsHoodieIncrSource.java @@ -40,6 +40,7 @@ import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.sources.helpers.CloudDataFetcher; import org.apache.hudi.utilities.sources.helpers.IncrSourceHelper; +import org.apache.hudi.utilities.sources.helpers.QueryInfo; import org.apache.hudi.utilities.sources.helpers.QueryRunner; import org.apache.hudi.utilities.sources.helpers.TestCloudObjectsSelectorCommon; @@ -56,6 +57,8 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; @@ -88,6 +91,8 @@ public class TestS3EventsHoodieIncrSource extends SparkClientFunctionalTestHarne QueryRunner mockQueryRunner; @Mock CloudDataFetcher mockCloudDataFetcher; + @Mock + QueryInfo queryInfo; private JavaSparkContext jsc; private HoodieTableMetaClient metaClient; @@ -248,7 +253,7 @@ public void testOneFileInCommit() throws IOException { Dataset inputDs = generateDataset(filePathSizeAndCommitTime); - when(mockQueryRunner.run(Mockito.any())).thenReturn(inputDs); + setMockQueryRunner(inputDs); when(mockCloudDataFetcher.getCloudObjectDataDF(Mockito.any(), Mockito.any(), Mockito.any(), eq(schemaProvider))) .thenReturn(Option.empty()); @@ -273,7 +278,7 @@ public void testTwoFilesAndContinueInSameCommit() throws IOException { Dataset inputDs = generateDataset(filePathSizeAndCommitTime); - when(mockQueryRunner.run(Mockito.any())).thenReturn(inputDs); + setMockQueryRunner(inputDs); when(mockCloudDataFetcher.getCloudObjectDataDF(Mockito.any(), Mockito.any(), Mockito.any(), eq(schemaProvider))) .thenReturn(Option.empty()); @@ -301,7 +306,7 @@ public void testTwoFilesAndContinueAcrossCommits() throws IOException { Dataset inputDs = generateDataset(filePathSizeAndCommitTime); - when(mockQueryRunner.run(Mockito.any())).thenReturn(inputDs); + setMockQueryRunner(inputDs); when(mockCloudDataFetcher.getCloudObjectDataDF(Mockito.any(), Mockito.any(), Mockito.any(), eq(schemaProvider))) .thenReturn(Option.empty()); @@ -329,7 +334,7 @@ public void testEmptyDataAfterFilter() throws IOException { Dataset inputDs = generateDataset(filePathSizeAndCommitTime); - when(mockQueryRunner.run(Mockito.any())).thenReturn(inputDs); + setMockQueryRunner(inputDs); TypedProperties typedProperties = setProps(READ_UPTO_LATEST_COMMIT); typedProperties.setProperty("hoodie.deltastreamer.source.s3incr.ignore.key.prefix", "path/to/skip"); @@ -361,7 +366,7 @@ public void testFilterAnEntireCommit() throws IOException { Dataset inputDs = generateDataset(filePathSizeAndCommitTime); - when(mockQueryRunner.run(Mockito.any())).thenReturn(inputDs); + setMockQueryRunner(inputDs); when(mockCloudDataFetcher.getCloudObjectDataDF(Mockito.any(), Mockito.any(), Mockito.any(), eq(schemaProvider))) .thenReturn(Option.empty()); TypedProperties typedProperties = setProps(READ_UPTO_LATEST_COMMIT); @@ -393,7 +398,7 @@ public void testFilterAnEntireMiddleCommit() throws IOException { Dataset inputDs = generateDataset(filePathSizeAndCommitTime); - when(mockQueryRunner.run(Mockito.any())).thenReturn(inputDs); + setMockQueryRunner(inputDs); when(mockCloudDataFetcher.getCloudObjectDataDF(Mockito.any(), Mockito.any(), Mockito.any(), eq(schemaProvider))) .thenReturn(Option.empty()); TypedProperties typedProperties = setProps(READ_UPTO_LATEST_COMMIT); @@ -407,6 +412,45 @@ public void testFilterAnEntireMiddleCommit() throws IOException { readAndAssert(READ_UPTO_LATEST_COMMIT, Option.of("1#path/to/file3.json"), 50L, "3#path/to/file4.json", typedProperties); } + @ParameterizedTest + @CsvSource({ + "1,1#path/to/file2.json,3#path/to/file4.json,1#path/to/file1.json,1", + "2,1#path/to/file2.json,3#path/to/file4.json,1#path/to/file1.json,2", + "3,3#path/to/file5.json,3,1#path/to/file1.json,3" + }) + public void testSplitSnapshotLoad(String snapshotCheckPoint, String exptected1, String exptected2, String exptected3, String exptected4) throws IOException { + + writeS3MetadataRecords("1"); + writeS3MetadataRecords("2"); + writeS3MetadataRecords("3"); + + List> filePathSizeAndCommitTime = new ArrayList<>(); + // Add file paths and sizes to the list + filePathSizeAndCommitTime.add(Triple.of("path/to/file1.json", 50L, "1")); + filePathSizeAndCommitTime.add(Triple.of("path/to/file2.json", 50L, "1")); + filePathSizeAndCommitTime.add(Triple.of("path/to/skip1.json", 50L, "2")); + filePathSizeAndCommitTime.add(Triple.of("path/to/skip2.json", 50L, "2")); + filePathSizeAndCommitTime.add(Triple.of("path/to/file5.json", 50L, "3")); + filePathSizeAndCommitTime.add(Triple.of("path/to/file4.json", 50L, "3")); + + Dataset inputDs = generateDataset(filePathSizeAndCommitTime); + + setMockQueryRunner(inputDs, Option.of(snapshotCheckPoint)); + when(mockCloudDataFetcher.getCloudObjectDataDF(Mockito.any(), Mockito.any(), Mockito.any(), eq(schemaProvider))) + .thenReturn(Option.empty()); + TypedProperties typedProperties = setProps(READ_UPTO_LATEST_COMMIT); + typedProperties.setProperty("hoodie.deltastreamer.source.s3incr.ignore.key.prefix", "path/to/skip"); + //1. snapshot query, read all records + readAndAssert(READ_UPTO_LATEST_COMMIT, Option.empty(), 50000L, exptected1, typedProperties); + //2. incremental query, as commit is present in timeline + readAndAssert(READ_UPTO_LATEST_COMMIT, Option.of(exptected1), 10L, exptected2, typedProperties); + //3. snapshot query with source limit less than first commit size + readAndAssert(READ_UPTO_LATEST_COMMIT, Option.empty(), 50L, exptected3, typedProperties); + typedProperties.setProperty("hoodie.deltastreamer.source.s3incr.ignore.key.prefix", "path/to"); + //4. As snapshotQuery will return 1 -> same would be return as nextCheckpoint (dataset is empty due to ignore prefix). + readAndAssert(READ_UPTO_LATEST_COMMIT, Option.empty(), 50L, exptected4, typedProperties); + } + private void readAndAssert(IncrSourceHelper.MissingCheckpointStrategy missingCheckpointStrategy, Option checkpointToPull, long sourceLimit, String expectedCheckpoint, TypedProperties typedProperties) { @@ -422,6 +466,28 @@ private void readAndAssert(IncrSourceHelper.MissingCheckpointStrategy missingChe Assertions.assertEquals(expectedCheckpoint, nextCheckPoint); } + private void setMockQueryRunner(Dataset inputDs) { + setMockQueryRunner(inputDs, Option.empty()); + } + + private void setMockQueryRunner(Dataset inputDs, Option nextCheckPointOpt) { + + when(mockQueryRunner.run(Mockito.any(QueryInfo.class), Mockito.any())).thenAnswer(invocation -> { + QueryInfo queryInfo = invocation.getArgument(0); + QueryInfo updatedQueryInfo = nextCheckPointOpt.map(nextCheckPoint -> + queryInfo.withUpdatedEndInstant(nextCheckPoint)) + .orElse(queryInfo); + if (updatedQueryInfo.isSnapshot()) { + return Pair.of(updatedQueryInfo, + inputDs.filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, + updatedQueryInfo.getStartInstant())) + .filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, + updatedQueryInfo.getEndInstant()))); + } + return Pair.of(updatedQueryInfo, inputDs); + }); + } + private void readAndAssert(IncrSourceHelper.MissingCheckpointStrategy missingCheckpointStrategy, Option checkpointToPull, long sourceLimit, String expectedCheckpoint) { TypedProperties typedProperties = setProps(missingCheckpointStrategy); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/gcs/TestPubsubMessagesFetcher.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/gcs/TestPubsubMessagesFetcher.java new file mode 100644 index 000000000000..2122dfa7af45 --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/gcs/TestPubsubMessagesFetcher.java @@ -0,0 +1,110 @@ +/* + * 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.utilities.sources.helpers.gcs; + +import com.google.cloud.pubsub.v1.stub.SubscriberStub; +import com.google.pubsub.v1.ProjectSubscriptionName; +import com.google.pubsub.v1.PubsubMessage; +import com.google.pubsub.v1.PullResponse; +import com.google.pubsub.v1.ReceivedMessage; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class TestPubsubMessagesFetcher { + private static final String PROJECT_ID = "test-project"; + private static final String SUBSCRIPTION_ID = "test-subscription"; + private static final String SUBSCRIPTION_NAME = ProjectSubscriptionName.format(PROJECT_ID, SUBSCRIPTION_ID); + private static final int SMALL_BATCH_SIZE = 1; + private static final int MAX_MESSAGES_IN_REQUEST = 1000; + private static final long MAX_WAIT_TIME_IN_REQUEST = TimeUnit.SECONDS.toMillis(1); + + private final SubscriberStub mockSubscriber = Mockito.mock(SubscriberStub.class); + private final PubsubQueueClient mockPubsubQueueClient = Mockito.mock(PubsubQueueClient.class); + + @Test + public void testFetchMessages() throws IOException { + doNothing().when(mockSubscriber).close(); + when(mockPubsubQueueClient.getSubscriber(any())).thenReturn(mockSubscriber); + when(mockPubsubQueueClient.getNumUnAckedMessages(SUBSCRIPTION_ID)).thenReturn(3L); + doNothing().when(mockSubscriber).close(); + ReceivedMessage message1 = ReceivedMessage.newBuilder().setAckId("1").setMessage(PubsubMessage.newBuilder().setMessageId("msgId1").build()).build(); + ReceivedMessage message2 = ReceivedMessage.newBuilder().setAckId("2").setMessage(PubsubMessage.newBuilder().setMessageId("msgId2").build()).build(); + ReceivedMessage message3 = ReceivedMessage.newBuilder().setAckId("3").setMessage(PubsubMessage.newBuilder().setMessageId("msgId3").build()).build(); + when(mockPubsubQueueClient.makePullRequest(mockSubscriber, SUBSCRIPTION_NAME, SMALL_BATCH_SIZE)) + .thenReturn(PullResponse.newBuilder().addReceivedMessages(message1).build()) + .thenReturn(PullResponse.newBuilder().addReceivedMessages(message2).build()) + .thenReturn(PullResponse.newBuilder().addReceivedMessages(message3).build()); + + PubsubMessagesFetcher fetcher = new PubsubMessagesFetcher( + PROJECT_ID, SUBSCRIPTION_ID, SMALL_BATCH_SIZE, + MAX_MESSAGES_IN_REQUEST, MAX_WAIT_TIME_IN_REQUEST, mockPubsubQueueClient + ); + List messages = fetcher.fetchMessages(); + + assertEquals(3, messages.size()); + assertEquals("1", messages.get(0).getAckId()); + assertEquals("2", messages.get(1).getAckId()); + assertEquals("3", messages.get(2).getAckId()); + verify(mockPubsubQueueClient, times(3)).makePullRequest(mockSubscriber, SUBSCRIPTION_NAME, SMALL_BATCH_SIZE); + } + + @Test + public void testFetchMessagesZeroTimeout() throws IOException { + doNothing().when(mockSubscriber).close(); + when(mockPubsubQueueClient.getSubscriber(any())).thenReturn(mockSubscriber); + when(mockPubsubQueueClient.getNumUnAckedMessages(SUBSCRIPTION_ID)).thenReturn(100L); + PubsubMessagesFetcher fetcher = new PubsubMessagesFetcher( + PROJECT_ID, SUBSCRIPTION_ID, SMALL_BATCH_SIZE, + MAX_MESSAGES_IN_REQUEST, 0, mockPubsubQueueClient + ); + + List messages = fetcher.fetchMessages(); + assertEquals(0, messages.size()); + } + + @Test + public void testSendAcks() throws IOException { + doNothing().when(mockSubscriber).close(); + when(mockPubsubQueueClient.getSubscriber(any())).thenReturn(mockSubscriber); + List messageAcks = IntStream.range(0, 20).mapToObj(i -> "msg_" + i).collect(Collectors.toList()); + doNothing().when(mockPubsubQueueClient).makeAckRequest(eq(mockSubscriber), eq(SUBSCRIPTION_NAME), any()); + PubsubMessagesFetcher fetcher = new PubsubMessagesFetcher( + PROJECT_ID, SUBSCRIPTION_ID, SMALL_BATCH_SIZE, + MAX_MESSAGES_IN_REQUEST, MAX_WAIT_TIME_IN_REQUEST, mockPubsubQueueClient + ); + + fetcher.sendAcks(messageAcks); + verify(mockPubsubQueueClient, times(2)).makeAckRequest(eq(mockSubscriber), eq(SUBSCRIPTION_NAME), any()); + } + +} \ No newline at end of file diff --git a/packaging/bundle-validation/README.md b/packaging/bundle-validation/README.md index f18419d98812..41a546486ce4 100644 --- a/packaging/bundle-validation/README.md +++ b/packaging/bundle-validation/README.md @@ -33,17 +33,17 @@ the folder. Here are the docker commands to build the image by specifying differ ```shell docker build \ --build-arg HIVE_VERSION=3.1.3 \ - --build-arg FLINK_VERSION=1.13.6 \ + --build-arg FLINK_VERSION=1.14.6 \ --build-arg SPARK_VERSION=3.1.3 \ --build-arg SPARK_HADOOP_VERSION=2.7 \ - -t hudi-ci-bundle-validation-base:flink1136hive313spark313 . -docker image tag hudi-ci-bundle-validation-base:flink1136hive313spark313 apachehudi/hudi-ci-bundle-validation-base:flink1136hive313spark313 + -t hudi-ci-bundle-validation-base:flink1146hive313spark313 . +docker image tag hudi-ci-bundle-validation-base:flink1146hive313spark313 apachehudi/hudi-ci-bundle-validation-base:flink1146hive313spark313 ``` To upload the image with the tag: ```shell -docker push apachehudi/hudi-ci-bundle-validation-base:flink1136hive313spark313 +docker push apachehudi/hudi-ci-bundle-validation-base:flink1146hive313spark313 ``` Note that for each library like Hive and Spark, the download and extraction happen under one `RUN` instruction so that diff --git a/packaging/bundle-validation/base/build_flink1146hive239spark248.sh b/packaging/bundle-validation/base/build_flink1146hive239spark248.sh new file mode 100755 index 000000000000..ecbb2fa7b2ac --- /dev/null +++ b/packaging/bundle-validation/base/build_flink1146hive239spark248.sh @@ -0,0 +1,26 @@ +#!/bin/bash + +# 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. + +docker build \ + --build-arg HIVE_VERSION=2.3.9 \ + --build-arg FLINK_VERSION=1.14.6 \ + --build-arg SPARK_VERSION=2.4.8 \ + --build-arg SPARK_HADOOP_VERSION=2.7 \ + -t hudi-ci-bundle-validation-base:flink1146hive239spark248 . +docker image tag hudi-ci-bundle-validation-base:flink1146hive239spark248 apachehudi/hudi-ci-bundle-validation-base:flink1146hive239spark248 diff --git a/packaging/bundle-validation/base/build_flink1153hive313spark323.sh b/packaging/bundle-validation/base/build_flink1153hive313spark323.sh new file mode 100755 index 000000000000..4f20a42d095f --- /dev/null +++ b/packaging/bundle-validation/base/build_flink1153hive313spark323.sh @@ -0,0 +1,26 @@ +#!/bin/bash + +# 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. + +docker build \ + --build-arg HIVE_VERSION=3.1.3 \ + --build-arg FLINK_VERSION=1.15.3 \ + --build-arg SPARK_VERSION=3.2.3 \ + --build-arg SPARK_HADOOP_VERSION=2.7 \ + -t hudi-ci-bundle-validation-base:flink1153hive313spark323 . +docker image tag hudi-ci-bundle-validation-base:flink1153hive313spark323 apachehudi/hudi-ci-bundle-validation-base:flink1153hive313spark323 diff --git a/packaging/bundle-validation/base/build_flink1162hive313spark331.sh b/packaging/bundle-validation/base/build_flink1162hive313spark331.sh new file mode 100755 index 000000000000..03e01729ef60 --- /dev/null +++ b/packaging/bundle-validation/base/build_flink1162hive313spark331.sh @@ -0,0 +1,26 @@ +#!/bin/bash + +# 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. + +docker build \ + --build-arg HIVE_VERSION=3.1.3 \ + --build-arg FLINK_VERSION=1.16.2 \ + --build-arg SPARK_VERSION=3.3.1 \ + --build-arg SPARK_HADOOP_VERSION=2 \ + -t hudi-ci-bundle-validation-base:flink1162hive313spark331 . +docker image tag hudi-ci-bundle-validation-base:flink1162hive313spark331 apachehudi/hudi-ci-bundle-validation-base:flink1162hive313spark331 diff --git a/packaging/bundle-validation/ci_run.sh b/packaging/bundle-validation/ci_run.sh index 59fc5d9df397..987d093a5312 100755 --- a/packaging/bundle-validation/ci_run.sh +++ b/packaging/bundle-validation/ci_run.sh @@ -38,12 +38,12 @@ if [[ ${SPARK_RUNTIME} == 'spark2.4.8' ]]; then HADOOP_VERSION=2.7.7 HIVE_VERSION=2.3.9 DERBY_VERSION=10.10.2.0 - FLINK_VERSION=1.13.6 + FLINK_VERSION=1.14.6 SPARK_VERSION=2.4.8 SPARK_HADOOP_VERSION=2.7 CONFLUENT_VERSION=5.5.12 KAFKA_CONNECT_HDFS_VERSION=10.1.13 - IMAGE_TAG=flink1136hive239spark248 + IMAGE_TAG=flink1146hive239spark248 elif [[ ${SPARK_RUNTIME} == 'spark3.0.2' ]]; then HADOOP_VERSION=2.7.7 HIVE_VERSION=3.1.3 @@ -58,32 +58,32 @@ elif [[ ${SPARK_RUNTIME} == 'spark3.1.3' ]]; then HADOOP_VERSION=2.7.7 HIVE_VERSION=3.1.3 DERBY_VERSION=10.14.1.0 - FLINK_VERSION=1.13.6 + FLINK_VERSION=1.14.6 SPARK_VERSION=3.1.3 SPARK_HADOOP_VERSION=2.7 CONFLUENT_VERSION=5.5.12 KAFKA_CONNECT_HDFS_VERSION=10.1.13 - IMAGE_TAG=flink1136hive313spark313 + IMAGE_TAG=flink1146hive313spark313 elif [[ ${SPARK_RUNTIME} == 'spark3.2.3' ]]; then HADOOP_VERSION=2.7.7 HIVE_VERSION=3.1.3 DERBY_VERSION=10.14.1.0 - FLINK_VERSION=1.14.6 + FLINK_VERSION=1.15.3 SPARK_VERSION=3.2.3 SPARK_HADOOP_VERSION=2.7 CONFLUENT_VERSION=5.5.12 KAFKA_CONNECT_HDFS_VERSION=10.1.13 - IMAGE_TAG=flink1146hive313spark323 + IMAGE_TAG=flink1153hive313spark323 elif [[ ${SPARK_RUNTIME} == 'spark3.3.1' ]]; then HADOOP_VERSION=2.7.7 HIVE_VERSION=3.1.3 DERBY_VERSION=10.14.1.0 - FLINK_VERSION=1.15.3 + FLINK_VERSION=1.16.2 SPARK_VERSION=3.3.1 SPARK_HADOOP_VERSION=2 CONFLUENT_VERSION=5.5.12 KAFKA_CONNECT_HDFS_VERSION=10.1.13 - IMAGE_TAG=flink1153hive313spark331 + IMAGE_TAG=flink1162hive313spark331 elif [[ ${SPARK_RUNTIME} == 'spark3.3.2' ]]; then HADOOP_VERSION=2.7.7 HIVE_VERSION=3.1.3 @@ -98,12 +98,12 @@ elif [[ ${SPARK_RUNTIME} == 'spark3.4.0' ]]; then HADOOP_VERSION=3.3.5 HIVE_VERSION=3.1.3 DERBY_VERSION=10.14.1.0 - FLINK_VERSION=1.17.0 + FLINK_VERSION=1.18.0 SPARK_VERSION=3.4.0 SPARK_HADOOP_VERSION=3 CONFLUENT_VERSION=5.5.12 KAFKA_CONNECT_HDFS_VERSION=10.1.13 - IMAGE_TAG=flink1170hive313spark340 + IMAGE_TAG=flink1180hive313spark340 elif [[ ${SPARK_RUNTIME} == 'spark3.5.0' ]]; then HADOOP_VERSION=3.3.5 HIVE_VERSION=3.1.3 @@ -156,15 +156,21 @@ else HUDI_SPARK_BUNDLE_NAME=hudi-spark3.3-bundle_2.12 HUDI_UTILITIES_BUNDLE_NAME=hudi-utilities-bundle_2.12 HUDI_UTILITIES_SLIM_BUNDLE_NAME=hudi-utilities-slim-bundle_2.12 + elif [[ ${SPARK_PROFILE} == 'spark3.4' ]]; then + HUDI_SPARK_BUNDLE_NAME=hudi-spark3.4-bundle_2.12 + HUDI_UTILITIES_BUNDLE_NAME=hudi-utilities-bundle_2.12 + HUDI_UTILITIES_SLIM_BUNDLE_NAME=hudi-utilities-slim-bundle_2.12 + elif [[ ${SPARK_PROFILE} == 'spark3.5' ]]; then + HUDI_SPARK_BUNDLE_NAME=hudi-spark3.5-bundle_2.12 + HUDI_UTILITIES_BUNDLE_NAME=hudi-utilities-bundle_2.12 + HUDI_UTILITIES_SLIM_BUNDLE_NAME=hudi-utilities-slim-bundle_2.12 elif [[ ${SPARK_PROFILE} == 'spark3' ]]; then HUDI_SPARK_BUNDLE_NAME=hudi-spark3-bundle_2.12 HUDI_UTILITIES_BUNDLE_NAME=hudi-utilities-bundle_2.12 HUDI_UTILITIES_SLIM_BUNDLE_NAME=hudi-utilities-slim-bundle_2.12 fi - if [[ ${FLINK_PROFILE} == 'flink1.13' ]]; then - HUDI_FLINK_BUNDLE_NAME=hudi-flink1.13-bundle - elif [[ ${FLINK_PROFILE} == 'flink1.14' ]]; then + if [[ ${FLINK_PROFILE} == 'flink1.14' ]]; then HUDI_FLINK_BUNDLE_NAME=hudi-flink1.14-bundle elif [[ ${FLINK_PROFILE} == 'flink1.15' ]]; then HUDI_FLINK_BUNDLE_NAME=hudi-flink1.15-bundle diff --git a/packaging/bundle-validation/run_docker_java17.sh b/packaging/bundle-validation/run_docker_java17.sh index d9f50cc90768..1b774eefdf19 100755 --- a/packaging/bundle-validation/run_docker_java17.sh +++ b/packaging/bundle-validation/run_docker_java17.sh @@ -27,12 +27,12 @@ if [[ ${SPARK_RUNTIME} == 'spark2.4.8' ]]; then HADOOP_VERSION=2.7.7 HIVE_VERSION=2.3.9 DERBY_VERSION=10.10.2.0 - FLINK_VERSION=1.13.6 + FLINK_VERSION=1.14.6 SPARK_VERSION=2.4.8 SPARK_HADOOP_VERSION=2.7 CONFLUENT_VERSION=5.5.12 KAFKA_CONNECT_HDFS_VERSION=10.1.13 - IMAGE_TAG=flink1136hive239spark248 + IMAGE_TAG=flink1146hive239spark248 elif [[ ${SPARK_RUNTIME} == 'spark3.0.2' ]]; then HADOOP_VERSION=2.7.7 HIVE_VERSION=3.1.3 @@ -47,12 +47,12 @@ elif [[ ${SPARK_RUNTIME} == 'spark3.1.3' ]]; then HADOOP_VERSION=2.7.7 HIVE_VERSION=3.1.3 DERBY_VERSION=10.14.1.0 - FLINK_VERSION=1.13.6 + FLINK_VERSION=1.14.6 SPARK_VERSION=3.1.3 SPARK_HADOOP_VERSION=2.7 CONFLUENT_VERSION=5.5.12 KAFKA_CONNECT_HDFS_VERSION=10.1.13 - IMAGE_TAG=flink1136hive313spark313 + IMAGE_TAG=flink1146hive313spark313 elif [[ ${SPARK_RUNTIME} == 'spark3.2.3' ]]; then HADOOP_VERSION=2.7.7 HIVE_VERSION=3.1.3 diff --git a/pom.xml b/pom.xml index 6fdae65db390..2d80c0155fd8 100644 --- a/pom.xml +++ b/pom.xml @@ -141,7 +141,6 @@ 1.16.2 1.15.1 1.14.5 - 1.13.6 ${flink1.18.version} hudi-flink1.18.x 1.18 @@ -472,6 +471,8 @@ org.apache.hbase.thirdparty:hbase-shaded-netty org.apache.hbase.thirdparty:hbase-shaded-protobuf org.apache.htrace:htrace-core4 + + com.fasterxml.jackson.module:jackson-module-afterburner @@ -567,6 +568,11 @@ org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles + + com.fasterxml.jackson.module + org.apache.hudi.com.fasterxml.jackson.module + + @@ -873,6 +879,18 @@ jackson-module-scala_${scala.binary.version} ${fasterxml.jackson.module.scala.version} + + + com.fasterxml.jackson.module + jackson-module-afterburner + ${fasterxml.version} + + + + com.fasterxml.jackson.module + jackson-module-afterburner + ${fasterxml.jackson.databind.version} + @@ -2175,7 +2193,7 @@ 1.8.2 4.7 2.6.7 - 2.6.7.3 + ${fasterxml.version} 2.6.7.1 2.7.4 true @@ -2207,7 +2225,7 @@ 1.8.2 4.7 2.6.7 - 2.6.7.3 + ${fasterxml.version} 2.6.7.1 2.7.4 true @@ -2666,33 +2684,6 @@ - - flink1.13 - - ${flink1.13.version} - hudi-flink1.13.x - 1.13 - 1.5.6 - 1.11.1 - flink-runtime_${scala.binary.version} - flink-table-runtime-blink_${scala.binary.version} - flink-table-planner-blink_${scala.binary.version} - flink-parquet_${scala.binary.version} - flink-statebackend-rocksdb_${scala.binary.version} - flink-test-utils_${scala.binary.version} - flink-streaming-java_${scala.binary.version} - flink-clients_${scala.binary.version} - flink-connector-kafka_${scala.binary.version} - flink-hadoop-compatibility_${scala.binary.version} - ${flink1.13.version} - true - - - - flink1.13 - - - skipShadeSources diff --git a/scripts/release/deploy_staging_jars.sh b/scripts/release/deploy_staging_jars.sh index 4e54d30a2a9b..944017e04ade 100755 --- a/scripts/release/deploy_staging_jars.sh +++ b/scripts/release/deploy_staging_jars.sh @@ -66,9 +66,13 @@ declare -a ALL_VERSION_OPTS=( "-Dscala-2.12 -Dspark3.3 -pl hudi-spark-datasource/hudi-spark3.3.x,packaging/hudi-spark-bundle -am" # For Spark 3.4, Scala 2.12: # hudi-spark3.4.x_2.12 -# hudi-cli-bundle_2.12 # hudi-spark3.4-bundle_2.12 -"-Dscala-2.12 -Dspark3.4 -pl hudi-spark-datasource/hudi-spark3.4.x,packaging/hudi-spark-bundle,packaging/hudi-cli-bundle -am" +"-Dscala-2.12 -Dspark3.4 -pl hudi-spark-datasource/hudi-spark3.4.x,packaging/hudi-spark-bundle -am" +# For Spark 3.5, Scala 2.12: +# hudi-spark3.5.x_2.12 +# hudi-cli-bundle_2.12 +# hudi-spark3.5-bundle_2.12 +"-Dscala-2.12 -Dspark3.5 -pl hudi-spark-datasource/hudi-spark3.5.x,packaging/hudi-spark-bundle,packaging/hudi-cli-bundle -am" # For Spark 3.1, Scala 2.12: # All other modules and bundles using avro 1.8 "-Dscala-2.12 -Dspark3.1" @@ -79,7 +83,6 @@ declare -a ALL_VERSION_OPTS=( "-Dscala-2.12 -Dspark3 -pl packaging/hudi-spark-bundle -am" # for legacy bundle name hudi-spark3-bundle_2.12 # Upload Flink bundles (overwriting previous uploads) -"-Dscala-2.12 -Dflink1.13 -Davro.version=1.10.0 -pl packaging/hudi-flink-bundle -am" "-Dscala-2.12 -Dflink1.14 -Davro.version=1.10.0 -pl packaging/hudi-flink-bundle -am" "-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" diff --git a/scripts/release/validate_staged_bundles.sh b/scripts/release/validate_staged_bundles.sh index 866b8cee335b..1fc7b9f6e1c7 100755 --- a/scripts/release/validate_staged_bundles.sh +++ b/scripts/release/validate_staged_bundles.sh @@ -32,12 +32,12 @@ declare -a extensions=("-javadoc.jar" "-javadoc.jar.asc" "-javadoc.jar.md5" "-ja "-sources.jar.asc" "-sources.jar.md5" "-sources.jar.sha1" ".jar" ".jar.asc" ".jar.md5" ".jar.sha1" ".pom" ".pom.asc" ".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" +declare -a bundles=("hudi-aws-bundle" "hudi-cli-bundle_2.11" "hudi-cli-bundle_2.12" "hudi-datahub-sync-bundle" "hudi-flink1.14-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" -"hudi-utilities-bundle_2.11" "hudi-utilities-bundle_2.12" "hudi-utilities-slim-bundle_2.11" +"hudi-spark3.2-bundle_2.12" "hudi-spark3.3-bundle_2.12" "hudi-spark3.4-bundle_2.12" "hudi-spark3.5-bundle_2.12" "hudi-timeline-server-bundle" +"hudi-trino-bundle" "hudi-utilities-bundle_2.11" "hudi-utilities-bundle_2.12" "hudi-utilities-slim-bundle_2.11" "hudi-utilities-slim-bundle_2.12") NOW=$(date +%s)