diff --git a/docs/config.toml b/docs/config.toml index 1274b5418328..c84c33d9b622 100644 --- a/docs/config.toml +++ b/docs/config.toml @@ -14,7 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -baseURL = '//paimon.apache.org/docs/master' +baseURL = '//paimon.apache.org/docs/1.2' languageCode = 'en-us' title = 'Apache Paimon' enableGitInfo = false @@ -24,7 +24,7 @@ pygmentsUseClasses = true [params] # Flag whether this is a stable version or not. # Used for the quickstart page. - IsStable = false + IsStable = true # Flag to indicate whether an outdated warning should be shown. ShowOutDatedWarning = false @@ -34,14 +34,14 @@ pygmentsUseClasses = true # we change the version for the complete docs when forking of a release branch # etc. # The full version string as referenced in Maven (e.g. 1.2.1) - Version = "1.2-SNAPSHOT" + Version = "1.2.0" # For stable releases, leave the bugfix version out (e.g. 1.2). For snapshot # release this should be the same as the regular version - VersionTitle = "1.2-SNAPSHOT" + VersionTitle = "1.2" # The branch for this version of Apache Paimon - Branch = "master" + Branch = "1.2" # The most recent supported Apache Flink version FlinkVersion = "1.20" @@ -67,14 +67,14 @@ pygmentsUseClasses = true ["JavaDocs", "//paimon.apache.org/docs/master/api/java/"], ] - StableDocs = "https://paimon.apache.org/docs/1.0" + StableDocs = "https://paimon.apache.org/docs/1.2" PreviousDocs = [ ["master", "https://paimon.apache.org/docs/master"], - ["stable", "https://paimon.apache.org/docs/1.0"], + ["stable", "https://paimon.apache.org/docs/1.2"], + ["1.2", "https://paimon.apache.org/docs/1.2"], + ["1.1", "https://paimon.apache.org/docs/1.1"], ["1.0", "https://paimon.apache.org/docs/1.0"], - ["0.9", "https://paimon.apache.org/docs/0.9"], - ["0.8", "https://paimon.apache.org/docs/0.8"], ] BookSection = '/' diff --git a/docs/content/concepts/overview.md b/docs/content/concepts/overview.md index a55151ead55d..659e1f04d346 100644 --- a/docs/content/concepts/overview.md +++ b/docs/content/concepts/overview.md @@ -42,7 +42,7 @@ As shown in the architecture above: - batch insert/overwrite from offline data. **Ecosystem:** In addition to Apache Flink, Paimon also supports read by other computation -engines like Apache Hive, Apache Spark and Trino. +engines like Apache Spark, StarRocks, Apache Doris, Apache Hive and Trino. **Internal:** - Under the hood, Paimon stores the columnar files on the filesystem/object-store diff --git a/docs/content/flink/sql-query.md b/docs/content/flink/sql-query.md index 85940a36e833..c9745273584e 100644 --- a/docs/content/flink/sql-query.md +++ b/docs/content/flink/sql-query.md @@ -288,3 +288,20 @@ SELECT * FROM orders WHERE order_id=29495; SELECT * FROM orders WHERE catalog_id=1025 OR order_id=29495; ``` + +## Dedicated Split Generation + +When Paimon table snapshots contain large amount of source splits, Flink jobs reading from this table might endure long +initialization time or even OOM in JobManagers. In this case, you can configure `'scan.dedicated-split-generation' = 'true'` +to avoid such problem. This option would enable executing the source split generation process in a dedicated subtask +that runs on TaskManager, instead of in the source coordinator on the JobManager. + +Note that this feature could have some side effects on your Flink jobs. For example: + +1. It will change the DAG of the flink job, thus breaking checkpoint compatibility if enabled on an existing job. +2. It may lead to the Flink AdaptiveBatchScheduler inferring a small parallelism for the source reader operator. you can + configure `scan.infer-parallelism` to avoid this possible drawback. +3. The failover strategy of the Flink job would be forced into global failover instead of regional failover, given that + the dedicated source split generation task would be connected to all downstream subtasks. + +So please make sure these side effects are acceptable to you before enabling it. diff --git a/docs/content/maintenance/filesystems.md b/docs/content/maintenance/filesystems.md index 673fe1f073b4..4a0f5e6f3804 100644 --- a/docs/content/maintenance/filesystems.md +++ b/docs/content/maintenance/filesystems.md @@ -44,6 +44,7 @@ FileSystem pluggable jars for user to query tables from Spark/Hive side. | Tencent Cloud Object Storage | cosn:// | Y | | | Microsoft Azure Storage | abfs:// | Y | | | Huawei OBS | obs:// | Y | | +| Google Cloud Storage | gs:// | Y | | ## Dependency @@ -405,7 +406,7 @@ Download [paimon-gs-{{< version >}}.jar](https://repository.apache.org/snapshots {{< tab "Flink" >}} {{< hint info >}} -If you have already configured [oss access through Flink](https://nightlies.apache.org/flink/flink-docs-release-2.0/docs/deployment/filesystems/gcs/) (Via Flink FileSystem), +If you have already configured [gcs access through Flink](https://nightlies.apache.org/flink/flink-docs-release-2.0/docs/deployment/filesystems/gcs/) (Via Flink FileSystem), here you can skip the following configuration. {{< /hint >}} @@ -414,7 +415,7 @@ Put `paimon-gs-{{< version >}}.jar` into `lib` directory of your Flink home, and ```sql CREATE CATALOG my_catalog WITH ( 'type' = 'paimon', - 'warehouse' = 'oss:///', + 'warehouse' = 'gs:///', 'fs.gs.auth.type' = 'SERVICE_ACCOUNT_JSON_KEYFILE', 'fs.gs.auth.service.account.json.keyfile' = '/path/to/service-account-.json' ); diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index 020b03c0c812..d7b21f18f767 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -182,6 +182,12 @@ Double Ratio of the deleted rows in a data file to be forced compacted for append-only table. + +
compaction.force-rewrite-all-files
+ false + Boolean + Whether to force pick all files for a full compaction. Usually seen in a compaction task to external paths. +
compaction.force-up-level-0
false @@ -765,7 +771,7 @@
partition.sink-strategy
NONE

Enum

- This is only for partitioned unaware-buckets append table, and the purpose is to reduce small files and improve write performance. Through this repartitioning strategy to reduce the number of partitions written by each task to as few as possible.
  • none: Rebalanced or Forward partitioning, this is the default behavior, this strategy is suitable for the number of partitions you write in a batch is much smaller than write parallelism.
  • hash: Hash the partitions value, this strategy is suitable for the number of partitions you write in a batch is greater equals than write parallelism.


Possible values:
  • "NONE"
  • "HASH"
+ This is only for partitioned append table or postpone pk table, and the purpose is to reduce small files and improve write performance. Through this repartitioning strategy to reduce the number of partitions written by each task to as few as possible.
  • none: Rebalanced or Forward partitioning, this is the default behavior, this strategy is suitable for the number of partitions you write in a batch is much smaller than write parallelism.
  • hash: Hash the partitions value, this strategy is suitable for the number of partitions you write in a batch is greater equals than write parallelism.


Possible values:
  • "NONE"
  • "HASH"
partition.timestamp-formatter
diff --git a/docs/layouts/shortcodes/generated/flink_connector_configuration.html b/docs/layouts/shortcodes/generated/flink_connector_configuration.html index 8b80512fc4f3..fa9b8e4d9cc5 100644 --- a/docs/layouts/shortcodes/generated/flink_connector_configuration.html +++ b/docs/layouts/shortcodes/generated/flink_connector_configuration.html @@ -60,7 +60,7 @@
lookup.cache
AUTO

Enum

- The cache mode of lookup join.

Possible values:
  • "AUTO"
  • "FULL"
+ The cache mode of lookup join.

Possible values:
  • "AUTO"
  • "FULL"
  • "MEMORY"
lookup.dynamic-partition.refresh-interval
@@ -122,12 +122,24 @@ Boolean If true, it will add a compact coordinator and worker operator after the writer operator,in order to compact several changelog files (for primary key tables) or newly created data files (for unaware bucket tables) from the same partition into large ones, which can decrease the number of small files. + +
read.shuffle-bucket-with-partition
+ true + Boolean + Whether shuffle by partition and bucket when read. +
scan.bounded
(none) Boolean Bounded mode for Paimon consumer. By default, Paimon automatically selects bounded mode based on the mode of the Flink job. + +
scan.dedicated-split-generation
+ false + Boolean + If true, the split generation process would be performed during runtime on a Flink task, instead of on the JobManager during initialization phase. +
scan.infer-parallelism
true @@ -314,12 +326,6 @@ String Set the uid suffix for the source operators. After setting, the uid format is ${UID_PREFIX}_${TABLE_NAME}_${USER_UID_SUFFIX}. If the uid suffix is not set, flink will automatically generate the operator uid, which may be incompatible when the topology changes. - -
streaming-read.shuffle-bucket-with-partition
- true - Boolean - Whether shuffle by partition and bucket when streaming read. -
unaware-bucket.compaction.parallelism
(none) diff --git a/docs/static/img/architecture.png b/docs/static/img/architecture.png index 639447ecc02e..3eae75233b35 100644 Binary files a/docs/static/img/architecture.png and b/docs/static/img/architecture.png differ diff --git a/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java index e79d1593d5db..4c446dc4e570 100644 --- a/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java @@ -151,6 +151,13 @@ public class CoreOptions implements Serializable { + ExternalPathStrategy.SPECIFIC_FS + ", should be the prefix scheme of the external path, now supported are s3 and oss."); + public static final ConfigOption COMPACTION_FORCE_REWRITE_ALL_FILES = + key("compaction.force-rewrite-all-files") + .booleanType() + .defaultValue(false) + .withDescription( + "Whether to force pick all files for a full compaction. Usually seen in a compaction task to external paths."); + @ExcludeFromDocumentation("Internal use only") public static final ConfigOption PATH = key("path") @@ -1325,7 +1332,7 @@ public class CoreOptions implements Serializable { .withDescription( Description.builder() .text( - "This is only for partitioned unaware-buckets append table, and the purpose is to reduce small files and improve write performance." + "This is only for partitioned append table or postpone pk table, and the purpose is to reduce small files and improve write performance." + " Through this repartitioning strategy to reduce the number of partitions written by each task to as few as possible.") .list( text( @@ -2466,6 +2473,10 @@ public String externalSpecificFS() { return options.get(DATA_FILE_EXTERNAL_PATHS_SPECIFIC_FS); } + public Boolean forceRewriteAllFiles() { + return options.get(COMPACTION_FORCE_REWRITE_ALL_FILES); + } + public String partitionTimestampFormatter() { return options.get(PARTITION_TIMESTAMP_FORMATTER); } diff --git a/paimon-api/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-api/src/main/java/org/apache/paimon/rest/HttpClient.java index 8aba11d52cfc..7d0f07b80b00 100644 --- a/paimon-api/src/main/java/org/apache/paimon/rest/HttpClient.java +++ b/paimon-api/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -59,6 +59,7 @@ public class HttpClient implements RESTClient { .addInterceptor(new LoggingInterceptor()) .connectTimeout(Duration.ofMinutes(3)) .readTimeout(Duration.ofMinutes(3)) + .writeTimeout(Duration.ofMinutes(3)) .build(); private static final MediaType MEDIA_TYPE = MediaType.parse("application/json"); diff --git a/paimon-api/src/main/java/org/apache/paimon/utils/StringUtils.java b/paimon-api/src/main/java/org/apache/paimon/utils/StringUtils.java index 140d0750a5ec..ca32b50b6896 100644 --- a/paimon-api/src/main/java/org/apache/paimon/utils/StringUtils.java +++ b/paimon-api/src/main/java/org/apache/paimon/utils/StringUtils.java @@ -515,7 +515,17 @@ public static boolean isNumeric(final CharSequence cs) { return false; } final int sz = cs.length(); - for (int i = 0; i < sz; i++) { + int startIndex = 0; + + // Handle negative sign + if (sz > 0 && cs.charAt(0) == '-') { + if (sz == 1) { + return false; // Just a minus sign is not numeric + } + startIndex = 1; + } + + for (int i = startIndex; i < sz; i++) { if (!Character.isDigit(cs.charAt(i))) { return false; } diff --git a/paimon-api/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java b/paimon-api/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java index a4790583c5c3..7ad34253670b 100644 --- a/paimon-api/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java +++ b/paimon-api/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java @@ -21,8 +21,6 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.Iterators; import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; -import javax.annotation.Nullable; - import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collection; @@ -36,8 +34,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -78,20 +74,12 @@ public static ThreadPoolExecutor createCachedThreadPool( return executor; } - public static ScheduledExecutorService createScheduledThreadPool( - int threadNum, String namePrefix) { - return new ScheduledThreadPoolExecutor(threadNum, newDaemonThreadFactory(namePrefix)); - } - /** This method aims to parallel process tasks with memory control and sequentially. */ public static Iterable sequentialBatchedExecute( - ThreadPoolExecutor executor, + ExecutorService executor, Function> processor, List input, - @Nullable Integer queueSize) { - if (queueSize == null) { - queueSize = executor.getMaximumPoolSize(); - } + int queueSize) { if (queueSize <= 0) { throw new NegativeArraySizeException("queue size should not be negative"); } diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/BinaryStringUtils.java b/paimon-common/src/main/java/org/apache/paimon/utils/BinaryStringUtils.java index 43ab156f1777..ae306c3764ec 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/BinaryStringUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/BinaryStringUtils.java @@ -345,6 +345,15 @@ private static Timestamp fromMillisToTimestamp(long epoch, int precision) { default: throw new RuntimeException("Unsupported precision: " + precision); } + + // If nanoseconds is negative, remove a millisecond + // and calculate the nanosecond offset forwards instead + // as nanoseconds should always be a positive offset on top of the milliseconds. + if (nanosOfMillis < 0) { + nanosOfMillis = 1000000 + nanosOfMillis; + millis -= 1; + } + return Timestamp.fromEpochMillis(millis, nanosOfMillis); } diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/SemaphoredDelegatingExecutor.java b/paimon-common/src/main/java/org/apache/paimon/utils/SemaphoredDelegatingExecutor.java new file mode 100644 index 000000000000..bdbb23796b43 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/utils/SemaphoredDelegatingExecutor.java @@ -0,0 +1,184 @@ +/* + * 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.paimon.utils; + +import org.apache.paimon.shade.guava30.com.google.common.util.concurrent.ForwardingExecutorService; +import org.apache.paimon.shade.guava30.com.google.common.util.concurrent.Futures; + +import java.util.Collection; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +/** + * A {@link ForwardingExecutorService} to delegate tasks to limit the number of tasks executed + * concurrently. + */ +public class SemaphoredDelegatingExecutor extends ForwardingExecutorService { + + private final Semaphore queueingPermits; + private final ExecutorService executorDelegated; + private final int permitCount; + + public SemaphoredDelegatingExecutor( + ExecutorService executorDelegated, int permitCount, boolean fair) { + this.permitCount = permitCount; + this.queueingPermits = new Semaphore(permitCount, fair); + this.executorDelegated = executorDelegated; + } + + @Override + protected ExecutorService delegate() { + return this.executorDelegated; + } + + @Override + public List> invokeAll(Collection> tasks) { + throw new RuntimeException("Not implemented"); + } + + @Override + public List> invokeAll( + Collection> tasks, long timeout, TimeUnit unit) { + throw new RuntimeException("Not implemented"); + } + + @Override + public T invokeAny(Collection> tasks) { + throw new RuntimeException("Not implemented"); + } + + @Override + public T invokeAny(Collection> tasks, long timeout, TimeUnit unit) { + throw new RuntimeException("Not implemented"); + } + + @Override + public Future submit(Callable task) { + try { + this.queueingPermits.acquire(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return Futures.immediateFailedFuture(e); + } + + return super.submit(new CallableWithPermitRelease(task)); + } + + @Override + public Future submit(Runnable task, T result) { + try { + this.queueingPermits.acquire(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return Futures.immediateFailedFuture(e); + } + + return super.submit(new RunnableWithPermitRelease(task), result); + } + + @Override + public Future submit(Runnable task) { + try { + this.queueingPermits.acquire(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return Futures.immediateFailedFuture(e); + } + + return super.submit(new RunnableWithPermitRelease(task)); + } + + @Override + public void execute(Runnable command) { + try { + this.queueingPermits.acquire(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + + super.execute(new RunnableWithPermitRelease(command)); + } + + public int getAvailablePermits() { + return this.queueingPermits.availablePermits(); + } + + public int getWaitingCount() { + return this.queueingPermits.getQueueLength(); + } + + public int getPermitCount() { + return this.permitCount; + } + + @Override + public String toString() { + return "SemaphoredDelegatingExecutor{" + + "permitCount=" + + getPermitCount() + + ", available=" + + getAvailablePermits() + + ", waiting=" + + getWaitingCount() + + '}'; + } + + private class RunnableWithPermitRelease implements Runnable { + + private final Runnable delegated; + + RunnableWithPermitRelease(Runnable delegated) { + this.delegated = delegated; + } + + @Override + public void run() { + try { + this.delegated.run(); + } finally { + SemaphoredDelegatingExecutor.this.queueingPermits.release(); + } + } + } + + private class CallableWithPermitRelease implements Callable { + + private final Callable delegated; + + CallableWithPermitRelease(Callable delegated) { + this.delegated = delegated; + } + + @Override + public T call() throws Exception { + T result; + try { + result = this.delegated.call(); + } finally { + SemaphoredDelegatingExecutor.this.queueingPermits.release(); + } + + return result; + } + } +} diff --git a/paimon-common/src/test/java/org/apache/paimon/utils/BinaryStringUtilsTest.java b/paimon-common/src/test/java/org/apache/paimon/utils/BinaryStringUtilsTest.java new file mode 100644 index 000000000000..b3b360259c4d --- /dev/null +++ b/paimon-common/src/test/java/org/apache/paimon/utils/BinaryStringUtilsTest.java @@ -0,0 +1,130 @@ +/* + * 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.paimon.utils; + +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.Timestamp; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; +import org.junit.jupiter.params.provider.ValueSource; + +import java.time.DateTimeException; +import java.util.TimeZone; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link BinaryStringUtils}. */ +class BinaryStringUtilsTest { + @ParameterizedTest + @CsvSource({ + "0, 0, 0, 0", // Unix epoch + "86400, 0, 86400000, 0", // One day in seconds + "3600000, 3, 3600000, 0", // One hour in milliseconds + "3600000000, 6, 3600000, 0", // One hour in microseconds + "3600000000000, 9, 3600000, 0", // One hour in nanoseconds + "1609459200123456789, 9, 1609459200123, 456789", // 2021-01-01 00:00:00.123456789 UTC + "1609459200123456, 6, 1609459200123, 456000", // 2021-01-01 00:00:00.123456 UTC + "1609459200123, 3, 1609459200123, 000000", // 2021-01-01 00:00:00 UTC + "1609459200, 0, 1609459200000, 000000", // 2021-01-01 00:00:00 UTC + "-1, 0, -1000, 0", // One second before epoch + "-1000, 3, -1000, 0", // One second before epoch in milliseconds + "-1000000, 6, -1000, 0", // One second before epoch in microseconds + "-1000000000, 9, -1000, 0", // One second before epoch in nanoseconds + // One second and one nanosecond before epoch in nanoseconds + // The negative nanosecond gets flipped and the milliseconds decremented + "-1000000001, 9, -1001, 999999", + "-86400123456, 6, -86400124, 544000" + }) + void testToTimestamp(String input, int precision, long expectedMillis, int expectedNanos) { + BinaryString binaryInput = BinaryString.fromString(input); + Timestamp result = BinaryStringUtils.toTimestamp(binaryInput, precision); + + assertThat(result.getMillisecond()).isEqualTo(expectedMillis); + assertThat(result.getNanoOfMillisecond()).isEqualTo(expectedNanos); + } + + @ParameterizedTest + @ValueSource(ints = {1, 2, 4, 5, 7, 8, 10, -1}) + void testInvalidPrecisions(int precision) { + BinaryString input = BinaryString.fromString("1609459200"); + + assertThatThrownBy(() -> BinaryStringUtils.toTimestamp(input, precision)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Unsupported precision: " + precision); + } + + @Test + void testDateStringInput() { + // Test with date string input - should delegate to DateTimeUtils.parseTimestampData + BinaryString input = BinaryString.fromString("2021-01-01 12:30:45"); + Timestamp result = BinaryStringUtils.toTimestamp(input, 3); + + // Verify it's not null and has reasonable values + assertThat(result).isNotNull(); + assertThat(result.getMillisecond()).isEqualTo(1609504245000L); + } + + @Test + void testDateOnlyStringInput() { + // Test with date-only string input + BinaryString input = BinaryString.fromString("2021-01-01"); + Timestamp result = BinaryStringUtils.toTimestamp(input, 3); + + assertThat(result).isNotNull(); + assertThat(result.getMillisecond()).isEqualTo(1609459200000L); + } + + @Test + void testInvalidStringInput() { + // Test with invalid string input + BinaryString input = BinaryString.fromString("invalid-date"); + + assertThatThrownBy(() -> BinaryStringUtils.toTimestamp(input, 3)) + .isInstanceOf(DateTimeException.class); + } + + @Test + void testToTimestampWithTimeZone() { + // Test the timezone variant of toTimestamp method + BinaryString input = BinaryString.fromString("2021-01-01 12:30:45"); + TimeZone timeZone = TimeZone.getTimeZone("UTC"); + + Timestamp result = BinaryStringUtils.toTimestamp(input, 3, timeZone); + + assertThat(result).isNotNull(); + assertThat(result.getMillisecond()).isEqualTo(1609504245000L); + } + + @Test + void testToTimestampWithDifferentTimeZones() { + BinaryString input = BinaryString.fromString("2021-01-01 12:30:45"); + + Timestamp utcResult = BinaryStringUtils.toTimestamp(input, 3, TimeZone.getTimeZone("UTC")); + Timestamp estResult = + BinaryStringUtils.toTimestamp(input, 3, TimeZone.getTimeZone("America/New_York")); + + assertThat(utcResult).isNotNull(); + assertThat(estResult).isNotNull(); + // The results should be different due to timezone offset + assertThat(utcResult.getMillisecond()).isNotEqualTo(estResult.getMillisecond()); + } +} diff --git a/paimon-common/src/test/java/org/apache/paimon/utils/StringUtilsTest.java b/paimon-common/src/test/java/org/apache/paimon/utils/StringUtilsTest.java new file mode 100644 index 000000000000..10b56a97b7eb --- /dev/null +++ b/paimon-common/src/test/java/org/apache/paimon/utils/StringUtilsTest.java @@ -0,0 +1,445 @@ +/* + * 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.paimon.utils; + +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; +import org.junit.jupiter.params.provider.NullAndEmptySource; +import org.junit.jupiter.params.provider.ValueSource; + +import java.util.Arrays; +import java.util.List; +import java.util.Random; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link StringUtils}. */ +class StringUtilsTest { + + @Nested + class IsNullOrWhitespaceOnlyTests { + + @ParameterizedTest + @NullAndEmptySource + @ValueSource(strings = {" ", " ", "\t", "\n", "\r", " \t\n\r "}) + void testNullOrWhitespaceOnlyStrings(String input) { + assertThat(StringUtils.isNullOrWhitespaceOnly(input)).isTrue(); + } + + @ParameterizedTest + @ValueSource(strings = {"a", " a ", "hello", "hello world"}) + void testNonWhitespaceStrings(String input) { + assertThat(StringUtils.isNullOrWhitespaceOnly(input)).isFalse(); + } + } + + @Nested + class ByteToHexStringTests { + + @Test + void testByteToHexStringWithRange() { + byte[] bytes = {0x00, 0x0F, (byte) 0xFF, 0x12, 0x34}; + String result = StringUtils.byteToHexString(bytes, 1, 4); + assertThat(result).isEqualTo("0fff12"); + } + + @Test + void testByteToHexStringFullArray() { + byte[] bytes = {0x00, 0x0F, (byte) 0xFF}; + String result = StringUtils.byteToHexString(bytes); + assertThat(result).isEqualTo("000fff"); + } + + @Test + void testByteToHexStringEmptyRange() { + byte[] bytes = {0x00, 0x0F, (byte) 0xFF}; + String result = StringUtils.byteToHexString(bytes, 1, 1); + assertThat(result).isEmpty(); + } + + @Test + void testByteToHexStringNullArray() { + assertThatThrownBy(() -> StringUtils.byteToHexString(null, 0, 1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("bytes == null"); + } + + @Test + void testByteToHexStringAllValues() { + byte[] bytes = new byte[256]; + for (int i = 0; i < 256; i++) { + bytes[i] = (byte) i; + } + String result = StringUtils.byteToHexString(bytes); + assertThat(result).hasSize(512); // 256 bytes * 2 hex chars each + assertThat(result).startsWith("000102"); + assertThat(result).endsWith("fdfeff"); + } + } + + @Nested + class BytesToBinaryStringTests { + + @Test + void testBytesToBinaryString() { + byte[] bytes = {0x00, 0x0F, (byte) 0xFF}; + String result = StringUtils.bytesToBinaryString(bytes); + assertThat(result).isEqualTo("000000000000111111111111"); + } + + @Test + void testBytesToBinaryStringEmptyArray() { + byte[] bytes = {}; + String result = StringUtils.bytesToBinaryString(bytes); + assertThat(result).isEmpty(); + } + + @Test + void testBytesToBinaryStringSingleByte() { + byte[] bytes = {(byte) 0xAA}; // 10101010 + String result = StringUtils.bytesToBinaryString(bytes); + assertThat(result).isEqualTo("10101010"); + } + } + + @Nested + class GetRandomStringTests { + + @Test + void testGetRandomStringWithinRange() { + Random rnd = new Random(42); + String result = StringUtils.getRandomString(rnd, 5, 10); + assertThat(result.length()).isBetween(5, 10); + } + + @Test + void testGetRandomStringExactLength() { + Random rnd = new Random(42); + String result = StringUtils.getRandomString(rnd, 7, 7); + assertThat(result).hasSize(7); + } + + @Test + void testGetRandomStringWithCharRange() { + Random rnd = new Random(42); + String result = StringUtils.getRandomString(rnd, 10, 10, 'a', 'z'); + assertThat(result).hasSize(10); + for (char c : result.toCharArray()) { + assertThat(c).isBetween('a', 'z'); + } + } + + @Test + void testGetRandomStringMinLength() { + Random rnd = new Random(42); + String result = StringUtils.getRandomString(rnd, 0, 5); + assertThat(result.length()).isBetween(0, 5); + } + } + + @Nested + class RepeatTests { + + @ParameterizedTest + @CsvSource({"abc, 3, abcabcabc", "abc, 0, ''", "abc, 1, abc", "'', 5, ''"}) + void testRepeatValidCases(String input, int count, String expected) { + String result = StringUtils.repeat(input, count); + assertThat(result).isEqualTo(expected); + } + + @Test + void testRepeatNullString() { + assertThatThrownBy(() -> StringUtils.repeat(null, 3)) + .isInstanceOf(NullPointerException.class); + } + + @Test + void testRepeatNegativeCount() { + assertThatThrownBy(() -> StringUtils.repeat("abc", -1)) + .isInstanceOf(IllegalArgumentException.class); + } + + @Test + void testRepeatLargeString() { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < 1000; i++) { + sb.append("a"); + } + String input = sb.toString(); + String result = StringUtils.repeat(input, 2); + assertThat(result).hasSize(2000); + } + } + + @Nested + class ReplaceTests { + + @ParameterizedTest + @CsvSource({ + "aba, a, z, zbz", + "aba, a, '', b", + "abc, x, z, abc", + "'', a, z, ''", + "abc, '', z, abc" + }) + void testReplaceBasicCases( + String text, String search, String replacement, String expected) { + String result = StringUtils.replace(text, search, replacement); + assertThat(result).isEqualTo(expected); + } + + @ParameterizedTest + @CsvSource({ + "abaa, a, z, 1, zbaa", + "abaa, a, z, 2, zbza", + "abaa, a, z, -1, zbzz", + "aba, a, z, 0, aba" + }) + void testReplaceWithMaxCases( + String text, String search, String replacement, int max, String expected) { + String result = StringUtils.replace(text, search, replacement, max); + assertThat(result).isEqualTo(expected); + } + + @ParameterizedTest + @CsvSource( + value = {"null, a, z, null", "abc, null, z, abc", "aba, a, null, aba"}, + nullValues = "null") + void testReplaceWithNullValues( + String text, String search, String replacement, String expected) { + String result = StringUtils.replace(text, search, replacement); + assertThat(result).isEqualTo(expected); + } + } + + @Nested + class IsEmptyTests { + + @Test + void testEmptyOrNull() { + assertThat(StringUtils.isEmpty(null)).isTrue(); + assertThat(StringUtils.isEmpty("")).isTrue(); + } + + @ParameterizedTest + @ValueSource(strings = {" ", "a", "hello", " hello "}) + void testNonEmpty(String input) { + assertThat(StringUtils.isEmpty(input)).isFalse(); + } + + @Test + void testEmptyStringBuilder() { + StringBuilder sb = new StringBuilder(); + assertThat(StringUtils.isEmpty(sb)).isTrue(); + } + + @Test + void testNonEmptyStringBuilder() { + StringBuilder sb = new StringBuilder("test"); + assertThat(StringUtils.isEmpty(sb)).isFalse(); + } + } + + @Nested + class RandomNumericStringTests { + + @Test + void testRandomNumericStringLength() { + String result = StringUtils.randomNumericString(5); + assertThat(result).hasSize(5); + } + + @Test + void testRandomNumericStringContainsOnlyDigits() { + String result = StringUtils.randomNumericString(10); + assertThat(result).matches("\\d+"); + } + + @Test + void testRandomNumericStringZeroLength() { + String result = StringUtils.randomNumericString(0); + assertThat(result).isEmpty(); + } + + @Test + void testRandomNumericStringLargeLength() { + String result = StringUtils.randomNumericString(100); + assertThat(result).hasSize(100); + assertThat(result).matches("\\d+"); + } + } + + @Nested + class SplitTests { + + @Test + void testSplitBasicCases() { + assertThat(StringUtils.split("ab:cd:ef", ":")).containsExactly("ab", "cd", "ef"); + assertThat(StringUtils.split("abc def", " ")).containsExactly("abc", "def"); + assertThat(StringUtils.split("abc def", " ")).containsExactly("abc", "def"); + assertThat(StringUtils.split("a,b,c", ",")).containsExactly("a", "b", "c"); + } + + @Test + void testSplitEdgeCases() { + assertThat(StringUtils.split(null, ":")).isNull(); + assertThat(StringUtils.split("", ":")).isEmpty(); + assertThat(StringUtils.split("abc def", null)).containsExactly("abc", "def"); + } + + @Test + void testSplitWithMax() { + String[] result = StringUtils.split("a:b:c:d", ":", 2, false); + assertThat(result).containsExactly("a", "b:c:d"); + } + + @Test + void testSplitPreserveAllTokens() { + String[] result = StringUtils.split("a::b", ":", -1, true); + assertThat(result).containsExactly("a", "", "b"); + } + } + + @Nested + class JoinTests { + + @Test + void testJoinIterableBasicCases() { + assertThat(StringUtils.join(Arrays.asList("a", "b", "c"), ",")).isEqualTo("a,b,c"); + assertThat(StringUtils.join(Arrays.asList("a", "b", "c"), null)).isEqualTo("abc"); + assertThat(StringUtils.join(Arrays.asList("single"), ",")).isEqualTo("single"); + assertThat(StringUtils.join(Arrays.asList("a", null, "c"), ",")).isEqualTo("a,,c"); + } + + @Test + void testJoinIterableEdgeCases() { + assertThat(StringUtils.join((Iterable) null, ",")).isNull(); + assertThat(StringUtils.join(Arrays.asList(), ",")).isEmpty(); + } + + @Test + void testJoinIterator() { + List items = Arrays.asList("x", "y", "z"); + String result = StringUtils.join(items.iterator(), "-"); + assertThat(result).isEqualTo("x-y-z"); + } + + @Test + void testJoinNullIterator() { + String result = StringUtils.join((java.util.Iterator) null, ","); + assertThat(result).isNull(); + } + } + + @Nested + class QuoteTests { + + @ParameterizedTest + @CsvSource({"hello, `hello`", "'', ``", "hello world!, `hello world!`"}) + void testQuote(String input, String expected) { + String result = StringUtils.quote(input); + assertThat(result).isEqualTo(expected); + } + } + + @Nested + class ToLowerCaseIfNeedTests { + + @ParameterizedTest + @CsvSource({ + "HELLO, true, HELLO", + "HELLO, false, hello", + "hello, false, hello", + "HeLLo, false, hello" + }) + void testToLowerCaseIfNeed(String input, boolean caseSensitive, String expected) { + String result = StringUtils.toLowerCaseIfNeed(input, caseSensitive); + assertThat(result).isEqualTo(expected); + } + } + + @Nested + class IsNumericTests { + + @Test + void testIsNumericNull() { + assertThat(StringUtils.isNumeric(null)).isFalse(); + } + + @ParameterizedTest + @ValueSource( + strings = { + "0", + "1", + "123", + "999", + "-1", + "-123", + "-999", + "-0", + "123456789012345", + "-123456789012345" + }) + void testIsNumericValidNumbers(String input) { + assertThat(StringUtils.isNumeric(input)).isTrue(); + } + + @ParameterizedTest + @ValueSource( + strings = {"", " ", "abc", "12.3", "12a", "a12", " 12", "12 ", "+12", "-", "1 23"}) + void testIsNumericInvalidNumbers(String input) { + assertThat(StringUtils.isNumeric(input)).isFalse(); + } + } + + @Nested + class EdgeCaseTests { + + @Test + void testLargeStringOperations() { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < 10000; i++) { + sb.append("a"); + } + String largeString = sb.toString(); + assertThat(StringUtils.isEmpty(largeString)).isFalse(); + assertThat(StringUtils.isNullOrWhitespaceOnly(largeString)).isFalse(); + } + + @Test + void testUnicodeCharacters() { + String unicode = "Hello δΈ–η•Œ 🌍"; + assertThat(StringUtils.isEmpty(unicode)).isFalse(); + assertThat(StringUtils.isNullOrWhitespaceOnly(unicode)).isFalse(); + assertThat(StringUtils.quote(unicode)).isEqualTo("`Hello δΈ–η•Œ 🌍`"); + } + + @Test + void testSpecialWhitespaceCharacters() { + // Test various Unicode whitespace characters that are recognized by + // Character.isWhitespace() + String specialWhitespace = + "\u0009\u000B\u000C\u001C\u001D\u001E\u001F"; // Tab, VT, FF, FS, GS, RS, US + assertThat(StringUtils.isNullOrWhitespaceOnly(specialWhitespace)).isTrue(); + } + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java index 81039da369c7..be4416af9b3c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java @@ -177,12 +177,14 @@ public AbstractFileStoreWrite newWrite( if (options.bucket() == BucketMode.POSTPONE_BUCKET) { return new PostponeBucketFileStoreWrite( fileIO, + pathFactory(), schema, commitUser, partitionType, keyType, valueType, this::pathFactory, + newReaderFactoryBuilder(), snapshotManager(), newScan(ScanType.FOR_WRITE).withManifestCacheFilter(manifestFilter), options, diff --git a/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java b/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java index 694937c88f84..3534ac4e9eef 100644 --- a/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java @@ -23,7 +23,6 @@ import org.apache.paimon.compact.CompactManager; import org.apache.paimon.compression.CompressOptions; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.data.serializer.InternalRowSerializer; import org.apache.paimon.disk.IOManager; import org.apache.paimon.disk.RowBuffer; import org.apache.paimon.fileindex.FileIndexOptions; @@ -45,14 +44,15 @@ import org.apache.paimon.utils.BatchRecordWriter; import org.apache.paimon.utils.CommitIncrement; import org.apache.paimon.utils.IOFunction; -import org.apache.paimon.utils.IOUtils; import org.apache.paimon.utils.LongCounter; import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.RecordWriter; +import org.apache.paimon.utils.SinkWriter; +import org.apache.paimon.utils.SinkWriter.BufferedSinkWriter; +import org.apache.paimon.utils.SinkWriter.DirectSinkWriter; import javax.annotation.Nullable; -import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -72,7 +72,7 @@ public class AppendOnlyWriter implements BatchRecordWriter, MemoryOwner { private final RowType writeSchema; private final DataFilePathFactory pathFactory; private final CompactManager compactManager; - private final IOFunction, RecordReaderIterator> bucketFileRead; + private final IOFunction, RecordReaderIterator> dataFileRead; private final boolean forceCompact; private final boolean asyncFileWrite; private final boolean statsDenseStore; @@ -80,17 +80,17 @@ public class AppendOnlyWriter implements BatchRecordWriter, MemoryOwner { private final List deletedFiles; private final List compactBefore; private final List compactAfter; - @Nullable private CompactDeletionFile compactDeletionFile; private final LongCounter seqNumCounter; private final String fileCompression; private final CompressOptions spillCompression; - private SinkWriter sinkWriter; private final SimpleColStatsCollector.Factory[] statsCollectors; @Nullable private final IOManager ioManager; private final FileIndexOptions fileIndexOptions; + private final MemorySize maxDiskSize; + @Nullable private CompactDeletionFile compactDeletionFile; + private SinkWriter sinkWriter; private MemorySegmentPool memorySegmentPool; - private final MemorySize maxDiskSize; public AppendOnlyWriter( FileIO fileIO, @@ -101,7 +101,7 @@ public AppendOnlyWriter( RowType writeSchema, long maxSequenceNumber, CompactManager compactManager, - IOFunction, RecordReaderIterator> bucketFileRead, + IOFunction, RecordReaderIterator> dataFileRead, boolean forceCompact, DataFilePathFactory pathFactory, @Nullable CommitIncrement increment, @@ -121,7 +121,7 @@ public AppendOnlyWriter( this.writeSchema = writeSchema; this.pathFactory = pathFactory; this.compactManager = compactManager; - this.bucketFileRead = bucketFileRead; + this.dataFileRead = dataFileRead; this.forceCompact = forceCompact; this.asyncFileWrite = asyncFileWrite; this.statsDenseStore = statsDenseStore; @@ -139,8 +139,8 @@ public AppendOnlyWriter( this.sinkWriter = useWriteBuffer - ? new BufferedSinkWriter(spillable, maxDiskSize, spillCompression) - : new DirectSinkWriter(); + ? createBufferedSinkWriter(spillable) + : new DirectSinkWriter<>(this::createRollingRowWriter); if (increment != null) { newFiles.addAll(increment.newFilesIncrement().newFiles()); @@ -151,6 +151,18 @@ public AppendOnlyWriter( } } + private BufferedSinkWriter createBufferedSinkWriter(boolean spillable) { + return new BufferedSinkWriter<>( + this::createRollingRowWriter, + t -> t, + t -> t, + ioManager, + writeSchema, + spillable, + maxDiskSize, + spillCompression); + } + @Override public void write(InternalRow rowData) throws Exception { Preconditions.checkArgument( @@ -178,7 +190,7 @@ public void writeBundle(BundleRecords bundle) throws Exception { write(row); } } else { - ((DirectSinkWriter) sinkWriter).writeBundle(bundle); + ((DirectSinkWriter) sinkWriter).writeBundle(bundle); } } @@ -252,16 +264,16 @@ public void close() throws Exception { } public void toBufferedWriter() throws Exception { - if (sinkWriter != null && !sinkWriter.bufferSpillableWriter() && bucketFileRead != null) { + if (sinkWriter != null && !sinkWriter.bufferSpillableWriter() && dataFileRead != null) { // fetch the written results List files = sinkWriter.flush(); sinkWriter.close(); - sinkWriter = new BufferedSinkWriter(true, maxDiskSize, spillCompression); + sinkWriter = createBufferedSinkWriter(true); sinkWriter.setMemoryPool(memorySegmentPool); // rewrite small files - try (RecordReaderIterator reader = bucketFileRead.apply(files)) { + try (RecordReaderIterator reader = dataFileRead.apply(files)) { while (reader.hasNext()) { sinkWriter.write(reader.next()); } @@ -356,7 +368,7 @@ public void flushMemory() throws Exception { @VisibleForTesting public RowBuffer getWriteBuffer() { if (sinkWriter instanceof BufferedSinkWriter) { - return ((BufferedSinkWriter) sinkWriter).writeBuffer; + return ((BufferedSinkWriter) sinkWriter).rowBuffer(); } else { return null; } @@ -366,176 +378,4 @@ public RowBuffer getWriteBuffer() { List getNewFiles() { return newFiles; } - - /** Internal interface to Sink Data from input. */ - private interface SinkWriter { - - boolean write(InternalRow data) throws IOException; - - List flush() throws IOException; - - boolean flushMemory() throws IOException; - - long memoryOccupancy(); - - void close(); - - void setMemoryPool(MemorySegmentPool memoryPool); - - boolean bufferSpillableWriter(); - } - - /** - * Directly sink data to file, no memory cache here, use OrcWriter/ParquetWrite/etc directly - * write data. May cause out-of-memory. - */ - private class DirectSinkWriter implements SinkWriter { - - private RowDataRollingFileWriter writer; - - @Override - public boolean write(InternalRow data) throws IOException { - if (writer == null) { - writer = createRollingRowWriter(); - } - writer.write(data); - return true; - } - - public void writeBundle(BundleRecords bundle) throws IOException { - if (writer == null) { - writer = createRollingRowWriter(); - } - writer.writeBundle(bundle); - } - - @Override - public List flush() throws IOException { - List flushedFiles = new ArrayList<>(); - if (writer != null) { - writer.close(); - flushedFiles.addAll(writer.result()); - writer = null; - } - return flushedFiles; - } - - @Override - public boolean flushMemory() throws IOException { - return false; - } - - @Override - public long memoryOccupancy() { - return 0; - } - - @Override - public void close() { - if (writer != null) { - writer.abort(); - writer = null; - } - } - - @Override - public void setMemoryPool(MemorySegmentPool memoryPool) { - // do nothing - } - - @Override - public boolean bufferSpillableWriter() { - return false; - } - } - - /** - * Use buffered writer, segment pooled from segment pool. When spillable, may delay checkpoint - * acknowledge time. When non-spillable, may cause too many small files. - */ - private class BufferedSinkWriter implements SinkWriter { - - private final boolean spillable; - - private final MemorySize maxDiskSize; - - private final CompressOptions compression; - - private RowBuffer writeBuffer; - - private BufferedSinkWriter( - boolean spillable, MemorySize maxDiskSize, CompressOptions compression) { - this.spillable = spillable; - this.maxDiskSize = maxDiskSize; - this.compression = compression; - } - - @Override - public boolean write(InternalRow data) throws IOException { - return writeBuffer.put(data); - } - - @Override - public List flush() throws IOException { - List flushedFiles = new ArrayList<>(); - if (writeBuffer != null) { - writeBuffer.complete(); - RowDataRollingFileWriter writer = createRollingRowWriter(); - IOException exception = null; - try (RowBuffer.RowBufferIterator iterator = writeBuffer.newIterator()) { - while (iterator.advanceNext()) { - writer.write(iterator.getRow()); - } - } catch (IOException e) { - exception = e; - } finally { - if (exception != null) { - IOUtils.closeQuietly(writer); - // cleanup code that might throw another exception - throw exception; - } - writer.close(); - } - flushedFiles.addAll(writer.result()); - // reuse writeBuffer - writeBuffer.reset(); - } - return flushedFiles; - } - - @Override - public long memoryOccupancy() { - return writeBuffer.memoryOccupancy(); - } - - @Override - public void close() { - if (writeBuffer != null) { - writeBuffer.reset(); - writeBuffer = null; - } - } - - @Override - public void setMemoryPool(MemorySegmentPool memoryPool) { - writeBuffer = - RowBuffer.getBuffer( - ioManager, - memoryPool, - new InternalRowSerializer(writeSchema), - spillable, - maxDiskSize, - compression); - } - - @Override - public boolean bufferSpillableWriter() { - return spillable; - } - - @Override - public boolean flushMemory() throws IOException { - return writeBuffer.flushMemory(); - } - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/append/BucketedAppendCompactManager.java b/paimon-core/src/main/java/org/apache/paimon/append/BucketedAppendCompactManager.java index d5c47c3b883e..a27a40e8a9e1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/append/BucketedAppendCompactManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/append/BucketedAppendCompactManager.java @@ -60,6 +60,7 @@ public class BucketedAppendCompactManager extends CompactFutureManager { private final PriorityQueue toCompact; private final int minFileNum; private final long targetFileSize; + private final boolean forceRewriteAllFiles; private final CompactRewriter rewriter; private List compacting; @@ -72,6 +73,7 @@ public BucketedAppendCompactManager( @Nullable DeletionVectorsMaintainer dvMaintainer, int minFileNum, long targetFileSize, + boolean forceRewriteAllFiles, CompactRewriter rewriter, @Nullable CompactionMetrics.Reporter metricsReporter) { this.executor = executor; @@ -80,6 +82,7 @@ public BucketedAppendCompactManager( this.toCompact.addAll(restored); this.minFileNum = minFileNum; this.targetFileSize = targetFileSize; + this.forceRewriteAllFiles = forceRewriteAllFiles; this.rewriter = rewriter; this.metricsReporter = metricsReporter; } @@ -98,9 +101,10 @@ private void triggerFullCompaction() { taskFuture == null, "A compaction task is still running while the user " + "forces a new compaction. This is unexpected."); - // if deletion vector enables, always trigger compaction. - if (toCompact.isEmpty() - || (dvMaintainer == null && toCompact.size() < FULL_COMPACT_MIN_FILE)) { + // if all files are force picked or deletion vector enables, always trigger compaction. + if (!forceRewriteAllFiles + && (toCompact.isEmpty() + || (dvMaintainer == null && toCompact.size() < FULL_COMPACT_MIN_FILE))) { return; } @@ -114,6 +118,7 @@ private void triggerFullCompaction() { dvMaintainer, toCompact, targetFileSize, + forceRewriteAllFiles, rewriter, metricsReporter)); recordCompactionsQueuedRequest(); @@ -238,25 +243,28 @@ public static class FullCompactTask extends CompactTask { private final DeletionVectorsMaintainer dvMaintainer; private final LinkedList toCompact; private final long targetFileSize; + private final boolean forceRewriteAllFiles; private final CompactRewriter rewriter; public FullCompactTask( DeletionVectorsMaintainer dvMaintainer, Collection inputs, long targetFileSize, + boolean forceRewriteAllFiles, CompactRewriter rewriter, @Nullable CompactionMetrics.Reporter metricsReporter) { super(metricsReporter); this.dvMaintainer = dvMaintainer; this.toCompact = new LinkedList<>(inputs); this.targetFileSize = targetFileSize; + this.forceRewriteAllFiles = forceRewriteAllFiles; this.rewriter = rewriter; } @Override protected CompactResult doCompact() throws Exception { // remove large files - while (!toCompact.isEmpty()) { + while (!forceRewriteAllFiles && !toCompact.isEmpty()) { DataFileMeta file = toCompact.peekFirst(); // the data file with deletion file always need to be compacted. if (file.fileSize() >= targetFileSize && !hasDeletionFile(file)) { @@ -281,7 +289,8 @@ protected CompactResult doCompact() throws Exception { small++; } } - if (small > big && toCompact.size() >= FULL_COMPACT_MIN_FILE) { + if (forceRewriteAllFiles + || (small > big && toCompact.size() >= FULL_COMPACT_MIN_FILE)) { return compact(null, toCompact, rewriter); } else { return result(emptyList(), emptyList()); diff --git a/paimon-core/src/main/java/org/apache/paimon/compact/CompactUnit.java b/paimon-core/src/main/java/org/apache/paimon/compact/CompactUnit.java index 9b0fdd26ad03..ae367f9171de 100644 --- a/paimon-core/src/main/java/org/apache/paimon/compact/CompactUnit.java +++ b/paimon-core/src/main/java/org/apache/paimon/compact/CompactUnit.java @@ -25,31 +25,40 @@ import java.util.List; /** A files unit for compaction. */ -public interface CompactUnit { +public class CompactUnit { - int outputLevel(); + private final int outputLevel; + private final List files; + private final boolean fileRewrite; - List files(); + public CompactUnit(int outputLevel, List files, boolean fileRewrite) { + this.outputLevel = outputLevel; + this.files = files; + this.fileRewrite = fileRewrite; + } + + public int outputLevel() { + return outputLevel; + } + + public List files() { + return files; + } + + public boolean fileRewrite() { + return fileRewrite; + } - static CompactUnit fromLevelRuns(int outputLevel, List runs) { + public static CompactUnit fromLevelRuns(int outputLevel, List runs) { List files = new ArrayList<>(); for (LevelSortedRun run : runs) { files.addAll(run.run().files()); } - return fromFiles(outputLevel, files); + return fromFiles(outputLevel, files, false); } - static CompactUnit fromFiles(int outputLevel, List files) { - return new CompactUnit() { - @Override - public int outputLevel() { - return outputLevel; - } - - @Override - public List files() { - return files; - } - }; + public static CompactUnit fromFiles( + int outputLevel, List files, boolean fileRewrite) { + return new CompactUnit(outputLevel, files, fileRewrite); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/crosspartition/GlobalIndexAssigner.java b/paimon-core/src/main/java/org/apache/paimon/crosspartition/GlobalIndexAssigner.java index 892e27e96611..fe14048310a4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/crosspartition/GlobalIndexAssigner.java +++ b/paimon-core/src/main/java/org/apache/paimon/crosspartition/GlobalIndexAssigner.java @@ -28,11 +28,11 @@ import org.apache.paimon.data.serializer.RowCompactedSerializer; import org.apache.paimon.disk.IOManager; import org.apache.paimon.disk.RowBuffer; -import org.apache.paimon.lookup.BulkLoader; -import org.apache.paimon.lookup.RocksDBOptions; -import org.apache.paimon.lookup.RocksDBState; -import org.apache.paimon.lookup.RocksDBStateFactory; -import org.apache.paimon.lookup.RocksDBValueState; +import org.apache.paimon.lookup.rocksdb.RocksDBBulkLoader; +import org.apache.paimon.lookup.rocksdb.RocksDBOptions; +import org.apache.paimon.lookup.rocksdb.RocksDBState; +import org.apache.paimon.lookup.rocksdb.RocksDBStateFactory; +import org.apache.paimon.lookup.rocksdb.RocksDBValueState; import org.apache.paimon.memory.HeapMemorySegmentPool; import org.apache.paimon.options.MemorySize; import org.apache.paimon.options.Options; @@ -71,7 +71,7 @@ import java.util.function.Function; import java.util.stream.IntStream; -import static org.apache.paimon.lookup.RocksDBOptions.BLOCK_CACHE_SIZE; +import static org.apache.paimon.lookup.rocksdb.RocksDBOptions.BLOCK_CACHE_SIZE; import static org.apache.paimon.utils.ListUtils.pickRandomly; import static org.apache.paimon.utils.Preconditions.checkArgument; @@ -210,14 +210,14 @@ public CloseableIterator endBoostrapWithoutEmit(boolean isEndInput) bootstrapRecords.complete(); boolean isEmpty = true; if (bootstrapKeys.size() > 0) { - BulkLoader bulkLoader = keyIndex.createBulkLoader(); + RocksDBBulkLoader bulkLoader = keyIndex.createBulkLoader(); MutableObjectIterator keyIterator = bootstrapKeys.sortedIterator(); BinaryRow row = new BinaryRow(2); try { while ((row = keyIterator.next(row)) != null) { bulkLoader.write(row.getBinary(0), row.getBinary(1)); } - } catch (BulkLoader.WriteException e) { + } catch (RocksDBBulkLoader.WriteException e) { throw new RuntimeException( "Exception in bulkLoad, the most suspicious reason is that " + "your data contains duplicates, please check your sink table. " diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/BulkLoader.java b/paimon-core/src/main/java/org/apache/paimon/lookup/BulkLoader.java index 0bcdbbb5d29e..f0ea474f527e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/lookup/BulkLoader.java +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/BulkLoader.java @@ -18,106 +18,13 @@ package org.apache.paimon.lookup; -import org.rocksdb.ColumnFamilyHandle; -import org.rocksdb.EnvOptions; -import org.rocksdb.IngestExternalFileOptions; -import org.rocksdb.Options; -import org.rocksdb.RocksDB; -import org.rocksdb.RocksDBException; -import org.rocksdb.SstFileWriter; -import org.rocksdb.TtlDB; +/** Bulk loader for {@link State}, incoming keys must be sorted, and there must be no repetition. */ +public interface BulkLoader { -import java.io.File; -import java.util.ArrayList; -import java.util.List; -import java.util.UUID; - -/** Bulk loader for RocksDB. */ -public class BulkLoader { - - private final String uuid = UUID.randomUUID().toString(); - - private final ColumnFamilyHandle columnFamily; - private final String path; - private final RocksDB db; - private final boolean isTtlEnabled; - private final Options options; - private final List files = new ArrayList<>(); - private final int currentTimeSeconds; - - private SstFileWriter writer = null; - private int sstIndex = 0; - private long recordNum = 0; - - public BulkLoader(RocksDB db, Options options, ColumnFamilyHandle columnFamily, String path) { - this.db = db; - this.isTtlEnabled = db instanceof TtlDB; - this.options = options; - this.columnFamily = columnFamily; - this.path = path; - this.currentTimeSeconds = (int) (System.currentTimeMillis() / 1000); - } - - public void write(byte[] key, byte[] value) throws WriteException { - try { - if (writer == null) { - writer = new SstFileWriter(new EnvOptions(), options); - String path = new File(this.path, "sst-" + uuid + "-" + (sstIndex++)).getPath(); - writer.open(path); - files.add(path); - } - - if (isTtlEnabled) { - value = appendTimestamp(value); - } - - try { - writer.put(key, value); - } catch (RocksDBException e) { - throw new WriteException(e); - } - - recordNum++; - if (recordNum % 1000 == 0 && writer.fileSize() >= options.targetFileSizeBase()) { - writer.finish(); - writer.close(); - writer = null; - recordNum = 0; - } - } catch (RocksDBException e) { - throw new RuntimeException(e); - } - } - - private byte[] appendTimestamp(byte[] value) { - byte[] newValue = new byte[value.length + 4]; - System.arraycopy(value, 0, newValue, 0, value.length); - newValue[value.length] = (byte) (currentTimeSeconds & 0xff); - newValue[value.length + 1] = (byte) ((currentTimeSeconds >> 8) & 0xff); - newValue[value.length + 2] = (byte) ((currentTimeSeconds >> 16) & 0xff); - newValue[value.length + 3] = (byte) ((currentTimeSeconds >> 24) & 0xff); - return newValue; - } - - public void finish() { - try { - if (writer != null) { - writer.finish(); - writer.close(); - } - - if (files.size() > 0) { - IngestExternalFileOptions ingestOptions = new IngestExternalFileOptions(); - db.ingestExternalFile(columnFamily, files, ingestOptions); - ingestOptions.close(); - } - } catch (RocksDBException e) { - throw new RuntimeException(e); - } - } + void finish(); /** Exception during writing. */ - public static class WriteException extends Exception { + class WriteException extends Exception { public WriteException(Throwable cause) { super(cause); } diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/ByteArray.java b/paimon-core/src/main/java/org/apache/paimon/lookup/ByteArray.java new file mode 100644 index 000000000000..95a279d20f83 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/ByteArray.java @@ -0,0 +1,61 @@ +/* + * 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.paimon.lookup; + +import org.apache.paimon.utils.SortUtil; + +import org.jetbrains.annotations.NotNull; + +import java.util.Arrays; + +/** A class wraps byte[] to implement equals and hashCode. */ +public class ByteArray implements Comparable { + + public final byte[] bytes; + + public ByteArray(byte[] bytes) { + this.bytes = bytes; + } + + @Override + public int hashCode() { + return Arrays.hashCode(bytes); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ByteArray byteArray = (ByteArray) o; + return Arrays.equals(bytes, byteArray.bytes); + } + + public static ByteArray wrapBytes(byte[] bytes) { + return new ByteArray(bytes); + } + + @Override + public int compareTo(@NotNull ByteArray o) { + return SortUtil.compareBinary(bytes, o.bytes); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/ListBulkLoader.java b/paimon-core/src/main/java/org/apache/paimon/lookup/ListBulkLoader.java new file mode 100644 index 000000000000..6496d5186b8a --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/ListBulkLoader.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.lookup; + +import java.util.List; + +/** List bulk loader to load key values, incoming keys must be sorted. */ +public interface ListBulkLoader extends BulkLoader { + + void write(byte[] key, List value) throws WriteException; +} diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/ListState.java b/paimon-core/src/main/java/org/apache/paimon/lookup/ListState.java new file mode 100644 index 000000000000..8e3d7c2c8b54 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/ListState.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.lookup; + +import java.io.IOException; +import java.util.List; + +/** {@link State} interface for list state in Operations. */ +public interface ListState extends State { + + void add(K key, V value) throws IOException; + + List get(K key) throws IOException; + + ListBulkLoader createBulkLoader(); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/SetState.java b/paimon-core/src/main/java/org/apache/paimon/lookup/SetState.java new file mode 100644 index 000000000000..7874fb717876 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/SetState.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.lookup; + +import java.io.IOException; +import java.util.List; + +/** + * {@link State} interface for set state in Operations, the values must be sorted by byte array to + * be returned. + */ +public interface SetState extends State { + + List get(K key) throws IOException; + + void retract(K key, V value) throws IOException; + + void add(K key, V value) throws IOException; +} diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/State.java b/paimon-core/src/main/java/org/apache/paimon/lookup/State.java new file mode 100644 index 000000000000..e4174a9e21cd --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/State.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.lookup; + +import java.io.IOException; + +/** Interface that different types of state must implement. */ +public interface State { + + byte[] serializeKey(K key) throws IOException; + + byte[] serializeValue(V value) throws IOException; + + V deserializeValue(byte[] valueBytes) throws IOException; +} diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/StateFactory.java b/paimon-core/src/main/java/org/apache/paimon/lookup/StateFactory.java new file mode 100644 index 000000000000..478c6f9f3bd0 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/StateFactory.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.lookup; + +import org.apache.paimon.data.serializer.Serializer; + +import java.io.Closeable; +import java.io.IOException; + +/** State factory to create {@link State}. */ +public interface StateFactory extends Closeable { + + ValueState valueState( + String name, + Serializer keySerializer, + Serializer valueSerializer, + long lruCacheSize) + throws IOException; + + SetState setState( + String name, + Serializer keySerializer, + Serializer valueSerializer, + long lruCacheSize) + throws IOException; + + ListState listState( + String name, + Serializer keySerializer, + Serializer valueSerializer, + long lruCacheSize) + throws IOException; + + boolean preferBulkLoad(); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/ValueBulkLoader.java b/paimon-core/src/main/java/org/apache/paimon/lookup/ValueBulkLoader.java new file mode 100644 index 000000000000..eef737078f24 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/ValueBulkLoader.java @@ -0,0 +1,25 @@ +/* + * 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.paimon.lookup; + +/** Value bulk loader to load key values, incoming keys must be sorted. */ +public interface ValueBulkLoader extends BulkLoader { + + void write(byte[] key, byte[] value) throws WriteException; +} diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/ValueState.java b/paimon-core/src/main/java/org/apache/paimon/lookup/ValueState.java new file mode 100644 index 000000000000..f04e456b3b87 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/ValueState.java @@ -0,0 +1,36 @@ +/* + * 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.paimon.lookup; + +import javax.annotation.Nullable; + +import java.io.IOException; + +/** {@link State} interface for single-value state. The value can be deleted or updated. */ +public interface ValueState extends State { + + @Nullable + V get(K key) throws IOException; + + void put(K key, V value) throws IOException; + + void delete(K key) throws IOException; + + ValueBulkLoader createBulkLoader(); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/memory/InMemoryListState.java b/paimon-core/src/main/java/org/apache/paimon/lookup/memory/InMemoryListState.java new file mode 100644 index 000000000000..992fbc1abd39 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/memory/InMemoryListState.java @@ -0,0 +1,77 @@ +/* + * 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.paimon.lookup.memory; + +import org.apache.paimon.data.serializer.Serializer; +import org.apache.paimon.lookup.ByteArray; +import org.apache.paimon.lookup.ListBulkLoader; +import org.apache.paimon.lookup.ListState; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.paimon.lookup.ByteArray.wrapBytes; + +/** In-memory list state. */ +public class InMemoryListState extends InMemoryState implements ListState { + + private final Map> values; + + public InMemoryListState(Serializer keySerializer, Serializer valueSerializer) { + super(keySerializer, valueSerializer); + this.values = new HashMap<>(); + } + + @Override + public void add(K key, V value) throws IOException { + byte[] keyBytes = serializeKey(key); + byte[] valueBytes = serializeValue(value); + values.computeIfAbsent(wrapBytes(keyBytes), k -> new ArrayList<>()).add(valueBytes); + } + + @Override + public List get(K key) throws IOException { + List list = this.values.get(wrapBytes(serializeKey(key))); + List result = new ArrayList<>(); + if (list != null) { + for (byte[] value : list) { + result.add(deserializeValue(value)); + } + } + return result; + } + + @Override + public ListBulkLoader createBulkLoader() { + return new ListBulkLoader() { + + @Override + public void write(byte[] key, List value) { + // copy the list, outside will reuse list + values.put(wrapBytes(key), new ArrayList<>(value)); + } + + @Override + public void finish() {} + }; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/memory/InMemorySetState.java b/paimon-core/src/main/java/org/apache/paimon/lookup/memory/InMemorySetState.java new file mode 100644 index 000000000000..f694a576431a --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/memory/InMemorySetState.java @@ -0,0 +1,69 @@ +/* + * 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.paimon.lookup.memory; + +import org.apache.paimon.data.serializer.Serializer; +import org.apache.paimon.lookup.ByteArray; +import org.apache.paimon.lookup.SetState; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; + +import static org.apache.paimon.lookup.ByteArray.wrapBytes; + +/** In-memory set state. */ +public class InMemorySetState extends InMemoryState implements SetState { + + private final Map> values; + + public InMemorySetState(Serializer keySerializer, Serializer valueSerializer) { + super(keySerializer, valueSerializer); + this.values = new HashMap<>(); + } + + @Override + public List get(K key) throws IOException { + Set set = values.get(wrapBytes(serializeKey(key))); + List result = new ArrayList<>(); + if (set != null) { + for (ByteArray value : set) { + result.add(deserializeValue(value.bytes)); + } + } + return result; + } + + @Override + public void retract(K key, V value) throws IOException { + values.get(wrapBytes(serializeKey(key))).remove(wrapBytes(serializeValue(value))); + } + + @Override + public void add(K key, V value) throws IOException { + byte[] keyBytes = serializeKey(key); + byte[] valueBytes = serializeValue(value); + values.computeIfAbsent(wrapBytes(keyBytes), k -> new TreeSet<>()) + .add(wrapBytes(valueBytes)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/memory/InMemoryState.java b/paimon-core/src/main/java/org/apache/paimon/lookup/memory/InMemoryState.java new file mode 100644 index 000000000000..6b79776a70ec --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/memory/InMemoryState.java @@ -0,0 +1,64 @@ +/* + * 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.paimon.lookup.memory; + +import org.apache.paimon.data.serializer.Serializer; +import org.apache.paimon.io.DataInputDeserializer; +import org.apache.paimon.io.DataOutputSerializer; +import org.apache.paimon.lookup.State; + +import java.io.IOException; + +/** In-memory state. */ +public abstract class InMemoryState implements State { + + protected final Serializer keySerializer; + protected final Serializer valueSerializer; + protected final DataOutputSerializer keyOutView; + protected final DataInputDeserializer valueInputView; + protected final DataOutputSerializer valueOutputView; + + public InMemoryState(Serializer keySerializer, Serializer valueSerializer) { + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + this.keyOutView = new DataOutputSerializer(32); + this.valueInputView = new DataInputDeserializer(); + this.valueOutputView = new DataOutputSerializer(32); + } + + @Override + public byte[] serializeKey(K key) throws IOException { + keyOutView.clear(); + keySerializer.serialize(key, keyOutView); + return keyOutView.getCopyOfBuffer(); + } + + @Override + public byte[] serializeValue(V value) throws IOException { + valueOutputView.clear(); + valueSerializer.serialize(value, valueOutputView); + return valueOutputView.getCopyOfBuffer(); + } + + @Override + public V deserializeValue(byte[] valueBytes) throws IOException { + valueInputView.setBuffer(valueBytes); + return valueSerializer.deserialize(valueInputView); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/memory/InMemoryStateFactory.java b/paimon-core/src/main/java/org/apache/paimon/lookup/memory/InMemoryStateFactory.java new file mode 100644 index 000000000000..77aa1cf17e5d --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/memory/InMemoryStateFactory.java @@ -0,0 +1,66 @@ +/* + * 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.paimon.lookup.memory; + +import org.apache.paimon.data.serializer.Serializer; +import org.apache.paimon.lookup.ListState; +import org.apache.paimon.lookup.SetState; +import org.apache.paimon.lookup.StateFactory; +import org.apache.paimon.lookup.ValueState; + +import java.io.IOException; + +/** Factory to create in-memory state. */ +public class InMemoryStateFactory implements StateFactory { + + @Override + public ValueState valueState( + String name, + Serializer keySerializer, + Serializer valueSerializer, + long lruCacheSize) { + return new InMemoryValueState<>(keySerializer, valueSerializer); + } + + @Override + public SetState setState( + String name, + Serializer keySerializer, + Serializer valueSerializer, + long lruCacheSize) { + return new InMemorySetState<>(keySerializer, valueSerializer); + } + + @Override + public ListState listState( + String name, + Serializer keySerializer, + Serializer valueSerializer, + long lruCacheSize) { + return new InMemoryListState<>(keySerializer, valueSerializer); + } + + @Override + public boolean preferBulkLoad() { + return false; + } + + @Override + public void close() throws IOException {} +} diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/memory/InMemoryValueState.java b/paimon-core/src/main/java/org/apache/paimon/lookup/memory/InMemoryValueState.java new file mode 100644 index 000000000000..07db364ef1b0 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/memory/InMemoryValueState.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.lookup.memory; + +import org.apache.paimon.data.serializer.Serializer; +import org.apache.paimon.lookup.ByteArray; +import org.apache.paimon.lookup.ValueBulkLoader; +import org.apache.paimon.lookup.ValueState; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.paimon.lookup.ByteArray.wrapBytes; + +/** In-memory value state. */ +public class InMemoryValueState extends InMemoryState implements ValueState { + + private final Map values; + + public InMemoryValueState(Serializer keySerializer, Serializer valueSerializer) { + super(keySerializer, valueSerializer); + this.values = new HashMap<>(); + } + + @Override + public @Nullable V get(K key) throws IOException { + byte[] bytes = values.get(wrapBytes(serializeKey(key))); + if (bytes == null) { + return null; + } + return deserializeValue(bytes); + } + + @Override + public void put(K key, V value) throws IOException { + values.put(wrapBytes(serializeKey(key)), serializeValue(value)); + } + + @Override + public void delete(K key) throws IOException { + values.remove(wrapBytes(serializeKey(key))); + } + + @Override + public ValueBulkLoader createBulkLoader() { + return new ValueBulkLoader() { + + @Override + public void write(byte[] key, byte[] value) { + values.put(wrapBytes(key), value); + } + + @Override + public void finish() {} + }; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBBulkLoader.java b/paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBBulkLoader.java new file mode 100644 index 000000000000..8278376f73a8 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBBulkLoader.java @@ -0,0 +1,138 @@ +/* + * 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.paimon.lookup.rocksdb; + +import org.apache.paimon.lookup.ListBulkLoader; +import org.apache.paimon.lookup.ValueBulkLoader; +import org.apache.paimon.utils.ListDelimitedSerializer; + +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.EnvOptions; +import org.rocksdb.IngestExternalFileOptions; +import org.rocksdb.Options; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksDBException; +import org.rocksdb.SstFileWriter; +import org.rocksdb.TtlDB; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; + +/** Bulk loader for RocksDB. */ +public class RocksDBBulkLoader implements ValueBulkLoader, ListBulkLoader { + + private final String uuid = UUID.randomUUID().toString(); + private final ListDelimitedSerializer listSerializer = new ListDelimitedSerializer(); + + private final ColumnFamilyHandle columnFamily; + private final String path; + private final RocksDB db; + private final boolean isTtlEnabled; + private final Options options; + private final List files = new ArrayList<>(); + private final int currentTimeSeconds; + + private SstFileWriter writer = null; + private int sstIndex = 0; + private long recordNum = 0; + + public RocksDBBulkLoader( + RocksDB db, Options options, ColumnFamilyHandle columnFamily, String path) { + this.db = db; + this.isTtlEnabled = db instanceof TtlDB; + this.options = options; + this.columnFamily = columnFamily; + this.path = path; + this.currentTimeSeconds = (int) (System.currentTimeMillis() / 1000); + } + + @Override + public void write(byte[] key, byte[] value) throws WriteException { + try { + if (writer == null) { + writer = new SstFileWriter(new EnvOptions(), options); + String path = new File(this.path, "sst-" + uuid + "-" + (sstIndex++)).getPath(); + writer.open(path); + files.add(path); + } + + if (isTtlEnabled) { + value = appendTimestamp(value); + } + + try { + writer.put(key, value); + } catch (RocksDBException e) { + throw new WriteException(e); + } + + recordNum++; + if (recordNum % 1000 == 0 && writer.fileSize() >= options.targetFileSizeBase()) { + writer.finish(); + writer.close(); + writer = null; + recordNum = 0; + } + } catch (RocksDBException e) { + throw new RuntimeException(e); + } + } + + @Override + public void write(byte[] key, List value) throws WriteException { + byte[] bytes; + try { + bytes = listSerializer.serializeList(value); + } catch (IOException e) { + throw new RuntimeException(e); + } + write(key, bytes); + } + + private byte[] appendTimestamp(byte[] value) { + byte[] newValue = new byte[value.length + 4]; + System.arraycopy(value, 0, newValue, 0, value.length); + newValue[value.length] = (byte) (currentTimeSeconds & 0xff); + newValue[value.length + 1] = (byte) ((currentTimeSeconds >> 8) & 0xff); + newValue[value.length + 2] = (byte) ((currentTimeSeconds >> 16) & 0xff); + newValue[value.length + 3] = (byte) ((currentTimeSeconds >> 24) & 0xff); + return newValue; + } + + @Override + public void finish() { + try { + if (writer != null) { + writer.finish(); + writer.close(); + } + + if (files.size() > 0) { + IngestExternalFileOptions ingestOptions = new IngestExternalFileOptions(); + db.ingestExternalFile(columnFamily, files, ingestOptions); + ingestOptions.close(); + } + } catch (RocksDBException e) { + throw new RuntimeException(e); + } + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBListState.java b/paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBListState.java similarity index 87% rename from paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBListState.java rename to paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBListState.java index 7fe29d209a70..70716c40cba8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBListState.java +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBListState.java @@ -16,9 +16,10 @@ * limitations under the License. */ -package org.apache.paimon.lookup; +package org.apache.paimon.lookup.rocksdb; import org.apache.paimon.data.serializer.Serializer; +import org.apache.paimon.lookup.ListState; import org.apache.paimon.utils.ListDelimitedSerializer; import org.rocksdb.ColumnFamilyHandle; @@ -29,7 +30,7 @@ import java.util.List; /** RocksDB state for key -> List of value. */ -public class RocksDBListState extends RocksDBState> { +public class RocksDBListState extends RocksDBState> implements ListState { private final ListDelimitedSerializer listSerializer = new ListDelimitedSerializer(); @@ -42,6 +43,7 @@ public RocksDBListState( super(stateFactory, columnFamily, keySerializer, valueSerializer, lruCacheSize); } + @Override public void add(K key, V value) throws IOException { byte[] keyBytes = serializeKey(key); byte[] valueBytes = serializeValue(value); @@ -53,6 +55,7 @@ public void add(K key, V value) throws IOException { cache.invalidate(wrap(keyBytes)); } + @Override public List get(K key) throws IOException { byte[] keyBytes = serializeKey(key); return cache.get( @@ -71,14 +74,4 @@ public List get(K key) throws IOException { return rows; }); } - - public byte[] serializeValue(V value) throws IOException { - valueOutputView.clear(); - valueSerializer.serialize(value, valueOutputView); - return valueOutputView.getCopyOfBuffer(); - } - - public byte[] serializeList(List valueList) throws IOException { - return listSerializer.serializeList(valueList); - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBOptions.java b/paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBOptions.java similarity index 99% rename from paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBOptions.java rename to paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBOptions.java index e8c3fecdf868..2cde6d6e7b23 100644 --- a/paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBOptions.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.paimon.lookup; +package org.apache.paimon.lookup.rocksdb; import org.apache.paimon.annotation.Documentation; import org.apache.paimon.options.ConfigOption; diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBSetState.java b/paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBSetState.java similarity index 95% rename from paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBSetState.java rename to paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBSetState.java index 5c06cbd51a42..9f654e152c39 100644 --- a/paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBSetState.java +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBSetState.java @@ -16,9 +16,11 @@ * limitations under the License. */ -package org.apache.paimon.lookup; +package org.apache.paimon.lookup.rocksdb; import org.apache.paimon.data.serializer.Serializer; +import org.apache.paimon.lookup.ByteArray; +import org.apache.paimon.lookup.SetState; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.RocksDBException; @@ -32,7 +34,8 @@ import static org.apache.paimon.utils.Preconditions.checkArgument; /** Rocksdb state for key -> Set values. */ -public class RocksDBSetState extends RocksDBState> { +public class RocksDBSetState extends RocksDBState> + implements SetState { private static final byte[] EMPTY = new byte[0]; @@ -45,6 +48,7 @@ public RocksDBSetState( super(stateFactory, columnFamily, keySerializer, valueSerializer, lruCacheSize); } + @Override public List get(K key) throws IOException { ByteArray keyBytes = wrap(serializeKey(key)); List valueBytes = cache.getIfPresent(keyBytes); @@ -73,6 +77,7 @@ public List get(K key) throws IOException { return values; } + @Override public void retract(K key, V value) throws IOException { try { byte[] bytes = invalidKeyAndGetKVBytes(key, value); @@ -84,6 +89,7 @@ public void retract(K key, V value) throws IOException { } } + @Override public void add(K key, V value) throws IOException { try { byte[] bytes = invalidKeyAndGetKVBytes(key, value); diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBState.java b/paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBState.java similarity index 82% rename from paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBState.java rename to paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBState.java index 0181917a7a26..62257cfd34d3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBState.java +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBState.java @@ -16,13 +16,15 @@ * limitations under the License. */ -package org.apache.paimon.lookup; +package org.apache.paimon.lookup.rocksdb; import org.apache.paimon.CoreOptions; import org.apache.paimon.data.serializer.Serializer; import org.apache.paimon.disk.IOManager; import org.apache.paimon.io.DataInputDeserializer; import org.apache.paimon.io.DataOutputSerializer; +import org.apache.paimon.lookup.ByteArray; +import org.apache.paimon.lookup.State; import org.apache.paimon.sort.BinaryExternalSortBuffer; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; @@ -37,10 +39,9 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.util.Arrays; /** Rocksdb state for key value. */ -public abstract class RocksDBState { +public abstract class RocksDBState implements State { protected final RocksDBStateFactory stateFactory; @@ -85,12 +86,26 @@ public RocksDBState( .build(); } + @Override public byte[] serializeKey(K key) throws IOException { keyOutView.clear(); keySerializer.serialize(key, keyOutView); return keyOutView.getCopyOfBuffer(); } + @Override + public byte[] serializeValue(V value) throws IOException { + valueOutputView.clear(); + valueSerializer.serialize(value, valueOutputView); + return valueOutputView.getCopyOfBuffer(); + } + + @Override + public V deserializeValue(byte[] valueBytes) throws IOException { + valueInputView.setBuffer(valueBytes); + return valueSerializer.deserialize(valueInputView); + } + protected ByteArray wrap(byte[] bytes) { return new ByteArray(bytes); } @@ -99,8 +114,8 @@ protected Reference ref(byte[] bytes) { return new Reference(bytes); } - public BulkLoader createBulkLoader() { - return new BulkLoader(db, stateFactory.options(), columnFamily, stateFactory.path()); + public RocksDBBulkLoader createBulkLoader() { + return new RocksDBBulkLoader(db, stateFactory.options(), columnFamily, stateFactory.path()); } public static BinaryExternalSortBuffer createBulkLoadSorter( @@ -117,33 +132,6 @@ public static BinaryExternalSortBuffer createBulkLoadSorter( options.sequenceFieldSortOrderIsAscending()); } - /** A class wraps byte[] to implement equals and hashCode. */ - protected static class ByteArray { - - protected final byte[] bytes; - - protected ByteArray(byte[] bytes) { - this.bytes = bytes; - } - - @Override - public int hashCode() { - return Arrays.hashCode(bytes); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ByteArray byteArray = (ByteArray) o; - return Arrays.equals(bytes, byteArray.bytes); - } - } - /** A class wraps byte[] to indicate contain or not contain. */ protected static class Reference { diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBStateFactory.java b/paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBStateFactory.java similarity index 94% rename from paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBStateFactory.java rename to paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBStateFactory.java index 9168fbf4cb14..883c82906b65 100644 --- a/paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBStateFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBStateFactory.java @@ -16,9 +16,10 @@ * limitations under the License. */ -package org.apache.paimon.lookup; +package org.apache.paimon.lookup.rocksdb; import org.apache.paimon.data.serializer.Serializer; +import org.apache.paimon.lookup.StateFactory; import org.rocksdb.ColumnFamilyDescriptor; import org.rocksdb.ColumnFamilyHandle; @@ -31,13 +32,12 @@ import javax.annotation.Nullable; -import java.io.Closeable; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.time.Duration; /** Factory to create state. */ -public class RocksDBStateFactory implements Closeable { +public class RocksDBStateFactory implements StateFactory { public static final String MERGE_OPERATOR_NAME = "stringappendtest"; @@ -85,6 +85,7 @@ public String path() { return path; } + @Override public RocksDBValueState valueState( String name, Serializer keySerializer, @@ -95,6 +96,7 @@ public RocksDBValueState valueState( this, createColumnFamily(name), keySerializer, valueSerializer, lruCacheSize); } + @Override public RocksDBSetState setState( String name, Serializer keySerializer, @@ -105,6 +107,7 @@ public RocksDBSetState setState( this, createColumnFamily(name), keySerializer, valueSerializer, lruCacheSize); } + @Override public RocksDBListState listState( String name, Serializer keySerializer, @@ -116,6 +119,11 @@ public RocksDBListState listState( this, createColumnFamily(name), keySerializer, valueSerializer, lruCacheSize); } + @Override + public boolean preferBulkLoad() { + return true; + } + private ColumnFamilyHandle createColumnFamily(String name) throws IOException { try { return db.createColumnFamily( diff --git a/paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBValueState.java b/paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBValueState.java similarity index 86% rename from paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBValueState.java rename to paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBValueState.java index 444ed5ad061a..3d8768a9c489 100644 --- a/paimon-core/src/main/java/org/apache/paimon/lookup/RocksDBValueState.java +++ b/paimon-core/src/main/java/org/apache/paimon/lookup/rocksdb/RocksDBValueState.java @@ -16,9 +16,11 @@ * limitations under the License. */ -package org.apache.paimon.lookup; +package org.apache.paimon.lookup.rocksdb; import org.apache.paimon.data.serializer.Serializer; +import org.apache.paimon.lookup.ByteArray; +import org.apache.paimon.lookup.ValueState; import org.rocksdb.ColumnFamilyHandle; @@ -29,7 +31,8 @@ import static org.apache.paimon.utils.Preconditions.checkArgument; /** Rocksdb state for key -> a single value. */ -public class RocksDBValueState extends RocksDBState { +public class RocksDBValueState extends RocksDBState + implements ValueState { public RocksDBValueState( RocksDBStateFactory stateFactory, @@ -41,6 +44,7 @@ public RocksDBValueState( } @Nullable + @Override public V get(K key) throws IOException { try { Reference valueRef = get(wrap(serializeKey(key))); @@ -60,6 +64,7 @@ private Reference get(ByteArray keyBytes) throws Exception { return valueRef; } + @Override public void put(K key, V value) throws IOException { checkArgument(value != null); @@ -73,6 +78,7 @@ public void put(K key, V value) throws IOException { } } + @Override public void delete(K key) throws IOException { try { byte[] keyBytes = serializeKey(key); @@ -85,15 +91,4 @@ public void delete(K key) throws IOException { throw new IOException(e); } } - - public V deserializeValue(byte[] valueBytes) throws IOException { - valueInputView.setBuffer(valueBytes); - return valueSerializer.deserialize(valueInputView); - } - - public byte[] serializeValue(V value) throws IOException { - valueOutputView.clear(); - valueSerializer.serialize(value, valueOutputView); - return valueOutputView.getCopyOfBuffer(); - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/CompactStrategy.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/CompactStrategy.java index ec82e9e530e0..0ab09819636d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/CompactStrategy.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/CompactStrategy.java @@ -32,7 +32,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; -import java.util.stream.Collectors; /** Compact strategy to decide which files to select for compaction. */ public interface CompactStrategy { @@ -55,16 +54,23 @@ static Optional pickFullCompaction( int numLevels, List runs, @Nullable RecordLevelExpire recordLevelExpire, - @Nullable DeletionVectorsMaintainer dvMaintainer) { + @Nullable DeletionVectorsMaintainer dvMaintainer, + boolean forceRewriteAllFiles) { int maxLevel = numLevels - 1; if (runs.isEmpty()) { // no sorted run, no need to compact return Optional.empty(); - } else if ((runs.size() == 1 && runs.get(0).level() == maxLevel)) { + } + + // only max level files + if ((runs.size() == 1 && runs.get(0).level() == maxLevel)) { List filesToBeCompacted = new ArrayList<>(); for (DataFileMeta file : runs.get(0).run().files()) { - if (recordLevelExpire != null && recordLevelExpire.isExpireFile(file)) { + if (forceRewriteAllFiles) { + // add all files when force compacted + filesToBeCompacted.add(file); + } else if (recordLevelExpire != null && recordLevelExpire.isExpireFile(file)) { // check record level expire for large files filesToBeCompacted.add(file); } else if (dvMaintainer != null @@ -74,27 +80,14 @@ static Optional pickFullCompaction( } } - if (LOG.isDebugEnabled()) { - LOG.debug( - "Pick these files which have expired records or dv index for full compaction: {}", - filesToBeCompacted.stream() - .map( - file -> - String.format( - "(%s, %d, %d)", - file.fileName(), - file.level(), - file.fileSize())) - .collect(Collectors.joining(", "))); - } - - if (!filesToBeCompacted.isEmpty()) { - return Optional.of(CompactUnit.fromFiles(maxLevel, filesToBeCompacted)); - } else { + if (filesToBeCompacted.isEmpty()) { return Optional.empty(); } - } else { - return Optional.of(CompactUnit.fromLevelRuns(maxLevel, runs)); + + return Optional.of(CompactUnit.fromFiles(maxLevel, filesToBeCompacted, true)); } + + // full compaction + return Optional.of(CompactUnit.fromLevelRuns(maxLevel, runs)); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/FileRewriteCompactTask.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/FileRewriteCompactTask.java new file mode 100644 index 000000000000..620ea0748a87 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/FileRewriteCompactTask.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.mergetree.compact; + +import org.apache.paimon.compact.CompactResult; +import org.apache.paimon.compact.CompactTask; +import org.apache.paimon.compact.CompactUnit; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.mergetree.SortedRun; +import org.apache.paimon.operation.metrics.CompactionMetrics; + +import javax.annotation.Nullable; + +import java.util.List; + +import static java.util.Collections.singletonList; + +/** Compact task for file rewrite compaction. */ +public class FileRewriteCompactTask extends CompactTask { + + private final CompactRewriter rewriter; + private final int outputLevel; + private final List files; + private final boolean dropDelete; + + public FileRewriteCompactTask( + CompactRewriter rewriter, + CompactUnit unit, + boolean dropDelete, + @Nullable CompactionMetrics.Reporter metricsReporter) { + super(metricsReporter); + this.rewriter = rewriter; + this.outputLevel = unit.outputLevel(); + this.files = unit.files(); + this.dropDelete = dropDelete; + } + + @Override + protected CompactResult doCompact() throws Exception { + CompactResult result = new CompactResult(); + for (DataFileMeta file : files) { + rewriteFile(file, result); + } + return result; + } + + private void rewriteFile(DataFileMeta file, CompactResult toUpdate) throws Exception { + List> candidate = singletonList(singletonList(SortedRun.fromSingle(file))); + toUpdate.merge(rewriter.rewrite(outputLevel, dropDelete, candidate)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManager.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManager.java index a4852e73464e..025fd2015632 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManager.java @@ -23,6 +23,7 @@ import org.apache.paimon.compact.CompactDeletionFile; import org.apache.paimon.compact.CompactFutureManager; import org.apache.paimon.compact.CompactResult; +import org.apache.paimon.compact.CompactTask; import org.apache.paimon.compact.CompactUnit; import org.apache.paimon.data.InternalRow; import org.apache.paimon.deletionvectors.DeletionVectorsMaintainer; @@ -65,6 +66,7 @@ public class MergeTreeCompactManager extends CompactFutureManager { @Nullable private final DeletionVectorsMaintainer dvMaintainer; private final boolean lazyGenDeletionFile; private final boolean needLookup; + private final boolean forceRewriteAllFiles; @Nullable private final RecordLevelExpire recordLevelExpire; @@ -80,7 +82,8 @@ public MergeTreeCompactManager( @Nullable DeletionVectorsMaintainer dvMaintainer, boolean lazyGenDeletionFile, boolean needLookup, - @Nullable RecordLevelExpire recordLevelExpire) { + @Nullable RecordLevelExpire recordLevelExpire, + boolean forceRewriteAllFiles) { this.executor = executor; this.levels = levels; this.strategy = strategy; @@ -93,6 +96,7 @@ public MergeTreeCompactManager( this.lazyGenDeletionFile = lazyGenDeletionFile; this.recordLevelExpire = recordLevelExpire; this.needLookup = needLookup; + this.forceRewriteAllFiles = forceRewriteAllFiles; MetricUtils.safeCall(this::reportMetrics, LOG); } @@ -135,7 +139,11 @@ public void triggerCompaction(boolean fullCompaction) { } optionalUnit = CompactStrategy.pickFullCompaction( - levels.numberOfLevels(), runs, recordLevelExpire, dvMaintainer); + levels.numberOfLevels(), + runs, + recordLevelExpire, + dvMaintainer, + forceRewriteAllFiles); } else { if (taskFuture != null) { return; @@ -145,7 +153,7 @@ public void triggerCompaction(boolean fullCompaction) { } optionalUnit = strategy.pick(levels.numberOfLevels(), runs) - .filter(unit -> unit.files().size() > 0) + .filter(unit -> !unit.files().isEmpty()) .filter( unit -> unit.files().size() > 1 @@ -199,21 +207,28 @@ private void submitCompaction(CompactUnit unit, boolean dropDelete) { : () -> CompactDeletionFile.generateFiles(dvMaintainer); } - MergeTreeCompactTask task = - new MergeTreeCompactTask( - keyComparator, - compactionFileSize, - rewriter, - unit, - dropDelete, - levels.maxLevel(), - metricsReporter, - compactDfSupplier, - dvMaintainer, - recordLevelExpire); + CompactTask task; + if (unit.fileRewrite()) { + task = new FileRewriteCompactTask(rewriter, unit, dropDelete, metricsReporter); + } else { + task = + new MergeTreeCompactTask( + keyComparator, + compactionFileSize, + rewriter, + unit, + dropDelete, + levels.maxLevel(), + metricsReporter, + compactDfSupplier, + recordLevelExpire, + forceRewriteAllFiles); + } + if (LOG.isDebugEnabled()) { LOG.debug( - "Pick these files (name, level, size) for compaction: {}", + "Pick these files (name, level, size) for {} compaction: {}", + task.getClass().getSimpleName(), unit.files().stream() .map( file -> diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactTask.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactTask.java index 0ce9fbb4c202..667a965c621d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactTask.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactTask.java @@ -23,7 +23,6 @@ import org.apache.paimon.compact.CompactTask; import org.apache.paimon.compact.CompactUnit; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.deletionvectors.DeletionVectorsMaintainer; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.RecordLevelExpire; import org.apache.paimon.mergetree.SortedRun; @@ -45,18 +44,15 @@ public class MergeTreeCompactTask extends CompactTask { private final CompactRewriter rewriter; private final int outputLevel; private final Supplier compactDfSupplier; - private final List> partitioned; - private final boolean dropDelete; private final int maxLevel; + @Nullable private final RecordLevelExpire recordLevelExpire; + private final boolean forceRewriteAllFiles; // metric private int upgradeFilesNum; - @Nullable private final RecordLevelExpire recordLevelExpire; - @Nullable private final DeletionVectorsMaintainer dvMaintainer; - public MergeTreeCompactTask( Comparator keyComparator, long minFileSize, @@ -66,18 +62,18 @@ public MergeTreeCompactTask( int maxLevel, @Nullable CompactionMetrics.Reporter metricsReporter, Supplier compactDfSupplier, - @Nullable DeletionVectorsMaintainer dvMaintainer, - @Nullable RecordLevelExpire recordLevelExpire) { + @Nullable RecordLevelExpire recordLevelExpire, + boolean forceRewriteAllFiles) { super(metricsReporter); this.minFileSize = minFileSize; this.rewriter = rewriter; this.outputLevel = unit.outputLevel(); this.compactDfSupplier = compactDfSupplier; - this.dvMaintainer = dvMaintainer; this.partitioned = new IntervalPartition(unit.files(), keyComparator).partition(); this.dropDelete = dropDelete; this.maxLevel = maxLevel; this.recordLevelExpire = recordLevelExpire; + this.forceRewriteAllFiles = forceRewriteAllFiles; this.upgradeFilesNum = 0; } @@ -125,32 +121,19 @@ protected String logMetric( } private void upgrade(DataFileMeta file, CompactResult toUpdate) throws Exception { - if (file.level() == outputLevel) { - if (isContainExpiredRecords(file) - || (dvMaintainer != null - && dvMaintainer.deletionVectorOf(file.fileName()).isPresent())) { - /* - * 1. if the large file in maxLevel has expired records, we need to rewrite it. - * 2. if the large file in maxLevel has corresponding deletion vector, we need to rewrite it. - */ - rewriteFile(file, toUpdate); - } + if ((outputLevel == maxLevel && containsDeleteRecords(file)) + || forceRewriteAllFiles + || containsExpiredRecords(file)) { + List> candidate = new ArrayList<>(); + candidate.add(singletonList(SortedRun.fromSingle(file))); + rewriteImpl(candidate, toUpdate); return; } - if (outputLevel != maxLevel || file.deleteRowCount().map(d -> d == 0).orElse(false)) { - if (isContainExpiredRecords(file)) { - // if the file which could be directly upgraded has expired records, we need to - // rewrite it - rewriteFile(file, toUpdate); - } else { - CompactResult upgradeResult = rewriter.upgrade(outputLevel, file); - toUpdate.merge(upgradeResult); - upgradeFilesNum++; - } - } else { - // files with delete records should not be upgraded directly to max level - rewriteFile(file, toUpdate); + if (file.level() != outputLevel) { + CompactResult upgradeResult = rewriter.upgrade(outputLevel, file); + toUpdate.merge(upgradeResult); + upgradeFilesNum++; } } @@ -180,14 +163,11 @@ private void rewriteImpl(List> candidate, CompactResult toUpdate candidate.clear(); } - private void rewriteFile(DataFileMeta file, CompactResult toUpdate) throws Exception { - List> candidate = new ArrayList<>(); - candidate.add(new ArrayList<>()); - candidate.get(0).add(SortedRun.fromSingle(file)); - rewriteImpl(candidate, toUpdate); + private boolean containsDeleteRecords(DataFileMeta file) { + return file.deleteRowCount().map(d -> d > 0).orElse(true); } - private boolean isContainExpiredRecords(DataFileMeta file) { + private boolean containsExpiredRecords(DataFileMeta file) { return recordLevelExpire != null && recordLevelExpire.isExpireFile(file); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java index fad898da4522..a7bac3f12a32 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java @@ -567,7 +567,7 @@ protected WriterContainer( } @VisibleForTesting - Map>> writers() { + public Map>> writers() { return writers; } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/BucketedAppendFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/BucketedAppendFileStoreWrite.java index 36201c7f7f0c..03320ad6e1fc 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/BucketedAppendFileStoreWrite.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/BucketedAppendFileStoreWrite.java @@ -91,6 +91,7 @@ protected CompactManager getCompactManager( dvMaintainer, options.compactionMinFileNum(), options.targetFileSize(false), + options.forceRewriteAllFiles(), files -> compactRewrite(partition, bucket, dvFactory, files), compactionMetrics == null ? null diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java index 9a65e83d0688..30fce29821af 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java @@ -290,7 +290,8 @@ private CompactManager createCompactManager( dvMaintainer, options.prepareCommitWaitCompaction(), options.needLookup(), - recordLevelExpire); + recordLevelExpire, + options.forceRewriteAllFiles()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/postpone/PostponeBucketFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/postpone/PostponeBucketFileStoreWrite.java index 09b3afdff209..97b5dfe7c2fc 100644 --- a/paimon-core/src/main/java/org/apache/paimon/postpone/PostponeBucketFileStoreWrite.java +++ b/paimon-core/src/main/java/org/apache/paimon/postpone/PostponeBucketFileStoreWrite.java @@ -26,11 +26,15 @@ import org.apache.paimon.format.avro.AvroSchemaConverter; import org.apache.paimon.fs.FileIO; import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.io.KeyValueFileReaderFactory; import org.apache.paimon.io.KeyValueFileWriterFactory; -import org.apache.paimon.operation.AbstractFileStoreWrite; +import org.apache.paimon.mergetree.compact.ConcatRecordReader; import org.apache.paimon.operation.FileStoreScan; import org.apache.paimon.operation.FileStoreWrite; +import org.apache.paimon.operation.MemoryFileStoreWrite; import org.apache.paimon.options.Options; +import org.apache.paimon.reader.ReaderSupplier; +import org.apache.paimon.reader.RecordReaderIterator; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.BucketMode; import org.apache.paimon.types.RowType; @@ -39,10 +43,17 @@ import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.SnapshotManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.ThreadLocalRandom; import java.util.function.BiFunction; @@ -52,25 +63,37 @@ import static org.apache.paimon.utils.FileStorePathFactory.createFormatPathFactories; /** {@link FileStoreWrite} for {@code bucket = -2} tables. */ -public class PostponeBucketFileStoreWrite extends AbstractFileStoreWrite { +public class PostponeBucketFileStoreWrite extends MemoryFileStoreWrite { + + private static final Logger LOG = LoggerFactory.getLogger(PostponeBucketFileStoreWrite.class); private final CoreOptions options; private final KeyValueFileWriterFactory.Builder writerFactoryBuilder; + private final FileIO fileIO; + private final FileStorePathFactory pathFactory; + private final KeyValueFileReaderFactory.Builder readerFactoryBuilder; + + private boolean forceBufferSpill = false; public PostponeBucketFileStoreWrite( FileIO fileIO, + FileStorePathFactory pathFactory, TableSchema schema, String commitUser, RowType partitionType, RowType keyType, RowType valueType, BiFunction formatPathFactory, + KeyValueFileReaderFactory.Builder readerFactoryBuilder, SnapshotManager snapshotManager, FileStoreScan scan, CoreOptions options, String tableName, @Nullable Integer writeId) { - super(snapshotManager, scan, null, null, tableName, options, partitionType); + super(snapshotManager, scan, options, partitionType, null, null, tableName); + this.fileIO = fileIO; + this.pathFactory = pathFactory; + this.readerFactoryBuilder = readerFactoryBuilder; Options newOptions = new Options(options.toMap()); try { @@ -120,6 +143,25 @@ public PostponeBucketFileStoreWrite( withIgnorePreviousFiles(true); } + @Override + protected void forceBufferSpill() throws Exception { + if (ioManager == null) { + return; + } + if (forceBufferSpill) { + return; + } + forceBufferSpill = true; + LOG.info( + "Force buffer spill for postpone file store write, writer number is: {}", + writers.size()); + for (Map> bucketWriters : writers.values()) { + for (WriterContainer writerContainer : bucketWriters.values()) { + ((PostponeBucketWriter) writerContainer.writer).toBufferedWriter(); + } + } + } + @Override public void withIgnorePreviousFiles(boolean ignorePrevious) { // see comments in constructor @@ -141,7 +183,28 @@ protected PostponeBucketWriter createWriter( "Postpone bucket writers should not restore previous files. This is unexpected."); KeyValueFileWriterFactory writerFactory = writerFactoryBuilder.build(partition, bucket, options); - return new PostponeBucketWriter(writerFactory, restoreIncrement); + return new PostponeBucketWriter( + fileIO, + pathFactory.createDataFilePathFactory(partition, bucket), + options.spillCompressOptions(), + options.writeBufferSpillDiskSize(), + ioManager, + writerFactory, + files -> newFileRead(partition, bucket, files), + forceBufferSpill, + forceBufferSpill, + restoreIncrement); + } + + private RecordReaderIterator newFileRead( + BinaryRow partition, int bucket, List files) throws IOException { + KeyValueFileReaderFactory readerFactory = + readerFactoryBuilder.build(partition, bucket, name -> Optional.empty()); + List> suppliers = new ArrayList<>(); + for (DataFileMeta file : files) { + suppliers.add(() -> readerFactory.createRecordReader(file)); + } + return new RecordReaderIterator<>(ConcatRecordReader.create(suppliers)); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/postpone/PostponeBucketWriter.java b/paimon-core/src/main/java/org/apache/paimon/postpone/PostponeBucketWriter.java index 10259650d4b4..c41db8aac7eb 100644 --- a/paimon-core/src/main/java/org/apache/paimon/postpone/PostponeBucketWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/postpone/PostponeBucketWriter.java @@ -19,14 +19,29 @@ package org.apache.paimon.postpone; import org.apache.paimon.KeyValue; +import org.apache.paimon.KeyValueSerializer; +import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.compression.CompressOptions; +import org.apache.paimon.disk.IOManager; +import org.apache.paimon.fs.FileIO; import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.io.DataFilePathFactory; import org.apache.paimon.io.DataIncrement; import org.apache.paimon.io.KeyValueFileWriterFactory; import org.apache.paimon.io.RollingFileWriter; import org.apache.paimon.manifest.FileSource; +import org.apache.paimon.memory.MemoryOwner; +import org.apache.paimon.memory.MemorySegmentPool; +import org.apache.paimon.options.MemorySize; +import org.apache.paimon.reader.RecordReaderIterator; +import org.apache.paimon.types.RowType; import org.apache.paimon.utils.CommitIncrement; +import org.apache.paimon.utils.IOFunction; import org.apache.paimon.utils.RecordWriter; +import org.apache.paimon.utils.SinkWriter; +import org.apache.paimon.utils.SinkWriter.BufferedSinkWriter; +import org.apache.paimon.utils.SinkWriter.DirectSinkWriter; import javax.annotation.Nullable; @@ -36,30 +51,69 @@ import java.util.List; /** {@link RecordWriter} for {@code bucket = -2} tables. */ -public class PostponeBucketWriter implements RecordWriter { +public class PostponeBucketWriter implements RecordWriter, MemoryOwner { + private final FileIO fileIO; + private final DataFilePathFactory pathFactory; private final KeyValueFileWriterFactory writerFactory; private final List files; + private final IOFunction, RecordReaderIterator> fileRead; + private final @Nullable IOManager ioManager; + private final CompressOptions spillCompression; + private final MemorySize maxDiskSize; - private RollingFileWriter writer; + private SinkWriter sinkWriter; + private MemorySegmentPool memorySegmentPool; public PostponeBucketWriter( - KeyValueFileWriterFactory writerFactory, @Nullable CommitIncrement restoreIncrement) { + FileIO fileIO, + DataFilePathFactory pathFactory, + CompressOptions spillCompression, + MemorySize maxDiskSize, + @Nullable IOManager ioManager, + KeyValueFileWriterFactory writerFactory, + IOFunction, RecordReaderIterator> fileRead, + boolean useWriteBuffer, + boolean spillable, + @Nullable CommitIncrement restoreIncrement) { + this.ioManager = ioManager; this.writerFactory = writerFactory; + this.fileRead = fileRead; + this.fileIO = fileIO; + this.pathFactory = pathFactory; + this.spillCompression = spillCompression; + this.maxDiskSize = maxDiskSize; this.files = new ArrayList<>(); if (restoreIncrement != null) { files.addAll(restoreIncrement.newFilesIncrement().newFiles()); } + this.sinkWriter = + useWriteBuffer + ? createBufferedSinkWriter(spillable) + : new DirectSinkWriter<>(this::createRollingRowWriter); + } - this.writer = null; + private RollingFileWriter createRollingRowWriter() { + return writerFactory.createRollingMergeTreeFileWriter(0, FileSource.APPEND); } @Override public void write(KeyValue record) throws Exception { - if (writer == null) { - writer = writerFactory.createRollingMergeTreeFileWriter(0, FileSource.APPEND); + boolean success = sinkWriter.write(record); + if (!success) { + flush(); + success = sinkWriter.write(record); + if (!success) { + // Should not get here, because writeBuffer will throw too big exception out. + // But we throw again in case of something unexpected happens. (like someone changed + // code in SpillableBuffer.) + throw new RuntimeException("Mem table is too small to hold a single element."); + } } - writer.write(record); + } + + private void flush() throws Exception { + files.addAll(sinkWriter.flush()); } @Override @@ -82,13 +136,66 @@ public long maxSequenceNumber() { } @Override - public CommitIncrement prepareCommit(boolean waitCompaction) throws Exception { - if (writer != null) { - writer.close(); - files.addAll(writer.result()); - writer = null; + public void setMemoryPool(MemorySegmentPool memoryPool) { + this.memorySegmentPool = memoryPool; + sinkWriter.setMemoryPool(memoryPool); + } + + @Override + public long memoryOccupancy() { + return sinkWriter.memoryOccupancy(); + } + + @Override + public void flushMemory() throws Exception { + boolean success = sinkWriter.flushMemory(); + if (!success) { + flush(); } + } + private BufferedSinkWriter createBufferedSinkWriter(boolean spillable) { + RowType keyType = writerFactory.keyType(); + RowType valueType = writerFactory.valueType(); + RowType kvRowType = KeyValue.schema(keyType, valueType); + KeyValueSerializer serializer = new KeyValueSerializer(keyType, valueType); + return new BufferedSinkWriter<>( + this::createRollingRowWriter, + serializer::toRow, + serializer::fromRow, + ioManager, + kvRowType, + spillable, + maxDiskSize, + spillCompression); + } + + public void toBufferedWriter() throws Exception { + if (sinkWriter != null && !sinkWriter.bufferSpillableWriter() && fileRead != null) { + // fetch the written results + List files = sinkWriter.flush(); + + sinkWriter.close(); + sinkWriter = createBufferedSinkWriter(true); + sinkWriter.setMemoryPool(memorySegmentPool); + + // rewrite small files + try (RecordReaderIterator reader = fileRead.apply(files)) { + while (reader.hasNext()) { + sinkWriter.write(reader.next()); + } + } finally { + // remove small files + for (DataFileMeta file : files) { + fileIO.deleteQuietly(pathFactory.toPath(file)); + } + } + } + } + + @Override + public CommitIncrement prepareCommit(boolean waitCompaction) throws Exception { + flush(); List result = new ArrayList<>(files); files.clear(); return new CommitIncrement( @@ -97,6 +204,11 @@ public CommitIncrement prepareCommit(boolean waitCompaction) throws Exception { null); } + @VisibleForTesting + public boolean useBufferedSinkWriter() { + return sinkWriter instanceof BufferedSinkWriter; + } + @Override public boolean compactNotCompleted() { return false; @@ -107,9 +219,6 @@ public void sync() throws Exception {} @Override public void close() throws Exception { - if (writer != null) { - writer.abort(); - writer = null; - } + sinkWriter.close(); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/ManifestReadThreadPool.java b/paimon-core/src/main/java/org/apache/paimon/utils/ManifestReadThreadPool.java index 49fcfc8bd909..16bd73ee80df 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/ManifestReadThreadPool.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/ManifestReadThreadPool.java @@ -22,6 +22,7 @@ import java.util.Iterator; import java.util.List; +import java.util.concurrent.ExecutorService; import java.util.concurrent.ThreadPoolExecutor; import java.util.function.Function; @@ -35,28 +36,38 @@ public class ManifestReadThreadPool { private static ThreadPoolExecutor executorService = createCachedThreadPool(Runtime.getRuntime().availableProcessors(), THREAD_NAME); - public static synchronized ThreadPoolExecutor getExecutorService(@Nullable Integer threadNum) { - if (threadNum == null || threadNum <= executorService.getMaximumPoolSize()) { + public static synchronized ExecutorService getExecutorService(@Nullable Integer threadNum) { + if (threadNum == null || threadNum == executorService.getMaximumPoolSize()) { return executorService; } - // we don't need to close previous pool - // it is just cached pool - executorService = createCachedThreadPool(threadNum, THREAD_NAME); + if (threadNum < executorService.getMaximumPoolSize()) { + return new SemaphoredDelegatingExecutor(executorService, threadNum, false); + } else { + // we don't need to close previous pool + // it is just cached pool + executorService = createCachedThreadPool(threadNum, THREAD_NAME); - return executorService; + return executorService; + } } /** This method aims to parallel process tasks with memory control and sequentially. */ public static Iterable sequentialBatchedExecute( Function> processor, List input, @Nullable Integer threadNum) { - ThreadPoolExecutor executor = getExecutorService(threadNum); + ExecutorService executor = getExecutorService(threadNum); + if (threadNum == null) { + threadNum = + executor instanceof ThreadPoolExecutor + ? ((ThreadPoolExecutor) executor).getMaximumPoolSize() + : ((SemaphoredDelegatingExecutor) executor).getPermitCount(); + } return ThreadPoolUtils.sequentialBatchedExecute(executor, processor, input, threadNum); } /** This method aims to parallel process tasks with randomly but return values sequentially. */ public static Iterator randomlyExecuteSequentialReturn( Function> processor, List input, @Nullable Integer threadNum) { - ThreadPoolExecutor executor = getExecutorService(threadNum); + ExecutorService executor = getExecutorService(threadNum); return ThreadPoolUtils.randomlyExecuteSequentialReturn(executor, processor, input); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/SinkWriter.java b/paimon-core/src/main/java/org/apache/paimon/utils/SinkWriter.java new file mode 100644 index 000000000000..c4596a1df464 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/utils/SinkWriter.java @@ -0,0 +1,234 @@ +/* + * 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.paimon.utils; + +import org.apache.paimon.compression.CompressOptions; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.serializer.InternalRowSerializer; +import org.apache.paimon.disk.IOManager; +import org.apache.paimon.disk.RowBuffer; +import org.apache.paimon.io.BundleRecords; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.io.RollingFileWriter; +import org.apache.paimon.memory.MemorySegmentPool; +import org.apache.paimon.options.MemorySize; +import org.apache.paimon.types.RowType; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +/** Internal interface to Sink Data from input. */ +public interface SinkWriter { + + boolean write(T data) throws IOException; + + List flush() throws IOException; + + boolean flushMemory() throws IOException; + + long memoryOccupancy(); + + void close(); + + void setMemoryPool(MemorySegmentPool memoryPool); + + boolean bufferSpillableWriter(); + + /** + * Directly sink data to file, no memory cache here, use OrcWriter/ParquetWrite/etc directly + * write data. May cause out-of-memory. + */ + class DirectSinkWriter implements SinkWriter { + + private final Supplier> writerSupplier; + + private RollingFileWriter writer; + + public DirectSinkWriter(Supplier> writerSupplier) { + this.writerSupplier = writerSupplier; + } + + @Override + public boolean write(T data) throws IOException { + if (writer == null) { + writer = writerSupplier.get(); + } + writer.write(data); + return true; + } + + public void writeBundle(BundleRecords bundle) throws IOException { + if (writer == null) { + writer = writerSupplier.get(); + } + writer.writeBundle(bundle); + } + + @Override + public List flush() throws IOException { + List flushedFiles = new ArrayList<>(); + if (writer != null) { + writer.close(); + flushedFiles.addAll(writer.result()); + writer = null; + } + return flushedFiles; + } + + @Override + public boolean flushMemory() throws IOException { + return false; + } + + @Override + public long memoryOccupancy() { + return 0; + } + + @Override + public void close() { + if (writer != null) { + writer.abort(); + writer = null; + } + } + + @Override + public void setMemoryPool(MemorySegmentPool memoryPool) { + // do nothing + } + + @Override + public boolean bufferSpillableWriter() { + return false; + } + } + + /** + * Use buffered writer, segment pooled from segment pool. When spillable, may delay checkpoint + * acknowledge time. When non-spillable, may cause too many small files. + */ + class BufferedSinkWriter implements SinkWriter { + + private final Supplier> writerSupplier; + private final Function toRow; + private final Function fromRow; + private final IOManager ioManager; + private final RowType rowType; + private final boolean spillable; + private final MemorySize maxDiskSize; + private final CompressOptions compression; + + private RowBuffer writeBuffer; + + public BufferedSinkWriter( + Supplier> writerSupplier, + Function toRow, + Function fromRow, + IOManager ioManager, + RowType rowType, + boolean spillable, + MemorySize maxDiskSize, + CompressOptions compression) { + this.writerSupplier = writerSupplier; + this.toRow = toRow; + this.fromRow = fromRow; + this.ioManager = ioManager; + this.rowType = rowType; + this.spillable = spillable; + this.maxDiskSize = maxDiskSize; + this.compression = compression; + } + + public RowBuffer rowBuffer() { + return writeBuffer; + } + + @Override + public boolean write(T data) throws IOException { + return writeBuffer.put(toRow.apply(data)); + } + + @Override + public List flush() throws IOException { + List flushedFiles = new ArrayList<>(); + if (writeBuffer != null) { + writeBuffer.complete(); + RollingFileWriter writer = writerSupplier.get(); + IOException exception = null; + try (RowBuffer.RowBufferIterator iterator = writeBuffer.newIterator()) { + while (iterator.advanceNext()) { + writer.write(fromRow.apply(iterator.getRow())); + } + } catch (IOException e) { + exception = e; + } finally { + if (exception != null) { + IOUtils.closeQuietly(writer); + // cleanup code that might throw another exception + throw exception; + } + writer.close(); + } + flushedFiles.addAll(writer.result()); + // reuse writeBuffer + writeBuffer.reset(); + } + return flushedFiles; + } + + @Override + public long memoryOccupancy() { + return writeBuffer.memoryOccupancy(); + } + + @Override + public void close() { + if (writeBuffer != null) { + writeBuffer.reset(); + writeBuffer = null; + } + } + + @Override + public void setMemoryPool(MemorySegmentPool memoryPool) { + writeBuffer = + RowBuffer.getBuffer( + ioManager, + memoryPool, + new InternalRowSerializer(rowType), + spillable, + maxDiskSize, + compression); + } + + @Override + public boolean bufferSpillableWriter() { + return spillable; + } + + @Override + public boolean flushMemory() throws IOException { + return writeBuffer.flushMemory(); + } + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java b/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java index 2b280220ee70..8cb293cd8ee4 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java @@ -652,6 +652,7 @@ private Pair> createWriter( null, MIN_FILE_NUM, targetFileSize, + false, compactBefore -> { latch.await(); return compactBefore.isEmpty() diff --git a/paimon-core/src/test/java/org/apache/paimon/append/BucketedAppendCompactManagerTest.java b/paimon-core/src/test/java/org/apache/paimon/append/BucketedAppendCompactManagerTest.java index cfdf38558fdb..2f031548a846 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/BucketedAppendCompactManagerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/BucketedAppendCompactManagerTest.java @@ -207,6 +207,7 @@ private void innerTest( null, minFileNum, targetFileSize, + false, null, // not used null); Optional> actual = manager.pickCompactBefore(); diff --git a/paimon-core/src/test/java/org/apache/paimon/append/FullCompactTaskTest.java b/paimon-core/src/test/java/org/apache/paimon/append/FullCompactTaskTest.java index e7c3cce01da3..262acd5d4f05 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/FullCompactTaskTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/FullCompactTaskTest.java @@ -123,7 +123,7 @@ public MockFullCompactTask( Collection inputs, long targetFileSize, BucketedAppendCompactManager.CompactRewriter rewriter) { - super(null, inputs, targetFileSize, rewriter, null); + super(null, inputs, targetFileSize, false, rewriter, null); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java b/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java index c43b3c20c6a8..c6761d9c929d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java @@ -87,7 +87,7 @@ public void testFileSuffix(@TempDir java.nio.file.Path tempDir) throws Exception SCHEMA, 0, new BucketedAppendCompactManager( - null, toCompact, null, 4, 10, null, null), // not used + null, toCompact, null, 4, 10, false, null, null), // not used null, false, dataFilePathFactory, diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/MergeTreeTestBase.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/MergeTreeTestBase.java index 4af6c3980008..95daf384e797 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/MergeTreeTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/MergeTreeTestBase.java @@ -454,7 +454,8 @@ private MergeTreeCompactManager createCompactManager( null, false, options.needLookup(), - null); + null, + false); } static class MockFailResultCompactionManager extends MergeTreeCompactManager { @@ -478,7 +479,8 @@ public MockFailResultCompactionManager( null, false, false, - null); + null, + false); } protected CompactResult obtainCompactResult() diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManagerTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManagerTest.java index 424055597735..4adff9477857 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManagerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManagerTest.java @@ -208,7 +208,8 @@ public void testIsCompacting() { null, false, true, - null); + null, + false); MergeTreeCompactManager defaultManager = new MergeTreeCompactManager( @@ -223,7 +224,8 @@ public void testIsCompacting() { null, false, false, - null); + null, + false); assertThat(lookupManager.compactNotCompleted()).isTrue(); assertThat(defaultManager.compactNotCompleted()).isFalse(); @@ -259,7 +261,8 @@ private void innerTest( null, false, false, - null); + null, + false); manager.triggerCompaction(false); manager.getCompactionResult(true); List outputs = diff --git a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeySimpleTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeySimpleTableTest.java index 0bf4971f9408..35928e23356a 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeySimpleTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeySimpleTableTest.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.CoreOptions.ChangelogProducer; import org.apache.paimon.CoreOptions.LookupLocalFileType; +import org.apache.paimon.KeyValue; import org.apache.paimon.Snapshot; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.GenericRow; @@ -34,9 +35,12 @@ import org.apache.paimon.manifest.FileKind; import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestFileMeta; +import org.apache.paimon.operation.AbstractFileStoreWrite; import org.apache.paimon.operation.FileStoreScan; import org.apache.paimon.options.MemorySize; import org.apache.paimon.options.Options; +import org.apache.paimon.postpone.PostponeBucketFileStoreWrite; +import org.apache.paimon.postpone.PostponeBucketWriter; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.predicate.PredicateBuilder; import org.apache.paimon.reader.RecordReader; @@ -54,6 +58,7 @@ import org.apache.paimon.table.sink.StreamTableCommit; import org.apache.paimon.table.sink.StreamTableWrite; import org.apache.paimon.table.sink.StreamWriteBuilder; +import org.apache.paimon.table.sink.TableWriteImpl; import org.apache.paimon.table.sink.WriteSelector; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.InnerTableRead; @@ -136,6 +141,39 @@ /** Tests for {@link PrimaryKeyFileStoreTable}. */ public class PrimaryKeySimpleTableTest extends SimpleTableTestBase { + @Test + public void testPostponeBucketWithManyPartitions() throws Exception { + FileStoreTable table = + createFileStoreTable(options -> options.set(BUCKET, BucketMode.POSTPONE_BUCKET)); + + BatchWriteBuilder writeBuilder = table.newBatchWriteBuilder(); + try (BatchTableWrite write = writeBuilder.newWrite(); + BatchTableCommit commit = writeBuilder.newCommit()) { + write.withIOManager(new IOManagerImpl(tempDir.toString())); + for (int i = 0; i < 100; i++) { + write.write(rowData(i, i, (long) i)); + } + + for (Map> bucketWriters : + ((PostponeBucketFileStoreWrite) ((TableWriteImpl) write).getWrite()) + .writers() + .values()) { + for (AbstractFileStoreWrite.WriterContainer writerContainer : + bucketWriters.values()) { + PostponeBucketWriter writer = (PostponeBucketWriter) writerContainer.writer; + assertThat(writer.useBufferedSinkWriter()).isTrue(); + } + } + commit.commit(write.prepareCommit()); + } + + Snapshot snapshot = table.latestSnapshot().get(); + ManifestFileMeta manifest = + table.manifestListReader().read(snapshot.deltaManifestList()).get(0); + List entries = table.manifestFileReader().read(manifest.fileName()); + assertThat(entries.size()).isEqualTo(100); + } + @Test public void testPostponeBucket() throws Exception { FileStoreTable table = diff --git a/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java b/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java index 71792962b812..219e7df99c85 100644 --- a/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java +++ b/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java @@ -74,7 +74,7 @@ public class ConfigOptionsDocGenerator { new OptionsClassLocation[] { new OptionsClassLocation("paimon-api", "org.apache.paimon.options"), new OptionsClassLocation("paimon-api", "org.apache.paimon"), - new OptionsClassLocation("paimon-core", "org.apache.paimon.lookup"), + new OptionsClassLocation("paimon-core", "org.apache.paimon.lookup.rocksdb"), new OptionsClassLocation("paimon-core", "org.apache.paimon.jdbc"), new OptionsClassLocation("paimon-core", "org.apache.paimon.table"), new OptionsClassLocation("paimon-core", "org.apache.paimon.iceberg"), diff --git a/paimon-filesystems/paimon-cosn/src/main/java/org/apache/paimon/cosn/COSNLoader.java b/paimon-filesystems/paimon-cosn/src/main/java/org/apache/paimon/cosn/COSNLoader.java index 6d86fdc1791e..35b6c1ff28d1 100644 --- a/paimon-filesystems/paimon-cosn/src/main/java/org/apache/paimon/cosn/COSNLoader.java +++ b/paimon-filesystems/paimon-cosn/src/main/java/org/apache/paimon/cosn/COSNLoader.java @@ -25,6 +25,9 @@ import org.apache.paimon.fs.PluginFileIO; import org.apache.paimon.plugin.PluginLoader; +import java.util.ArrayList; +import java.util.List; + /** A {@link PluginLoader} to load cosn. */ public class COSNLoader implements FileIOLoader { private static final String COSN_JAR = "paimon-plugin-cosn"; @@ -45,6 +48,15 @@ public String getScheme() { return "cosn"; } + @Override + public List requiredOptions() { + List options = new ArrayList<>(); + options.add(new String[] {"fs.cosn.bucket.region"}); + options.add(new String[] {"fs.cosn.userinfo.secretId"}); + options.add(new String[] {"fs.cosn.userinfo.secretKey"}); + return options; + } + @Override public FileIO load(Path path) { return new COSNPluginFileIO(); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java index 114344badeed..80f99165e6d4 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java @@ -475,6 +475,7 @@ private static DataType fromDebeziumAvroType(Schema schema) { return DataTypes.INT(); case LONG: return DataTypes.BIGINT(); + case ENUM: case STRING: return DataTypes.STRING(); case RECORD: diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcAppendTableSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcAppendTableSink.java index d43690e5b296..7a5b2a0f11f8 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcAppendTableSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcAppendTableSink.java @@ -19,8 +19,10 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.flink.sink.Committable; +import org.apache.paimon.flink.sink.CommittableStateManager; import org.apache.paimon.flink.sink.FlinkWriteSink; import org.apache.paimon.flink.sink.StoreSinkWrite; +import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.table.FileStoreTable; import org.apache.flink.streaming.api.datastream.DataStream; @@ -52,4 +54,9 @@ public DataStream doWrite( DataStream input, String initialCommitUser, @Nullable Integer parallelism) { return super.doWrite(input, initialCommitUser, this.parallelism); } + + @Override + protected CommittableStateManager createCommittableStateManager() { + return createRestoreOnlyCommittableStateManager(table); + } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcAppendTableWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcAppendTableWriteOperator.java index 96b9fefcdfc2..b3e415372770 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcAppendTableWriteOperator.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcAppendTableWriteOperator.java @@ -19,11 +19,14 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.flink.sink.Committable; +import org.apache.paimon.flink.sink.NoopStoreSinkWriteState; import org.apache.paimon.flink.sink.PrepareCommitOperator; import org.apache.paimon.flink.sink.StoreSinkWrite; +import org.apache.paimon.flink.sink.StoreSinkWriteState; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.RowKind; +import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperatorParameters; @@ -40,6 +43,16 @@ private CdcAppendTableWriteOperator( super(parameters, table, storeSinkWriteProvider, initialCommitUser); } + @Override + protected StoreSinkWriteState createState( + int subtaskId, + StateInitializationContext context, + StoreSinkWriteState.StateValueFilter stateFilter) { + // No conflicts will occur in append only unaware bucket writer, so no state + // is needed. + return new NoopStoreSinkWriteState(subtaskId, stateFilter); + } + @Override public void processElement(StreamRecord element) throws Exception { // only accepts INSERT record diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java index 0cd2638179b8..a67e74ef55a4 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java @@ -177,6 +177,6 @@ public DataStreamSink sinkFrom( protected CommittableStateManager createCommittableStateManager() { return new RestoreAndFailCommittableStateManager<>( - WrappedManifestCommittableSerializer::new); + WrappedManifestCommittableSerializer::new, true); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtilsTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtilsTest.java new file mode 100644 index 000000000000..7381bb4890c8 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtilsTest.java @@ -0,0 +1,153 @@ +/* + * 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.paimon.flink.action.cdc.format.debezium; + +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; + +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; + +/** Test class for DebeziumSchemaUtils. */ +public class DebeziumSchemaUtilsTest { + @Test + public void testFromDebeziumAvroTypeWithGenericUnion() { + // Create a union schema with multiple non-null types + Schema stringSchema = Schema.create(Schema.Type.STRING); + Schema intSchema = Schema.create(Schema.Type.INT); + Schema unionSchema = Schema.createUnion(Arrays.asList(stringSchema, intSchema)); + + // Test that an exception is thrown for generic unions + UnsupportedOperationException exception = + Assertions.assertThrows( + UnsupportedOperationException.class, + () -> DebeziumSchemaUtils.avroToPaimonDataType(unionSchema)); + + Assertions.assertEquals("Generic unions are not supported", exception.getMessage()); + } + + @Test + public void testFromDebeziumAvroTypeWithLogicalTypes() { + // Test date logical type + Schema dateSchema = Schema.create(Schema.Type.INT); + LogicalTypes.date().addToSchema(dateSchema); + DataType dateType = DebeziumSchemaUtils.avroToPaimonDataType(dateSchema); + Assertions.assertEquals(DataTypes.DATE(), dateType); + + // Test decimal logical type + Schema decimalSchema = Schema.create(Schema.Type.BYTES); + LogicalTypes.decimal(10, 2).addToSchema(decimalSchema); + DataType decimalType = DebeziumSchemaUtils.avroToPaimonDataType(decimalSchema); + Assertions.assertEquals(DataTypes.DECIMAL(10, 2), decimalType); + + // Test timestamp-millis logical type + Schema timestampMillisSchema = Schema.create(Schema.Type.LONG); + LogicalTypes.timestampMillis().addToSchema(timestampMillisSchema); + DataType timestampMillisType = + DebeziumSchemaUtils.avroToPaimonDataType(timestampMillisSchema); + Assertions.assertEquals(DataTypes.TIMESTAMP_MILLIS(), timestampMillisType); + + // Test timestamp-micros logical type + Schema timestampMicrosSchema = Schema.create(Schema.Type.LONG); + LogicalTypes.timestampMicros().addToSchema(timestampMicrosSchema); + DataType timestampMicrosType = + DebeziumSchemaUtils.avroToPaimonDataType(timestampMicrosSchema); + Assertions.assertEquals(DataTypes.TIMESTAMP(), timestampMicrosType); + } + + @Test + public void testFromDebeziumAvroTypeWithPrimitiveTypes() { + // Test boolean type + Schema booleanSchema = Schema.create(Schema.Type.BOOLEAN); + DataType booleanType = DebeziumSchemaUtils.avroToPaimonDataType(booleanSchema); + Assertions.assertEquals(DataTypes.BOOLEAN(), booleanType); + + // Test int type + Schema intSchema = Schema.create(Schema.Type.INT); + DataType intType = DebeziumSchemaUtils.avroToPaimonDataType(intSchema); + Assertions.assertEquals(DataTypes.INT(), intType); + + // Test long type + Schema longSchema = Schema.create(Schema.Type.LONG); + DataType longType = DebeziumSchemaUtils.avroToPaimonDataType(longSchema); + Assertions.assertEquals(DataTypes.BIGINT(), longType); + + // Test float type + Schema floatSchema = Schema.create(Schema.Type.FLOAT); + DataType floatType = DebeziumSchemaUtils.avroToPaimonDataType(floatSchema); + Assertions.assertEquals(DataTypes.FLOAT(), floatType); + + // Test double type + Schema doubleSchema = Schema.create(Schema.Type.DOUBLE); + DataType doubleType = DebeziumSchemaUtils.avroToPaimonDataType(doubleSchema); + Assertions.assertEquals(DataTypes.DOUBLE(), doubleType); + + // Test enum type + Schema enumSchema = + Schema.createEnum("TestEnum", null, null, Arrays.asList("VALUE1", "VALUE2")); + DataType enumType = DebeziumSchemaUtils.avroToPaimonDataType(enumSchema); + Assertions.assertEquals(DataTypes.STRING(), enumType); + + // Test string type + Schema stringSchema = Schema.create(Schema.Type.STRING); + DataType stringType = DebeziumSchemaUtils.avroToPaimonDataType(stringSchema); + Assertions.assertEquals(DataTypes.STRING(), stringType); + + // Test bytes type + Schema bytesSchema = Schema.create(Schema.Type.BYTES); + DataType bytesType = DebeziumSchemaUtils.avroToPaimonDataType(bytesSchema); + Assertions.assertEquals(DataTypes.BYTES(), bytesType); + } + + @Test + public void testFromDebeziumAvroTypeWithComplexTypes() { + // Test array type + Schema stringSchema = Schema.create(Schema.Type.STRING); + Schema arraySchema = Schema.createArray(stringSchema); + DataType arrayType = DebeziumSchemaUtils.avroToPaimonDataType(arraySchema); + Assertions.assertEquals(DataTypes.ARRAY(DataTypes.STRING()), arrayType); + + // Test map type + Schema mapSchema = Schema.createMap(stringSchema); + DataType mapType = DebeziumSchemaUtils.avroToPaimonDataType(mapSchema); + Assertions.assertEquals(DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING()), mapType); + + // Test record type + Schema recordSchema = Schema.createRecord("TestRecord", null, null, false); + List fields = + Arrays.asList( + new Schema.Field("field1", stringSchema, null, null), + new Schema.Field("field2", Schema.create(Schema.Type.INT), null, null)); + recordSchema.setFields(fields); + DataType recordType = DebeziumSchemaUtils.avroToPaimonDataType(recordSchema); + + DataField[] expectedFields = + new DataField[] { + DataTypes.FIELD(0, "field1", DataTypes.STRING(), null), + DataTypes.FIELD(1, "field2", DataTypes.INT(), null) + }; + Assertions.assertEquals(DataTypes.ROW(expectedFields), recordType); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java index 9c0e9d619124..d524920e8248 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java @@ -224,12 +224,12 @@ public class FlinkConnectorOptions { + " Note: This is dangerous and is likely to cause data errors if downstream" + " is used to calculate aggregation and the input is not complete changelog."); - public static final ConfigOption STREAMING_READ_SHUFFLE_BUCKET_WITH_PARTITION = - key("streaming-read.shuffle-bucket-with-partition") + public static final ConfigOption READ_SHUFFLE_BUCKET_WITH_PARTITION = + key("read.shuffle-bucket-with-partition") .booleanType() .defaultValue(true) - .withDescription( - "Whether shuffle by partition and bucket when streaming read."); + .withFallbackKeys("streaming-read.shuffle-bucket-with-partition") + .withDescription("Whether shuffle by partition and bucket when read."); /** * Weight of writer buffer in managed memory, Flink will compute the memory size for writer @@ -508,6 +508,13 @@ public class FlinkConnectorOptions { .withDescription( "Bucket number for the partitions compacted for the first time in postpone bucket tables."); + public static final ConfigOption SCAN_DEDICATED_SPLIT_GENERATION = + key("scan.dedicated-split-generation") + .booleanType() + .defaultValue(false) + .withDescription( + "If true, the split generation process would be performed during runtime on a Flink task, instead of on the JobManager during initialization phase."); + public static List> getOptions() { final Field[] fields = FlinkConnectorOptions.class.getFields(); final List> list = new ArrayList<>(fields.length); @@ -534,7 +541,10 @@ public enum LookupCacheMode { AUTO, /** Use full caching mode. */ - FULL + FULL, + + /** Use in-memory caching mode. */ + MEMORY } /** Watermark emit strategy for scan. */ diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java index dec278fb704b..e3e15c4ce644 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java @@ -71,8 +71,8 @@ import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_CACHE_MODE; import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_REFRESH_TIME_PERIODS_BLACKLIST; import static org.apache.paimon.flink.query.RemoteTableQuery.isRemoteServiceAvailable; -import static org.apache.paimon.lookup.RocksDBOptions.LOOKUP_CACHE_ROWS; -import static org.apache.paimon.lookup.RocksDBOptions.LOOKUP_CONTINUOUS_DISCOVERY_INTERVAL; +import static org.apache.paimon.lookup.rocksdb.RocksDBOptions.LOOKUP_CACHE_ROWS; +import static org.apache.paimon.lookup.rocksdb.RocksDBOptions.LOOKUP_CONTINUOUS_DISCOVERY_INTERVAL; import static org.apache.paimon.predicate.PredicateBuilder.transformFieldMapping; /** A lookup {@link TableFunction} for file store. */ diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java index 4154b6742c06..27795a00ff4b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java @@ -24,9 +24,11 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; -import org.apache.paimon.lookup.BulkLoader; -import org.apache.paimon.lookup.RocksDBState; -import org.apache.paimon.lookup.RocksDBStateFactory; +import org.apache.paimon.lookup.StateFactory; +import org.apache.paimon.lookup.memory.InMemoryStateFactory; +import org.apache.paimon.lookup.rocksdb.RocksDBBulkLoader; +import org.apache.paimon.lookup.rocksdb.RocksDBState; +import org.apache.paimon.lookup.rocksdb.RocksDBStateFactory; import org.apache.paimon.options.Options; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.predicate.PredicateBuilder; @@ -64,11 +66,14 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_CACHE_MODE; import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_REFRESH_ASYNC; import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_REFRESH_ASYNC_PENDING_SNAPSHOT_COUNT; +import static org.apache.paimon.flink.FlinkConnectorOptions.LookupCacheMode.MEMORY; /** Lookup table of full cache. */ public abstract class FullCacheLookupTable implements LookupTable { + private static final Logger LOG = LoggerFactory.getLogger(FullCacheLookupTable.class); protected final Object lock = new Object(); @@ -79,7 +84,7 @@ public abstract class FullCacheLookupTable implements LookupTable { @Nullable protected final FieldsComparator userDefinedSeqComparator; protected final int appendUdsFieldNumber; - protected RocksDBStateFactory stateFactory; + protected StateFactory stateFactory; @Nullable private ExecutorService refreshExecutor; private final AtomicReference cachedException; private final int maxPendingSnapshotCount; @@ -143,11 +148,7 @@ public void specifyCacheRowFilter(Filter filter) { } protected void init() throws Exception { - this.stateFactory = - new RocksDBStateFactory( - context.tempPath.toString(), - context.table.coreOptions().toConfiguration(), - null); + this.stateFactory = createStateFactory(); this.refreshExecutor = this.refreshAsync ? Executors.newSingleThreadExecutor( @@ -158,6 +159,16 @@ protected void init() throws Exception { : null; } + private StateFactory createStateFactory() throws IOException { + String diskDir = context.tempPath.toString(); + Options options = context.table.coreOptions().toConfiguration(); + if (options.get(LOOKUP_CACHE_MODE) == MEMORY) { + return new InMemoryStateFactory(); + } else { + return new RocksDBStateFactory(diskDir, options, null); + } + } + protected void bootstrap() throws Exception { Predicate scanPredicate = PredicateBuilder.andNullable(context.tablePredicate, specificPartition); @@ -168,6 +179,11 @@ protected void bootstrap() throws Exception { scanPredicate, context.requiredCachedBucketIds, cacheRowFilter); + if (!stateFactory.preferBulkLoad()) { + doRefresh(); + return; + } + BinaryExternalSortBuffer bulkLoadSorter = RocksDBState.createBulkLoadSorter( IOManager.create(context.tempPath.toString()), context.table.coreOptions()); @@ -189,7 +205,7 @@ protected void bootstrap() throws Exception { while ((row = keyIterator.next(row)) != null) { bulkLoader.write(row.getBinary(0), row.getBinary(1)); } - } catch (BulkLoader.WriteException e) { + } catch (RocksDBBulkLoader.WriteException e) { throw new RuntimeException( "Exception in bulkLoad, the most suspicious reason is that " + "your data contains duplicates, please check your lookup table. ", @@ -331,7 +347,7 @@ public Future getRefreshFuture() { /** Bulk loader for the table. */ public interface TableBulkLoader { - void write(byte[] key, byte[] value) throws BulkLoader.WriteException, IOException; + void write(byte[] key, byte[] value) throws RocksDBBulkLoader.WriteException, IOException; void finish() throws IOException; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/NoPrimaryKeyLookupTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/NoPrimaryKeyLookupTable.java index 63af4f3506f0..546bc5a60a00 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/NoPrimaryKeyLookupTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/NoPrimaryKeyLookupTable.java @@ -20,8 +20,9 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.serializer.InternalSerializers; -import org.apache.paimon.lookup.BulkLoader; -import org.apache.paimon.lookup.RocksDBListState; +import org.apache.paimon.lookup.ListBulkLoader; +import org.apache.paimon.lookup.ListState; +import org.apache.paimon.lookup.rocksdb.RocksDBBulkLoader; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.types.RowKind; import org.apache.paimon.utils.KeyProjectedRow; @@ -40,7 +41,7 @@ public class NoPrimaryKeyLookupTable extends FullCacheLookupTable { private final KeyProjectedRow joinKeyRow; - private RocksDBListState state; + private ListState state; public NoPrimaryKeyLookupTable(Context context, long lruCacheSize) { super(context); @@ -105,7 +106,7 @@ public byte[] toValueBytes(InternalRow row) throws IOException { @Override public TableBulkLoader createBulkLoader() { - BulkLoader bulkLoader = state.createBulkLoader(); + ListBulkLoader bulkLoader = state.createBulkLoader(); return new TableBulkLoader() { private final List values = new ArrayList<>(); @@ -113,7 +114,7 @@ public TableBulkLoader createBulkLoader() { private byte[] currentKey; @Override - public void write(byte[] key, byte[] value) throws IOException { + public void write(byte[] key, byte[] value) { if (currentKey != null && !Arrays.equals(key, currentKey)) { flush(); } @@ -122,16 +123,16 @@ public void write(byte[] key, byte[] value) throws IOException { } @Override - public void finish() throws IOException { + public void finish() { flush(); bulkLoader.finish(); } - private void flush() throws IOException { + private void flush() { if (currentKey != null && values.size() > 0) { try { - bulkLoader.write(currentKey, state.serializeList(values)); - } catch (BulkLoader.WriteException e) { + bulkLoader.write(currentKey, values); + } catch (RocksDBBulkLoader.WriteException e) { throw new RuntimeException(e); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PrimaryKeyLookupTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PrimaryKeyLookupTable.java index 2a3099e9a62b..485a9cfaa6c1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PrimaryKeyLookupTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PrimaryKeyLookupTable.java @@ -20,8 +20,9 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.serializer.InternalSerializers; -import org.apache.paimon.lookup.BulkLoader; -import org.apache.paimon.lookup.RocksDBValueState; +import org.apache.paimon.lookup.ValueBulkLoader; +import org.apache.paimon.lookup.ValueState; +import org.apache.paimon.lookup.rocksdb.RocksDBBulkLoader; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.RowKind; @@ -44,7 +45,7 @@ public class PrimaryKeyLookupTable extends FullCacheLookupTable { @Nullable private final ProjectedRow keyRearrange; - protected RocksDBValueState tableState; + protected ValueState tableState; public PrimaryKeyLookupTable(Context context, long lruCacheSize, List joinKey) { super(context); @@ -129,12 +130,12 @@ public byte[] toValueBytes(InternalRow row) throws IOException { @Override public TableBulkLoader createBulkLoader() { - BulkLoader bulkLoader = tableState.createBulkLoader(); + ValueBulkLoader bulkLoader = tableState.createBulkLoader(); return new TableBulkLoader() { @Override public void write(byte[] key, byte[] value) - throws BulkLoader.WriteException, IOException { + throws RocksDBBulkLoader.WriteException, IOException { bulkLoader.write(key, value); bulkLoadWritePlus(key, value); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/SecondaryIndexLookupTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/SecondaryIndexLookupTable.java index 11c9cba24b39..668570d77d12 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/SecondaryIndexLookupTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/SecondaryIndexLookupTable.java @@ -20,7 +20,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.serializer.InternalSerializers; -import org.apache.paimon.lookup.RocksDBSetState; +import org.apache.paimon.lookup.SetState; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.types.RowKind; import org.apache.paimon.utils.KeyProjectedRow; @@ -35,7 +35,7 @@ public class SecondaryIndexLookupTable extends PrimaryKeyLookupTable { private final KeyProjectedRow secKeyRow; - private RocksDBSetState indexState; + private SetState indexState; public SecondaryIndexLookupTable(Context context, long lruCacheSize) { super(context, lruCacheSize / 2, context.table.primaryKeys()); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/PostponeBucketCompactSplitSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/PostponeBucketCompactSplitSource.java index 53bc7a50a7b2..833eff81207d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/PostponeBucketCompactSplitSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/PostponeBucketCompactSplitSource.java @@ -155,7 +155,7 @@ public static Pair, DataStream> buildSource( table.fullName(), partitionSpec), InternalTypeInfo.of( LogicalTypeConversion.toLogicalType(table.rowType())), - new ReadOperator(table::newRead, null)), + new ReadOperator(table::newRead, null, null)), source.forward() .transform( "Remove new files", diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendTableSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendTableSink.java index 24d0e2db6ff1..094d61f6ead8 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendTableSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendTableSink.java @@ -48,6 +48,8 @@ */ public abstract class AppendTableSink extends FlinkWriteSink { + private static final long serialVersionUID = 1L; + protected final FileStoreTable table; protected final LogSinkFunction logSinkFunction; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java index e83bbeecbbe4..348421001418 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java @@ -30,6 +30,7 @@ import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; +import org.apache.paimon.table.sink.ChannelComputer; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.streaming.api.datastream.DataStream; @@ -291,9 +292,15 @@ protected DataStreamSink buildForFixedBucket(DataStream input) { } private DataStreamSink buildPostponeBucketSink(DataStream input) { - DataStream partitioned = - partition(input, new PostponeBucketChannelComputer(table.schema()), parallelism); - FixedBucketSink sink = new FixedBucketSink(table, overwritePartition, null); + ChannelComputer channelComputer; + if (!table.partitionKeys().isEmpty() + && table.coreOptions().partitionSinkStrategy() == PartitionSinkStrategy.HASH) { + channelComputer = new RowDataHashPartitionChannelComputer(table.schema()); + } else { + channelComputer = new PostponeBucketChannelComputer(table.schema()); + } + DataStream partitioned = partition(input, channelComputer, parallelism); + PostponeBucketSink sink = new PostponeBucketSink(table, overwritePartition); return sink.sinkFrom(partitioned); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkWriteSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkWriteSink.java index 74f1febd9040..8dc2044734ba 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkWriteSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkWriteSink.java @@ -18,11 +18,17 @@ package org.apache.paimon.flink.sink; +import org.apache.paimon.data.InternalRow; import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.manifest.ManifestCommittableSerializer; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; + import javax.annotation.Nullable; import java.util.Map; @@ -63,4 +69,48 @@ protected CommittableStateManager createCommittableStateMan ManifestCommittableSerializer::new, options.get(PARTITION_MARK_DONE_RECOVER_FROM_STATE)); } + + protected static OneInputStreamOperatorFactory + createNoStateRowWriteOperatorFactory( + FileStoreTable table, + LogSinkFunction logSinkFunction, + StoreSinkWrite.Provider writeProvider, + String commitUser) { + return new RowDataStoreWriteOperator.Factory( + table, logSinkFunction, writeProvider, commitUser) { + @Override + @SuppressWarnings("unchecked, rawtypes") + public StreamOperator createStreamOperator(StreamOperatorParameters parameters) { + return new RowDataStoreWriteOperator( + parameters, table, logSinkFunction, writeProvider, commitUser) { + + @Override + protected StoreSinkWriteState createState( + int subtaskId, + StateInitializationContext context, + StoreSinkWriteState.StateValueFilter stateFilter) { + // No conflicts will occur in append only unaware bucket writer, so no state + // is needed. + return new NoopStoreSinkWriteState(subtaskId, stateFilter); + } + + @Override + protected String getCommitUser(StateInitializationContext context) + throws Exception { + // No conflicts will occur in append only unaware bucket writer, so + // commitUser does not matter. + return commitUser; + } + }; + } + }; + } + + protected static CommittableStateManager + createRestoreOnlyCommittableStateManager(FileStoreTable table) { + Options options = table.coreOptions().toConfiguration(); + return new RestoreCommittableStateManager<>( + ManifestCommittableSerializer::new, + options.get(PARTITION_MARK_DONE_RECOVER_FROM_STATE)); + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PostponeBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PostponeBucketSink.java new file mode 100644 index 000000000000..1ab6f398c5c6 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PostponeBucketSink.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.sink; + +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.manifest.ManifestCommittable; +import org.apache.paimon.table.FileStoreTable; + +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; + +import javax.annotation.Nullable; + +import java.util.Map; + +/** {@link FlinkSink} for writing records into fixed bucket Paimon table. */ +public class PostponeBucketSink extends FlinkWriteSink { + + private static final long serialVersionUID = 1L; + + public PostponeBucketSink( + FileStoreTable table, @Nullable Map overwritePartition) { + super(table, overwritePartition); + } + + @Override + protected OneInputStreamOperatorFactory createWriteOperatorFactory( + StoreSinkWrite.Provider writeProvider, String commitUser) { + return createNoStateRowWriteOperatorFactory(table, null, writeProvider, commitUser); + } + + @Override + protected CommittableStateManager createCommittableStateManager() { + return createRestoreOnlyCommittableStateManager(table); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RestoreAndFailCommittableStateManager.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RestoreAndFailCommittableStateManager.java index a9b0922d0b90..8556cb6e7765 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RestoreAndFailCommittableStateManager.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RestoreAndFailCommittableStateManager.java @@ -19,18 +19,9 @@ package org.apache.paimon.flink.sink; import org.apache.paimon.data.serializer.VersionedSerializer; -import org.apache.paimon.flink.VersionedSerializerWrapper; import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.utils.SerializableSupplier; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateSnapshotContext; -import org.apache.flink.streaming.api.operators.util.SimpleVersionedListState; - -import java.util.ArrayList; import java.util.List; /** @@ -44,52 +35,20 @@ * store writers. */ public class RestoreAndFailCommittableStateManager - implements CommittableStateManager { + extends RestoreCommittableStateManager { private static final long serialVersionUID = 1L; - /** The committable's serializer. */ - private final SerializableSupplier> committableSerializer; - - private final boolean partitionMarkDoneRecoverFromState; - - /** GlobalCommitT state of this job. Used to filter out previous successful commits. */ - private ListState streamingCommitterState; - - public RestoreAndFailCommittableStateManager( - SerializableSupplier> committableSerializer) { - this(committableSerializer, true); - } - public RestoreAndFailCommittableStateManager( SerializableSupplier> committableSerializer, boolean partitionMarkDoneRecoverFromState) { - this.committableSerializer = committableSerializer; - this.partitionMarkDoneRecoverFromState = partitionMarkDoneRecoverFromState; + super(committableSerializer, partitionMarkDoneRecoverFromState); } @Override - public void initializeState( - StateInitializationContext context, Committer committer) + protected int recover(List committables, Committer committer) throws Exception { - streamingCommitterState = - new SimpleVersionedListState<>( - context.getOperatorStateStore() - .getListState( - new ListStateDescriptor<>( - "streaming_committer_raw_states", - BytePrimitiveArraySerializer.INSTANCE)), - new VersionedSerializerWrapper<>(committableSerializer.get())); - List restored = new ArrayList<>(); - streamingCommitterState.get().forEach(restored::add); - streamingCommitterState.clear(); - recover(restored, committer); - } - - private void recover(List committables, Committer committer) - throws Exception { - int numCommitted = - committer.filterAndCommit(committables, true, partitionMarkDoneRecoverFromState); + int numCommitted = super.recover(committables, committer); if (numCommitted > 0) { throw new RuntimeException( "This exception is intentionally thrown " @@ -97,11 +56,6 @@ private void recover(List committables, Committer committables) - throws Exception { - streamingCommitterState.update(committables); + return numCommitted; } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RestoreCommittableStateManager.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RestoreCommittableStateManager.java new file mode 100644 index 000000000000..b1ed396bdcae --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RestoreCommittableStateManager.java @@ -0,0 +1,89 @@ +/* + * 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.paimon.flink.sink; + +import org.apache.paimon.data.serializer.VersionedSerializer; +import org.apache.paimon.flink.VersionedSerializerWrapper; +import org.apache.paimon.manifest.ManifestCommittable; +import org.apache.paimon.utils.SerializableSupplier; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.util.SimpleVersionedListState; + +import java.util.ArrayList; +import java.util.List; + +/** + * A {@link CommittableStateManager} which stores uncommitted {@link ManifestCommittable}s in state. + * + *

When the job restarts, these {@link ManifestCommittable}s will be restored and committed. + */ +public class RestoreCommittableStateManager + implements CommittableStateManager { + + private static final long serialVersionUID = 1L; + + /** The committable's serializer. */ + private final SerializableSupplier> committableSerializer; + + private final boolean partitionMarkDoneRecoverFromState; + + /** GlobalCommitT state of this job. Used to filter out previous successful commits. */ + private ListState streamingCommitterState; + + public RestoreCommittableStateManager( + SerializableSupplier> committableSerializer, + boolean partitionMarkDoneRecoverFromState) { + this.committableSerializer = committableSerializer; + this.partitionMarkDoneRecoverFromState = partitionMarkDoneRecoverFromState; + } + + @Override + public void initializeState( + StateInitializationContext context, Committer committer) + throws Exception { + streamingCommitterState = + new SimpleVersionedListState<>( + context.getOperatorStateStore() + .getListState( + new ListStateDescriptor<>( + "streaming_committer_raw_states", + BytePrimitiveArraySerializer.INSTANCE)), + new VersionedSerializerWrapper<>(committableSerializer.get())); + List restored = new ArrayList<>(); + streamingCommitterState.get().forEach(restored::add); + streamingCommitterState.clear(); + recover(restored, committer); + } + + protected int recover(List committables, Committer committer) + throws Exception { + return committer.filterAndCommit(committables, true, partitionMarkDoneRecoverFromState); + } + + @Override + public void snapshotState(StateSnapshotContext context, List committables) + throws Exception { + streamingCommitterState.update(committables); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowAppendTableSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowAppendTableSink.java index 69a339a411b1..a58839a84147 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowAppendTableSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowAppendTableSink.java @@ -19,18 +19,18 @@ package org.apache.paimon.flink.sink; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.table.FileStoreTable; -import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import java.util.Map; /** An {@link AppendTableSink} which handles {@link InternalRow}. */ public class RowAppendTableSink extends AppendTableSink { + private static final long serialVersionUID = 1L; + public RowAppendTableSink( FileStoreTable table, Map overwritePartitions, @@ -42,34 +42,12 @@ public RowAppendTableSink( @Override protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new RowDataStoreWriteOperator.Factory( - table, logSinkFunction, writeProvider, commitUser) { - @Override - @SuppressWarnings("unchecked, rawtypes") - public StreamOperator createStreamOperator(StreamOperatorParameters parameters) { - return new RowDataStoreWriteOperator( - parameters, table, logSinkFunction, writeProvider, commitUser) { - - @Override - protected StoreSinkWriteState createState( - int subtaskId, - StateInitializationContext context, - StoreSinkWriteState.StateValueFilter stateFilter) - throws Exception { - // No conflicts will occur in append only unaware bucket writer, so no state - // is needed. - return new NoopStoreSinkWriteState(subtaskId, stateFilter); - } + return createNoStateRowWriteOperatorFactory( + table, logSinkFunction, writeProvider, commitUser); + } - @Override - protected String getCommitUser(StateInitializationContext context) - throws Exception { - // No conflicts will occur in append only unaware bucket writer, so - // commitUser does not matter. - return commitUser; - } - }; - } - }; + @Override + protected CommittableStateManager createCommittableStateManager() { + return createRestoreOnlyCommittableStateManager(table); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataHashPartitionChannelComputer.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataHashPartitionChannelComputer.java index 73258e29669e..8943c549cef9 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataHashPartitionChannelComputer.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataHashPartitionChannelComputer.java @@ -20,9 +20,8 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.schema.TableSchema; -import org.apache.paimon.table.sink.AppendTableRowKeyExtractor; import org.apache.paimon.table.sink.ChannelComputer; -import org.apache.paimon.table.sink.KeyAndBucketExtractor; +import org.apache.paimon.table.sink.RowPartitionKeyExtractor; /** This is only for partitioned unaware-buckets Append only table. */ public class RowDataHashPartitionChannelComputer implements ChannelComputer { @@ -32,7 +31,7 @@ public class RowDataHashPartitionChannelComputer implements ChannelComputer extractor; + private transient RowPartitionKeyExtractor extractor; public RowDataHashPartitionChannelComputer(TableSchema schema) { this.schema = schema; @@ -41,13 +40,12 @@ public RowDataHashPartitionChannelComputer(TableSchema schema) { @Override public void setup(int numChannels) { this.numChannels = numChannels; - this.extractor = new AppendTableRowKeyExtractor(schema); + this.extractor = new RowPartitionKeyExtractor(schema); } @Override public int channel(InternalRow record) { - extractor.setRecord(record); - return ChannelComputer.select(extractor.partition(), 0, numChannels); + return ChannelComputer.select(extractor.partition(record), 0, numChannels); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/ContinuousFileStoreSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/ContinuousFileStoreSource.java index 358731374813..2c1273e811c3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/ContinuousFileStoreSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/ContinuousFileStoreSource.java @@ -111,7 +111,7 @@ protected SplitEnumerator buildEn scan, unawareBucket, options.get(CoreOptions.SCAN_MAX_SPLITS_PER_TASK), - options.get(FlinkConnectorOptions.STREAMING_READ_SHUFFLE_BUCKET_WITH_PARTITION), + options.get(FlinkConnectorOptions.READ_SHUFFLE_BUCKET_WITH_PARTITION), options.get(FlinkConnectorOptions.SCAN_MAX_SNAPSHOT_COUNT)); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java index feb9176c8549..44b579047218 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java @@ -78,6 +78,7 @@ * @since 0.8 */ public class FlinkSourceBuilder { + private static final String SOURCE_NAME = "Source"; private final Table table; @@ -294,6 +295,9 @@ public DataStream build() { } if (sourceBounded) { + if (conf.get(FlinkConnectorOptions.SCAN_DEDICATED_SPLIT_GENERATION)) { + return buildDedicatedSplitGenSource(true); + } return buildStaticFileSource(); } TableScanUtils.streamingReadingValidate(table); @@ -325,16 +329,16 @@ public DataStream build() { } else if (conf.contains(CoreOptions.CONSUMER_ID) && conf.get(CoreOptions.CONSUMER_CONSISTENCY_MODE) == CoreOptions.ConsumerMode.EXACTLY_ONCE) { - return buildContinuousStreamOperator(); + return buildDedicatedSplitGenSource(false); } else { return buildContinuousFileSource(); } } } - private DataStream buildContinuousStreamOperator() { + private DataStream buildDedicatedSplitGenSource(boolean isBounded) { DataStream dataStream; - if (limit != null) { + if (limit != null && !isBounded) { throw new IllegalArgumentException( "Cannot limit streaming source, please use batch execution mode."); } @@ -346,10 +350,11 @@ private DataStream buildContinuousStreamOperator() { createReadBuilder(projectedRowType()), conf.get(CoreOptions.CONTINUOUS_DISCOVERY_INTERVAL).toMillis(), watermarkStrategy == null, - conf.get( - FlinkConnectorOptions.STREAMING_READ_SHUFFLE_BUCKET_WITH_PARTITION), + conf.get(FlinkConnectorOptions.READ_SHUFFLE_BUCKET_WITH_PARTITION), unawareBucket, - outerProject()); + outerProject(), + isBounded, + limit); if (parallelism != null) { dataStream.getTransformation().setParallelism(parallelism); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java index 29ff63a31c37..59845901ff12 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java @@ -84,7 +84,7 @@ protected SplitEnumerator buildEn unawareBucket, options.get(FlinkConnectorOptions.SOURCE_CHECKPOINT_ALIGN_TIMEOUT).toMillis(), options.get(CoreOptions.SCAN_MAX_SPLITS_PER_TASK), - options.get(FlinkConnectorOptions.STREAMING_READ_SHUFFLE_BUCKET_WITH_PARTITION), + options.get(FlinkConnectorOptions.READ_SHUFFLE_BUCKET_WITH_PARTITION), options.get(FlinkConnectorOptions.SCAN_MAX_SNAPSHOT_COUNT)); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java index 228d0d5a4404..d664b526fffb 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java @@ -30,6 +30,7 @@ import org.apache.paimon.table.source.ReadBuilder; import org.apache.paimon.table.source.Split; import org.apache.paimon.table.source.StreamTableScan; +import org.apache.paimon.table.source.TableScan; import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.common.eventtime.WatermarkStrategy; @@ -48,6 +49,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.List; import java.util.NavigableMap; @@ -59,20 +62,22 @@ * *

    *
  1. Monitoring snapshots of the Paimon table. - *
  2. Creating the {@link Split splits} corresponding to the incremental files + *
  3. Creating the {@link Split splits} corresponding to the DateFiles *
  4. Assigning them to downstream tasks for further processing. *
* *

The splits to be read are forwarded to the downstream {@link ReadOperator} which can have * parallelism greater than one. * - *

Currently, there are two features that rely on this monitor: + *

Currently, there are three features that rely on this monitor: * *

    *
  1. Consumer-id: rely on this source to do aligned snapshot consumption, and ensure that all * data in a snapshot is consumed within each checkpoint. *
  2. Snapshot-watermark: when there is no watermark definition, the default Paimon table will * pass the watermark recorded in the snapshot. + *
  3. Optimize-coordinator-memory: rely on this source to get splits on a single task, this can + * reduce the memory pressure of source coordinator. *
*/ public class MonitorSource extends AbstractNonCoordinatedSource { @@ -84,17 +89,22 @@ public class MonitorSource extends AbstractNonCoordinatedSource { private final ReadBuilder readBuilder; private final long monitorInterval; private final boolean emitSnapshotWatermark; + private final boolean isBounded; public MonitorSource( - ReadBuilder readBuilder, long monitorInterval, boolean emitSnapshotWatermark) { + ReadBuilder readBuilder, + long monitorInterval, + boolean emitSnapshotWatermark, + boolean isBounded) { this.readBuilder = readBuilder; this.monitorInterval = monitorInterval; this.emitSnapshotWatermark = emitSnapshotWatermark; + this.isBounded = isBounded; } @Override public Boundedness getBoundedness() { - return Boundedness.CONTINUOUS_UNBOUNDED; + return isBounded ? Boundedness.BOUNDED : Boundedness.CONTINUOUS_UNBOUNDED; } @Override @@ -108,6 +118,7 @@ private class Reader extends AbstractNonCoordinatedSourceReader { private static final String NEXT_SNAPSHOT_STATE = "NSS"; private final StreamTableScan scan = readBuilder.newStreamScan(); + private final TableScan batchScan = readBuilder.newScan(); private final SplitListState checkpointState = new SplitListState<>(CHECKPOINT_STATE, x -> Long.toString(x), Long::parseLong); private final SplitListState> nextSnapshotState = @@ -178,11 +189,11 @@ public void addSplits(List list) { public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { boolean isEmpty; try { - List splits = scan.plan().splits(); + List splits = isBounded ? batchScan.plan().splits() : scan.plan().splits(); isEmpty = splits.isEmpty(); splits.forEach(readerOutput::collect); - if (emitSnapshotWatermark) { + if (emitSnapshotWatermark && !isBounded) { Long watermark = scan.watermark(); if (watermark != null) { readerOutput.emitWatermark(new Watermark(watermark)); @@ -193,6 +204,10 @@ public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { return InputStatus.END_OF_INPUT; } + if (isBounded) { + return InputStatus.END_OF_INPUT; + } + if (isEmpty) { Thread.sleep(monitorInterval); } @@ -209,11 +224,16 @@ public static DataStream buildSource( boolean emitSnapshotWatermark, boolean shuffleBucketWithPartition, boolean unawareBucket, - NestedProjectedRowData nestedProjectedRowData) { + NestedProjectedRowData nestedProjectedRowData, + boolean isBounded, + @Nullable Long limit) { SingleOutputStreamOperator singleOutputStreamOperator = env.fromSource( new MonitorSource( - readBuilder, monitorInterval, emitSnapshotWatermark), + readBuilder, + monitorInterval, + emitSnapshotWatermark, + isBounded), WatermarkStrategy.noWatermarks(), name + "-Monitor", new JavaTypeInfo<>(Split.class)) @@ -228,7 +248,7 @@ public static DataStream buildSource( return sourceDataStream.transform( name + "-Reader", typeInfo, - new ReadOperator(readBuilder::newRead, nestedProjectedRowData)); + new ReadOperator(readBuilder::newRead, nestedProjectedRowData, limit)); } private static DataStream shuffleUnawareBucket( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java index 685b32408802..a9b9767041e6 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java @@ -36,6 +36,8 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.data.RowData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import javax.annotation.Nullable; @@ -47,6 +49,8 @@ public class ReadOperator extends AbstractStreamOperator implements OneInputStreamOperator { + private static final Logger LOG = LoggerFactory.getLogger(ReadOperator.class); + private static final long serialVersionUID = 2L; private final SerializableSupplier readSupplier; @@ -64,12 +68,15 @@ public class ReadOperator extends AbstractStreamOperator private transient long emitEventTimeLag = FileStoreSourceReaderMetrics.UNDEFINED; private transient long idleStartTime = FileStoreSourceReaderMetrics.ACTIVE; private transient Counter numRecordsIn; + @Nullable private final Long limit; public ReadOperator( SerializableSupplier readSupplier, - @Nullable NestedProjectedRowData nestedProjectedRowData) { + @Nullable NestedProjectedRowData nestedProjectedRowData, + @Nullable Long limit) { this.readSupplier = readSupplier; this.nestedProjectedRowData = nestedProjectedRowData; + this.limit = limit; } @Override @@ -98,6 +105,10 @@ public void open() throws Exception { @Override public void processElement(StreamRecord record) throws Exception { + if (reachLimit()) { + return; + } + Split split = record.getValue(); // update metric when reading a new split long eventTime = @@ -122,6 +133,10 @@ public void processElement(StreamRecord record) throws Exception { numRecordsIn.inc(); } + if (reachLimit()) { + return; + } + reuseRow.replace(iterator.next()); if (nestedProjectedRowData == null) { reuseRecord.replace(reuseRow); @@ -144,6 +159,14 @@ public void close() throws Exception { } } + private boolean reachLimit() { + if (limit != null && numRecordsIn.getCount() > limit) { + LOG.info("Reader {} reach the limit record {}.", this, limit); + return true; + } + return false; + } + private void idlingStarted() { if (!isIdling()) { idleStartTime = System.currentTimeMillis(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableScanUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableScanUtils.java index 30b7bbdd5dc5..ca901ad1bfdb 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableScanUtils.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableScanUtils.java @@ -30,12 +30,14 @@ import java.util.Optional; import java.util.Set; +import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_DEDICATED_SPLIT_GENERATION; + /** Utility methods for {@link TableScan}, such as validating. */ public class TableScanUtils { public static void streamingReadingValidate(Table table) { - CoreOptions options = CoreOptions.fromMap(table.options()); - CoreOptions.MergeEngine mergeEngine = options.mergeEngine(); + CoreOptions coreOptions = CoreOptions.fromMap(table.options()); + CoreOptions.MergeEngine mergeEngine = coreOptions.mergeEngine(); HashMap mergeEngineDesc = new HashMap() { { @@ -45,7 +47,7 @@ public static void streamingReadingValidate(Table table) { } }; if (table.primaryKeys().size() > 0 && mergeEngineDesc.containsKey(mergeEngine)) { - if (options.changelogProducer() == CoreOptions.ChangelogProducer.NONE) { + if (coreOptions.changelogProducer() == CoreOptions.ChangelogProducer.NONE) { throw new RuntimeException( mergeEngineDesc.get(mergeEngine) + " streaming reading is not supported. You can use " @@ -53,6 +55,14 @@ public static void streamingReadingValidate(Table table) { + "('input' changelog producer is also supported, but only returns input records.)"); } } + + Options options = Options.fromMap(table.options()); + if (options.get(SCAN_DEDICATED_SPLIT_GENERATION)) { + throw new RuntimeException( + "The option " + + SCAN_DEDICATED_SPLIT_GENERATION.key() + + " can only used in batch mode."); + } } /** Get snapshot id from {@link FileStoreSourceSplit}. */ diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java index 2b846b68bcc0..e431b6984111 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java @@ -815,4 +815,42 @@ public void testBinlogTableWithProjection() { assertThat(sql("SELECT rowkind, b FROM `test_table$binlog`")) .containsExactly(Row.of("+I", new String[] {"A"})); } + + @Test + public void testBatchReadSourceWithSnapshot() { + batchSql("INSERT INTO T VALUES (1, 11, 111), (2, 22, 222), (3, 33, 333), (4, 44, 444)"); + assertThat( + batchSql( + "SELECT * FROM T /*+ OPTIONS('scan.snapshot-id'='1', 'scan.dedicated-split-generation'='true') */")) + .containsExactlyInAnyOrder( + Row.of(1, 11, 111), + Row.of(2, 22, 222), + Row.of(3, 33, 333), + Row.of(4, 44, 444)); + + batchSql("INSERT INTO T VALUES (5, 55, 555), (6, 66, 666)"); + assertThat( + batchSql( + "SELECT * FROM T /*+ OPTIONS('scan.dedicated-split-generation'='true') */")) + .containsExactlyInAnyOrder( + Row.of(1, 11, 111), + Row.of(2, 22, 222), + Row.of(3, 33, 333), + Row.of(4, 44, 444), + Row.of(5, 55, 555), + Row.of(6, 66, 666)); + + assertThat( + batchSql( + "SELECT * FROM T /*+ OPTIONS('scan.dedicated-split-generation'='true') */ limit 2")) + .containsExactlyInAnyOrder(Row.of(1, 11, 111), Row.of(2, 22, 222)); + } + + @Test + public void testBatchReadSourceWithoutSnapshot() { + assertThat( + batchSql( + "SELECT * FROM T /*+ OPTIONS('scan.dedicated-split-generation'='true') */")) + .hasSize(0); + } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/LookupJoinITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/LookupJoinITCase.java index dea30d4d13d2..652ef5bd9c93 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/LookupJoinITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/LookupJoinITCase.java @@ -57,8 +57,8 @@ private void initTable(LookupCacheMode cacheMode) { + "PARTITIONED BY (`i`) WITH ('continuous.discovery-interval'='1 ms' %s)"; String fullOption = ", 'lookup.cache' = 'full'"; - String lruOption = ", 'changelog-producer'='lookup'"; + String memoryOption = ", 'lookup.cache' = 'memory'"; switch (cacheMode) { case FULL: @@ -69,6 +69,10 @@ private void initTable(LookupCacheMode cacheMode) { tEnv.executeSql(String.format(dim, lruOption)); tEnv.executeSql(String.format(partitioned, lruOption)); break; + case MEMORY: + tEnv.executeSql(String.format(dim, memoryOption)); + tEnv.executeSql(String.format(partitioned, memoryOption)); + break; default: throw new UnsupportedOperationException(); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PostponeBucketTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PostponeBucketTableITCase.java index cf319ac1c272..f66c2f099837 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PostponeBucketTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PostponeBucketTableITCase.java @@ -31,6 +31,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; import static org.assertj.core.api.Assertions.assertThat; @@ -75,7 +76,15 @@ public void testWriteThenCompact() throws Exception { values.add(String.format("(%d, %d, %d)", i, j, i * numKeys + j)); } } - tEnv.executeSql("INSERT INTO T VALUES " + String.join(", ", values)).await(); + ThreadLocalRandom random = ThreadLocalRandom.current(); + if (random.nextBoolean()) { + tEnv.executeSql("INSERT INTO T VALUES " + String.join(", ", values)).await(); + } else { + tEnv.executeSql( + "INSERT INTO T /*+ OPTIONS('partition.sink-strategy'='hash') */ VALUES " + + String.join(", ", values)) + .await(); + } assertThat(collect(tEnv.executeSql("SELECT * FROM T"))).isEmpty(); tEnv.executeSql("CALL sys.compact(`table` => 'default.T')").await(); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/FileStoreLookupFunctionTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/FileStoreLookupFunctionTest.java index dcbc405d316f..4f7ff334f677 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/FileStoreLookupFunctionTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/FileStoreLookupFunctionTest.java @@ -26,7 +26,7 @@ import org.apache.paimon.flink.lookup.PrimaryKeyPartialLookupTable.LocalQueryExecutor; import org.apache.paimon.flink.lookup.PrimaryKeyPartialLookupTable.QueryExecutor; import org.apache.paimon.flink.lookup.PrimaryKeyPartialLookupTable.RemoteQueryExecutor; -import org.apache.paimon.lookup.RocksDBOptions; +import org.apache.paimon.lookup.rocksdb.RocksDBOptions; import org.apache.paimon.options.Options; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaManager; diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/LookupTableTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/LookupTableTest.java index 88b947133087..4a3ca45c3b44 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/LookupTableTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/LookupTableTest.java @@ -42,6 +42,8 @@ import org.apache.paimon.table.sink.BatchTableCommit; import org.apache.paimon.table.sink.BatchTableWrite; import org.apache.paimon.table.sink.BatchWriteBuilder; +import org.apache.paimon.testutils.junit.parameterized.ParameterizedTestExtension; +import org.apache.paimon.testutils.junit.parameterized.Parameters; import org.apache.paimon.types.IntType; import org.apache.paimon.types.RowKind; import org.apache.paimon.types.RowType; @@ -50,15 +52,15 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; import org.testcontainers.shaded.com.google.common.collect.ImmutableList; import java.io.IOException; import java.nio.file.Path; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -71,21 +73,34 @@ import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; +import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_CACHE_MODE; +import static org.apache.paimon.flink.FlinkConnectorOptions.LookupCacheMode.FULL; +import static org.apache.paimon.flink.FlinkConnectorOptions.LookupCacheMode.MEMORY; import static org.apache.paimon.types.DataTypes.INT; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Test for {@link LookupTable}. */ +@ExtendWith(ParameterizedTestExtension.class) public class LookupTableTest extends TableTestBase { - @TempDir Path tempDir; + private final boolean inMemory; + @TempDir Path tempDir; private RowType rowType; - private IOManager ioManager; - private FullCacheLookupTable table; + public LookupTableTest(boolean inMemory) { + this.inMemory = inMemory; + } + + @SuppressWarnings("unused") + @Parameters(name = "{0}") + public static List getVarSeg() { + return Arrays.asList(true, false); + } + @BeforeEach public void before() throws IOException { this.rowType = RowType.of(new IntType(), new IntType(), new IntType()); @@ -100,6 +115,9 @@ public void after() throws IOException { } private FileStoreTable createTable(List primaryKeys, Options options) throws Exception { + if (inMemory) { + options.set(LOOKUP_CACHE_MODE, MEMORY); + } Identifier identifier = new Identifier("default", "t"); Schema schema = new Schema( @@ -112,7 +130,7 @@ private FileStoreTable createTable(List primaryKeys, Options options) th return (FileStoreTable) catalog.getTable(identifier); } - @Test + @TestTemplate public void testPkTable() throws Exception { FileStoreTable storeTable = createTable(singletonList("f0"), new Options()); FullCacheLookupTable.Context context = @@ -132,7 +150,7 @@ public void testPkTable() throws Exception { table.open(); // test bulk load error - { + if (!inMemory) { TableBulkLoader bulkLoader = table.createBulkLoader(); bulkLoader.write(new byte[] {1}, new byte[] {1}); assertThatThrownBy(() -> bulkLoader.write(new byte[] {1}, new byte[] {2})) @@ -172,7 +190,7 @@ public void testPkTable() throws Exception { assertThat(table.get(row(3))).hasSize(0); } - @Test + @TestTemplate public void testPkTableWithSequenceField() throws Exception { Options options = new Options(); options.set(CoreOptions.SEQUENCE_FIELD, "f1"); @@ -223,7 +241,7 @@ public void testPkTableWithSequenceField() throws Exception { assertRow(result.get(0), 1, 22, 222); } - @Test + @TestTemplate public void testPkTableWithSequenceFieldProjection() throws Exception { Options options = new Options(); options.set(CoreOptions.SEQUENCE_FIELD, "f2"); @@ -263,7 +281,7 @@ public void testPkTableWithSequenceFieldProjection() throws Exception { assertRow(result.get(0), 1, 22); } - @Test + @TestTemplate public void testPkTablePkFilter() throws Exception { FileStoreTable storeTable = createTable(singletonList("f0"), new Options()); FullCacheLookupTable.Context context = @@ -299,7 +317,7 @@ public void testPkTablePkFilter() throws Exception { assertThat(table.get(row(3))).hasSize(0); } - @Test + @TestTemplate public void testPkTableNonPkFilter() throws Exception { FileStoreTable storeTable = createTable(singletonList("f0"), new Options()); FullCacheLookupTable.Context context = @@ -328,7 +346,7 @@ public void testPkTableNonPkFilter() throws Exception { assertThat(result).hasSize(0); } - @Test + @TestTemplate public void testSecKeyTable() throws Exception { FileStoreTable storeTable = createTable(singletonList("f0"), new Options()); FullCacheLookupTable.Context context = @@ -376,7 +394,7 @@ public void testSecKeyTable() throws Exception { assertThat(result.stream().map(row -> row.getInt(0))).contains(1); } - @Test + @TestTemplate public void testSecKeyTableWithSequenceField() throws Exception { Options options = new Options(); options.set(CoreOptions.SEQUENCE_FIELD, "f1"); @@ -431,7 +449,7 @@ public void testSecKeyTableWithSequenceField() throws Exception { assertThat(result.stream().map(row -> row.getInt(2))).doesNotContain(333); } - @Test + @TestTemplate public void testSecKeyTablePkFilter() throws Exception { FileStoreTable storeTable = createTable(singletonList("f0"), new Options()); FullCacheLookupTable.Context context = @@ -476,7 +494,7 @@ public void testSecKeyTablePkFilter() throws Exception { assertThat(table.get(row(33))).hasSize(0); } - @Test + @TestTemplate public void testNoPrimaryKeyTable() throws Exception { FileStoreTable storeTable = createTable(emptyList(), new Options()); FullCacheLookupTable.Context context = @@ -524,7 +542,7 @@ public void testNoPrimaryKeyTable() throws Exception { assertThat(result.stream().map(row -> row.getInt(0))).contains(1); } - @Test + @TestTemplate public void testNoPrimaryKeyTableFilter() throws Exception { FileStoreTable storeTable = createTable(emptyList(), new Options()); FullCacheLookupTable.Context context = @@ -559,7 +577,7 @@ public void testNoPrimaryKeyTableFilter() throws Exception { assertRow(result.get(1), 1, 11, 111); } - @Test + @TestTemplate public void testPkTableWithCacheRowFilter() throws Exception { FileStoreTable storeTable = createTable(singletonList("f0"), new Options()); writeWithBucketAssigner( @@ -600,7 +618,7 @@ public void testPkTableWithCacheRowFilter() throws Exception { assertThat(res).isEmpty(); } - @Test + @TestTemplate public void testRefreshExecutorRebuildAfterReopen() throws Exception { Options options = new Options(); options.set(FlinkConnectorOptions.LOOKUP_REFRESH_ASYNC, true); @@ -635,7 +653,7 @@ public void testRefreshExecutorRebuildAfterReopen() throws Exception { assertRow(res.get(0), 1, 22, 222); } - @Test + @TestTemplate public void testNoPkTableWithCacheRowFilter() throws Exception { FileStoreTable storeTable = createTable(emptyList(), new Options()); writeWithBucketAssigner( @@ -676,7 +694,7 @@ public void testNoPkTableWithCacheRowFilter() throws Exception { assertThat(res).isEmpty(); } - @Test + @TestTemplate public void testSecKeyTableWithCacheRowFilter() throws Exception { FileStoreTable storeTable = createTable(singletonList("f0"), new Options()); writeWithBucketAssigner( @@ -717,7 +735,7 @@ public void testSecKeyTableWithCacheRowFilter() throws Exception { assertThat(res).isEmpty(); } - @Test + @TestTemplate public void testPartialLookupTable() throws Exception { FileStoreTable dimTable = createDimTable(); PrimaryKeyPartialLookupTable table = @@ -750,7 +768,7 @@ public void testPartialLookupTable() throws Exception { assertThat(result).hasSize(0); } - @Test + @TestTemplate public void testPartialLookupTableWithRowFilter() throws Exception { Options options = new Options(); options.set(CoreOptions.BUCKET.key(), "2"); @@ -771,7 +789,7 @@ public void testPartialLookupTableWithRowFilter() throws Exception { assertThat(result).isEmpty(); } - @Test + @TestTemplate public void testPartialLookupTableWithProjection() throws Exception { FileStoreTable dimTable = createDimTable(); PrimaryKeyPartialLookupTable table = @@ -803,7 +821,7 @@ public void testPartialLookupTableWithProjection() throws Exception { assertRow(result.get(0), 22, -2); } - @Test + @TestTemplate public void testPartialLookupTableJoinKeyOrder() throws Exception { FileStoreTable dimTable = createDimTable(); PrimaryKeyPartialLookupTable table = @@ -835,9 +853,17 @@ public void testPartialLookupTableJoinKeyOrder() throws Exception { assertRow(result.get(0), 22, -2); } - @ParameterizedTest - @ValueSource(booleans = {false, true}) - public void testPKLookupTableRefreshAsync(boolean refreshAsync) throws Exception { + @TestTemplate + public void testPKLookupTableNotRefreshAsync() throws Exception { + innerTestPKLookupTableRefreshAsync(false); + } + + @TestTemplate + public void testPKLookupTableRefreshAsync() throws Exception { + innerTestPKLookupTableRefreshAsync(true); + } + + private void innerTestPKLookupTableRefreshAsync(boolean refreshAsync) throws Exception { Options options = new Options(); options.set(FlinkConnectorOptions.LOOKUP_REFRESH_ASYNC, refreshAsync); FileStoreTable storeTable = createTable(singletonList("f0"), options); @@ -902,13 +928,11 @@ public void testPKLookupTableRefreshAsync(boolean refreshAsync) throws Exception table.close(); } - @Test + @TestTemplate public void testFullCacheLookupTableWithForceLookup() throws Exception { Options options = new Options(); options.set(CoreOptions.MERGE_ENGINE, CoreOptions.MergeEngine.PARTIAL_UPDATE); - options.set( - FlinkConnectorOptions.LOOKUP_CACHE_MODE, - FlinkConnectorOptions.LookupCacheMode.FULL); + options.set(LOOKUP_CACHE_MODE, inMemory ? MEMORY : FULL); options.set(CoreOptions.WRITE_ONLY, true); options.set(CoreOptions.FORCE_LOOKUP, true); options.set(CoreOptions.BUCKET, 1); @@ -964,7 +988,7 @@ public void testFullCacheLookupTableWithForceLookup() throws Exception { assertRow(result.get(0), 1, 22, 222); // new value } - @Test + @TestTemplate public void testPartialLookupTableWithForceLookup() throws Exception { Options options = new Options(); options.set(CoreOptions.MERGE_ENGINE, CoreOptions.MergeEngine.PARTIAL_UPDATE); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/RocksDBListStateTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/RocksDBListStateTest.java index 7a942ddb562c..d98d1996232b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/RocksDBListStateTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/RocksDBListStateTest.java @@ -24,8 +24,8 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.serializer.InternalRowSerializer; -import org.apache.paimon.lookup.RocksDBListState; -import org.apache.paimon.lookup.RocksDBStateFactory; +import org.apache.paimon.lookup.rocksdb.RocksDBListState; +import org.apache.paimon.lookup.rocksdb.RocksDBStateFactory; import org.apache.paimon.options.Options; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowKind; diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java index d79d13f0260c..02001ff1af28 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java @@ -20,6 +20,7 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.flink.CatalogITCaseBase; +import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.StreamTableScan; @@ -35,6 +36,7 @@ import org.junit.jupiter.api.Test; import java.util.List; +import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -196,6 +198,71 @@ public void testHistoryPartitionCompact() throws Exception { } } + @Test + public void testForceCompactToExternalPath() throws Exception { + // test for pk table + String tmpPath = getTempDirPath("external/" + UUID.randomUUID()); + sql( + "CREATE TABLE Tpk (" + + " k INT," + + " v INT," + + " hh INT," + + " dt STRING," + + " PRIMARY KEY (k, dt, hh) NOT ENFORCED" + + ") PARTITIONED BY (dt, hh) WITH (" + + " 'write-only' = 'true'," + + " 'bucket' = '1'" + + ")"); + FileStoreTable pkTable = paimonTable("Tpk"); + + sql( + "INSERT INTO Tpk VALUES (1, 100, 15, '20221208'), (1, 100, 16, '20221208'), (1, 100, 15, '20221209')"); + sql( + "INSERT INTO Tpk VALUES (2, 100, 15, '20221208'), (2, 100, 16, '20221208'), (2, 100, 15, '20221209')"); + tEnv.getConfig().set(TableConfigOptions.TABLE_DML_SYNC, true); + sql( + "CALL sys.compact(`table` => 'default.Tpk'," + + " options => 'compaction.force-rewrite-all-files=true,data-file.external-paths=file://%s,data-file.external-paths.strategy=specific-fs,data-file.external-paths.specific-fs=file')", + tmpPath); + List splits = pkTable.newSnapshotReader().read().dataSplits(); + for (DataSplit split : splits) { + for (DataFileMeta meta : split.dataFiles()) { + assertThat(meta.externalPath().get().startsWith("file:" + tmpPath)).isTrue(); + } + } + + // test for append table + tmpPath = getTempDirPath("external/" + UUID.randomUUID()); + sql( + "CREATE TABLE Tap (" + + " k INT," + + " v INT," + + " hh INT," + + " dt STRING" + + ") PARTITIONED BY (dt, hh) WITH (" + + " 'write-only' = 'true'," + + " 'bucket' = '1'," + + " 'bucket-key' = 'k'" + + ")"); + FileStoreTable apTable = paimonTable("Tap"); + + sql( + "INSERT INTO Tap VALUES (1, 100, 15, '20221208'), (1, 100, 16, '20221208'), (1, 100, 15, '20221209')"); + sql( + "INSERT INTO Tap VALUES (2, 100, 15, '20221208'), (2, 100, 16, '20221208'), (2, 100, 15, '20221209')"); + tEnv.getConfig().set(TableConfigOptions.TABLE_DML_SYNC, true); + sql( + "CALL sys.compact(`table` => 'default.Tap'," + + " options => 'compaction.force-rewrite-all-files=true,data-file.external-paths=file://%s,data-file.external-paths.strategy=specific-fs,data-file.external-paths.specific-fs=file')", + tmpPath); + splits = apTable.newSnapshotReader().read().dataSplits(); + for (DataSplit split : splits) { + for (DataFileMeta meta : split.dataFiles()) { + assertThat(meta.externalPath().get().startsWith("file:" + tmpPath)).isTrue(); + } + } + } + // ----------------------- Sort Compact ----------------------- @Test diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java index 32ee2f42af72..1bc68477a323 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java @@ -67,7 +67,7 @@ public void testBatchWriteGeneratorTag() throws Exception { table, initialCommitUser, new RestoreAndFailCommittableStateManager<>( - ManifestCommittableSerializer::new)); + ManifestCommittableSerializer::new, true)); OneInputStreamOperator committerOperator = committerOperatorFactory.createStreamOperator( @@ -143,7 +143,7 @@ public void testBatchWriteGeneratorCustomizedTag() throws Exception { table, initialCommitUser, new RestoreAndFailCommittableStateManager<>( - ManifestCommittableSerializer::new)); + ManifestCommittableSerializer::new, true)); OneInputStreamOperator committerOperator = committerOperatorFactory.createStreamOperator( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java index 4ad1dff9aafb..220af9a73bb6 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java @@ -665,7 +665,7 @@ public void testCommitMetrics() throws Exception { table, null, new RestoreAndFailCommittableStateManager<>( - ManifestCommittableSerializer::new)); + ManifestCommittableSerializer::new, true)); OneInputStreamOperatorTestHarness testHarness = createTestHarness(operatorFactory); testHarness.open(); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java index a61a379bde72..041a692d9e25 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java @@ -652,7 +652,7 @@ public void testCommitMetrics() throws Exception { initialCommitUser, context -> new StoreMultiCommitter(catalogLoader, context), new RestoreAndFailCommittableStateManager<>( - WrappedManifestCommittableSerializer::new)); + WrappedManifestCommittableSerializer::new, true)); return createTestHarness(operator); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java index 0d14b60e0ca8..e4ab4ec15799 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java @@ -122,13 +122,26 @@ private List> readSplit(Split split) throws IOException { return result; } + @Test + public void testMonitorSourceWhenIsBoundedIsTrue() throws Exception { + MonitorSource source = new MonitorSource(table.newReadBuilder(), 10, false, true); + TestingSourceOperator operator = + (TestingSourceOperator) + TestingSourceOperator.createTestOperator( + source.createReader(null), WatermarkStrategy.noWatermarks(), false); + AbstractStreamOperatorTestHarness testHarness = + new AbstractStreamOperatorTestHarness<>(operator, 1, 1, 0); + testHarness.open(); + testReadSplit(operator, () -> 1, 1, 1, 1); + } + @Test public void testMonitorSource() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); // 1. run first OperatorSubtaskState snapshot; { - MonitorSource source = new MonitorSource(table.newReadBuilder(), 10, false); + MonitorSource source = new MonitorSource(table.newReadBuilder(), 10, false, false); TestingSourceOperator operator = (TestingSourceOperator) TestingSourceOperator.createTestOperator( @@ -143,7 +156,7 @@ public void testMonitorSource() throws Exception { // 2. restore from state { - MonitorSource sourceCopy1 = new MonitorSource(table.newReadBuilder(), 10, false); + MonitorSource sourceCopy1 = new MonitorSource(table.newReadBuilder(), 10, false, false); TestingSourceOperator operatorCopy1 = (TestingSourceOperator) TestingSourceOperator.createTestOperator( @@ -168,7 +181,7 @@ public void testMonitorSource() throws Exception { // 3. restore from consumer id { - MonitorSource sourceCopy2 = new MonitorSource(table.newReadBuilder(), 10, false); + MonitorSource sourceCopy2 = new MonitorSource(table.newReadBuilder(), 10, false, false); TestingSourceOperator operatorCopy2 = (TestingSourceOperator) TestingSourceOperator.createTestOperator( @@ -184,7 +197,8 @@ public void testMonitorSource() throws Exception { @Test public void testReadOperator() throws Exception { - ReadOperator readOperator = new ReadOperator(() -> table.newReadBuilder().newRead(), null); + ReadOperator readOperator = + new ReadOperator(() -> table.newReadBuilder().newRead(), null, null); OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>(readOperator); harness.setup( @@ -204,9 +218,41 @@ public void testReadOperator() throws Exception { new StreamRecord<>(GenericRowData.of(2, 2, 2))); } + @Test + public void testReadOperatorWithLimit() throws Exception { + ReadOperator readOperator = + new ReadOperator(() -> table.newReadBuilder().newRead(), null, 2L); + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>(readOperator); + harness.setup( + InternalSerializers.create( + RowType.of(new IntType(), new IntType(), new IntType()))); + writeToTable(1, 1, 1); + writeToTable(2, 2, 2); + writeToTable(3, 3, 3); + writeToTable(4, 4, 4); + List splits = table.newReadBuilder().newScan().plan().splits(); + harness.open(); + for (Split split : splits) { + harness.processElement(new StreamRecord<>(split)); + } + ArrayList values = new ArrayList<>(harness.getOutput()); + + // In ReadOperator each Split is already counted as one input record. But in this case it + // will not happen. + // So in this case the result values's size if 3 even if the limit is 2. + // The IT case see BatchFileStoreITCase#testBatchReadSourceWithSnapshot. + assertThat(values) + .containsExactlyInAnyOrder( + new StreamRecord<>(GenericRowData.of(1, 1, 1)), + new StreamRecord<>(GenericRowData.of(2, 2, 2)), + new StreamRecord<>(GenericRowData.of(3, 3, 3))); + } + @Test public void testReadOperatorMetricsRegisterAndUpdate() throws Exception { - ReadOperator readOperator = new ReadOperator(() -> table.newReadBuilder().newRead(), null); + ReadOperator readOperator = + new ReadOperator(() -> table.newReadBuilder().newRead(), null, null); OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>(readOperator); harness.setup( @@ -305,6 +351,8 @@ public void emitRecordAttributes(RecordAttributes recordAttributes) {} public void emitWatermark(WatermarkEvent watermarkEvent) {} }; + writeToTable(a, b, c); + AtomicBoolean isRunning = new AtomicBoolean(true); Thread runner = new Thread( @@ -320,8 +368,6 @@ public void emitWatermark(WatermarkEvent watermarkEvent) {} }); runner.start(); - writeToTable(a, b, c); - Split split = queue.poll(1, TimeUnit.MINUTES); assertThat(readSplit(split)).containsExactlyInAnyOrder(Arrays.asList(a, b, c)); diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index e0f94d92dad8..35747cdb63ad 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -74,6 +74,7 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.UnknownTableException; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1358,6 +1359,22 @@ private boolean isExternalTable(Table table) { return table != null && TableType.EXTERNAL_TABLE.name().equals(table.getTableType()); } + private static String currentUser() { + String username = null; + try { + username = UserGroupInformation.getCurrentUser().getShortUserName(); + } catch (IOException e) { + LOG.warn("Failed to get Hadoop user", e); + } + + if (username != null) { + return username; + } else { + LOG.warn("Hadoop user is null, defaulting to user.name"); + return System.getProperty("user.name"); + } + } + private Table newHmsTable( Identifier identifier, Map tableParameters, @@ -1368,8 +1385,7 @@ private Table newHmsTable( new Table( identifier.getTableName(), identifier.getDatabaseName(), - // current linux user - System.getProperty("user.name"), + currentUser(), (int) (currentTimeMillis / 1000), (int) (currentTimeMillis / 1000), Integer.MAX_VALUE, diff --git a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala index df15d166a610..692c1f6e9221 100644 --- a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala +++ b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala @@ -30,7 +30,7 @@ class PaimonOptimizationTest extends PaimonOptimizationTestBase { fieldIndex: Int): NamedExpression = { GetStructField( ScalarSubquery( - SparkShimLoader.getSparkShim + SparkShimLoader.shim .createCTERelationRef(cteIndex, resolved = true, output.toSeq, isStreaming = false)), fieldIndex) .as("scalarsubquery()") diff --git a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala index d2a20d6df3c0..9bf2c684ec34 100644 --- a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala +++ b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala @@ -31,7 +31,7 @@ class PaimonOptimizationTest extends PaimonOptimizationTestBase { fieldIndex: Int): NamedExpression = { GetStructField( ScalarSubquery( - SparkShimLoader.getSparkShim + SparkShimLoader.shim .createCTERelationRef(cteIndex, resolved = true, output.toSeq, isStreaming = false)), fieldIndex) .as("scalarsubquery()") diff --git a/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala b/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala index df15d166a610..692c1f6e9221 100644 --- a/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala +++ b/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala @@ -30,7 +30,7 @@ class PaimonOptimizationTest extends PaimonOptimizationTestBase { fieldIndex: Int): NamedExpression = { GetStructField( ScalarSubquery( - SparkShimLoader.getSparkShim + SparkShimLoader.shim .createCTERelationRef(cteIndex, resolved = true, output.toSeq, isStreaming = false)), fieldIndex) .as("scalarsubquery()") diff --git a/paimon-spark/paimon-spark-4.0/pom.xml b/paimon-spark/paimon-spark-4.0/pom.xml index 11ae457db573..20b59531381d 100644 --- a/paimon-spark/paimon-spark-4.0/pom.xml +++ b/paimon-spark/paimon-spark-4.0/pom.xml @@ -32,7 +32,7 @@ under the License. Paimon : Spark : 4.0 - 4.0.0-preview2 + 4.0.0 @@ -81,6 +81,12 @@ under the License. ${spark.version} tests test + + + org.apache.spark + spark-connect-shims_${scala.binary.version} + + diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala index bbba0b0197fd..ec140a89bbd3 100644 --- a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala @@ -21,6 +21,7 @@ package org.apache.paimon.spark.sql import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.{Attribute, GetStructField, NamedExpression, ScalarSubquery} import org.apache.spark.sql.paimon.shims.SparkShimLoader + class PaimonOptimizationTest extends PaimonOptimizationTestBase { override def extractorExpression( @@ -29,9 +30,10 @@ class PaimonOptimizationTest extends PaimonOptimizationTestBase { fieldIndex: Int): NamedExpression = { GetStructField( ScalarSubquery( - SparkShimLoader.getSparkShim + SparkShimLoader.shim .createCTERelationRef(cteIndex, resolved = true, output.toSeq, isStreaming = false)), - fieldIndex) + fieldIndex, + None) .as("scalarsubquery()") } } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java index df03286fd059..695f03d9ca39 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java @@ -38,6 +38,7 @@ import org.apache.paimon.types.DataField; import org.apache.paimon.utils.TypeUtils; +import org.apache.spark.sql.PaimonSparkSession$; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException; import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException; @@ -110,7 +111,7 @@ public void initialize(String name, CaseInsensitiveStringMap options) { CatalogContext catalogContext = CatalogContext.create( Options.fromMap(options), - SparkSession.active().sessionState().newHadoopConf()); + PaimonSparkSession$.MODULE$.active().sessionState().newHadoopConf()); this.catalog = CatalogFactory.createCatalog(catalogContext); this.defaultDatabase = options.getOrDefault(DEFAULT_DATABASE.key(), DEFAULT_DATABASE.defaultValue()); @@ -475,6 +476,7 @@ protected org.apache.spark.sql.connector.catalog.Table loadSparkTable( } private static FileTable convertToFileTable(Identifier ident, FormatTable formatTable) { + SparkSession spark = PaimonSparkSession$.MODULE$.active(); StructType schema = SparkTypeUtils.fromPaimonRowType(formatTable.rowType()); StructType partitionSchema = SparkTypeUtils.fromPaimonRowType( @@ -488,7 +490,7 @@ private static FileTable convertToFileTable(Identifier ident, FormatTable format dsOptions = new CaseInsensitiveStringMap(options.toMap()); return new PartitionedCSVTable( ident.name(), - SparkSession.active(), + spark, dsOptions, scala.collection.JavaConverters.asScalaBuffer(pathList).toSeq(), scala.Option.apply(schema), @@ -497,7 +499,7 @@ private static FileTable convertToFileTable(Identifier ident, FormatTable format } else if (formatTable.format() == FormatTable.Format.ORC) { return new PartitionedOrcTable( ident.name(), - SparkSession.active(), + spark, dsOptions, scala.collection.JavaConverters.asScalaBuffer(pathList).toSeq(), scala.Option.apply(schema), @@ -506,7 +508,7 @@ private static FileTable convertToFileTable(Identifier ident, FormatTable format } else if (formatTable.format() == FormatTable.Format.PARQUET) { return new PartitionedParquetTable( ident.name(), - SparkSession.active(), + spark, dsOptions, scala.collection.JavaConverters.asScalaBuffer(pathList).toSeq(), scala.Option.apply(schema), @@ -515,7 +517,7 @@ private static FileTable convertToFileTable(Identifier ident, FormatTable format } else if (formatTable.format() == FormatTable.Format.JSON) { return new PartitionedJsonTable( ident.name(), - SparkSession.active(), + spark, dsOptions, scala.collection.JavaConverters.asScalaBuffer(pathList).toSeq(), scala.Option.apply(schema), diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java index ac1543f2fe17..e4563c492f73 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java @@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.spark.SparkConf; +import org.apache.spark.sql.PaimonSparkSession$; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException; import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException; @@ -202,7 +203,7 @@ public Table createTable( return sparkCatalog.createTable(ident, schema, partitions, properties); } else { // delegate to the session catalog - return SparkShimLoader.getSparkShim() + return SparkShimLoader.shim() .createTable(asTableCatalog(), ident, schema, partitions, properties); } } @@ -238,7 +239,7 @@ public void renameTable(Identifier from, Identifier to) @Override public final void initialize(String name, CaseInsensitiveStringMap options) { - SparkSession sparkSession = SparkSession.active(); + SparkSession sparkSession = PaimonSparkSession$.MODULE$.active(); SessionState sessionState = sparkSession.sessionState(); Configuration hadoopConf = sessionState.newHadoopConf(); if (options.containsKey(METASTORE.key()) diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRowWrapper.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRowWrapper.java index 9c9569c5735b..f42b1fa495c5 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRowWrapper.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRowWrapper.java @@ -152,7 +152,7 @@ public byte[] getBinary(int pos) { @Override public Variant getVariant(int pos) { - return SparkShimLoader.getSparkShim().toPaimonVariant(internalRow, pos); + return SparkShimLoader.shim().toPaimonVariant(internalRow, pos); } @Override @@ -307,7 +307,7 @@ public byte[] getBinary(int pos) { @Override public Variant getVariant(int pos) { - return SparkShimLoader.getSparkShim().toPaimonVariant(arrayData, pos); + return SparkShimLoader.shim().toPaimonVariant(arrayData, pos); } @Override diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java index 7d0d8ceb22a6..0fb3e2bdb386 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java @@ -147,7 +147,7 @@ public byte[] getBinary(int i) { @Override public Variant getVariant(int i) { - return SparkShimLoader.getSparkShim().toPaimonVariant(row.getAs(i)); + return SparkShimLoader.shim().toPaimonVariant(row.getAs(i)); } @Override @@ -309,7 +309,7 @@ public byte[] getBinary(int i) { @Override public Variant getVariant(int i) { - return SparkShimLoader.getSparkShim().toPaimonVariant(getAs(i)); + return SparkShimLoader.shim().toPaimonVariant(getAs(i)); } @Override diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/BaseProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/BaseProcedure.java index fe9d01971df3..2bf38c9fc7c4 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/BaseProcedure.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/BaseProcedure.java @@ -22,6 +22,7 @@ import org.apache.paimon.spark.SparkUtils; import org.apache.paimon.utils.Preconditions; +import org.apache.spark.sql.PaimonSparkSession$; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; @@ -30,8 +31,8 @@ import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.Table; import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.apache.spark.sql.execution.CacheManager; import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation; +import org.apache.spark.sql.paimon.shims.SparkShimLoader; import java.util.function.Function; @@ -44,7 +45,7 @@ abstract class BaseProcedure implements Procedure { private final TableCatalog tableCatalog; protected BaseProcedure(TableCatalog tableCatalog) { - this.spark = SparkSession.active(); + this.spark = PaimonSparkSession$.MODULE$.active(); this.tableCatalog = tableCatalog; } @@ -114,10 +115,9 @@ protected DataSourceV2Relation createRelation(Identifier ident) { } protected void refreshSparkCache(Identifier ident, Table table) { - CacheManager cacheManager = spark.sharedState().cacheManager(); DataSourceV2Relation relation = DataSourceV2Relation.create(table, Option.apply(tableCatalog), Option.apply(ident)); - cacheManager.recacheByPlan(spark, relation); + SparkShimLoader.shim().classicApi().recacheByPlan(spark, relation); } protected InternalRow newInternalRow(Object... values) { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ScanHelper.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ScanHelper.scala index e68407903dcb..0d3282b621a9 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ScanHelper.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ScanHelper.scala @@ -23,14 +23,14 @@ import org.apache.paimon.io.DataFileMeta import org.apache.paimon.table.source.{DataSplit, DeletionFile, Split} import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{PaimonSparkSession, SparkSession} import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer trait ScanHelper extends Logging { - private val spark = SparkSession.active + private val spark = PaimonSparkSession.active val coreOptions: CoreOptions diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala index cdcd2a1668d3..979191cfabba 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala @@ -29,7 +29,7 @@ import org.apache.paimon.table.{DataTable, FileStoreTable, FileStoreTableFactory import org.apache.paimon.table.FormatTable.Format import org.apache.paimon.table.system.AuditLogTable -import org.apache.spark.sql.{DataFrame, SaveMode => SparkSaveMode, SparkSession, SQLContext} +import org.apache.spark.sql.{DataFrame, PaimonSparkSession, SaveMode => SparkSaveMode, SparkSession, SQLContext} import org.apache.spark.sql.connector.catalog.{SessionConfigSupport, Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.streaming.Sink @@ -90,7 +90,7 @@ class SparkSource options, extractCatalogName().getOrElse(NAME), Identifier.create(CatalogUtils.database(path), CatalogUtils.table(path)))), - SparkSession.active.sessionState.newHadoopConf() + PaimonSparkSession.active.sessionState.newHadoopConf() ) val table = FileStoreTableFactory.create(catalogContext) if (Options.fromMap(options).get(SparkConnectorOptions.READ_CHANGELOG)) { diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkTypeUtils.java b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTypeUtils.java similarity index 99% rename from paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkTypeUtils.java rename to paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTypeUtils.java index f72924edce42..ae9588162188 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkTypeUtils.java +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTypeUtils.java @@ -221,7 +221,7 @@ public DataType visit(LocalZonedTimestampType localZonedTimestampType) { @Override public DataType visit(VariantType variantType) { - return SparkShimLoader.getSparkShim().SparkVariantType(); + return SparkShimLoader.shim().SparkVariantType(); } @Override @@ -365,12 +365,12 @@ public org.apache.paimon.types.DataType atomic(DataType atomic) { return new FloatType(); } else if (atomic instanceof org.apache.spark.sql.types.DoubleType) { return new DoubleType(); - } else if (atomic instanceof org.apache.spark.sql.types.StringType) { - return new VarCharType(VarCharType.MAX_LENGTH); } else if (atomic instanceof org.apache.spark.sql.types.VarcharType) { return new VarCharType(((org.apache.spark.sql.types.VarcharType) atomic).length()); } else if (atomic instanceof org.apache.spark.sql.types.CharType) { return new CharType(((org.apache.spark.sql.types.CharType) atomic).length()); + } else if (atomic instanceof org.apache.spark.sql.types.StringType) { + return new VarCharType(VarCharType.MAX_LENGTH); } else if (atomic instanceof org.apache.spark.sql.types.DateType) { return new DateType(); } else if (atomic instanceof org.apache.spark.sql.types.TimestampType) { @@ -388,7 +388,7 @@ public org.apache.paimon.types.DataType atomic(DataType atomic) { } else if (atomic instanceof org.apache.spark.sql.types.TimestampNTZType) { // Move TimestampNTZType to the end for compatibility with spark3.3 and below return new TimestampType(); - } else if (SparkShimLoader.getSparkShim().isSparkVariantType(atomic)) { + } else if (SparkShimLoader.shim().isSparkVariantType(atomic)) { return new VariantType(); } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala index 682cf88fcfbe..5540f58e0ebc 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala @@ -108,11 +108,11 @@ trait ExpressionHelperBase extends PredicateHelper { } def toColumn(expr: Expression): Column = { - SparkShimLoader.getSparkShim.column(expr) + SparkShimLoader.shim.classicApi.column(expr) } def toExpression(spark: SparkSession, col: Column): Expression = { - SparkShimLoader.getSparkShim.convertToExpression(spark, col) + SparkShimLoader.shim.classicApi.expression(spark, col) } protected def resolveExpression( diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/EvalSubqueriesForDeleteTable.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/EvalSubqueriesForDeleteTable.scala index 4cf9284f97f6..66f8a10f3754 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/EvalSubqueriesForDeleteTable.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/EvalSubqueriesForDeleteTable.scala @@ -22,12 +22,13 @@ import org.apache.paimon.spark.catalyst.analysis.expressions.ExpressionHelper import org.apache.paimon.spark.commands.DeleteFromPaimonTableCommand import org.apache.spark.internal.Logging -import org.apache.spark.sql.{execution, SparkSession} +import org.apache.spark.sql.{execution, PaimonSparkSession, SparkSession} import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.expressions.{Expression, In, InSubquery, Literal, ScalarSubquery, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ExecSubqueryExpression, QueryExecution} +import org.apache.spark.sql.paimon.shims.SparkShimLoader import org.apache.spark.sql.types.BooleanType import scala.collection.JavaConverters._ @@ -42,7 +43,7 @@ import scala.collection.JavaConverters._ */ object EvalSubqueriesForDeleteTable extends Rule[LogicalPlan] with ExpressionHelper with Logging { - lazy val spark: SparkSession = SparkSession.active + lazy val spark: SparkSession = PaimonSparkSession.active lazy val resolver: Resolver = spark.sessionState.conf.resolver override def apply(plan: LogicalPlan): LogicalPlan = { @@ -75,7 +76,8 @@ object EvalSubqueriesForDeleteTable extends Rule[LogicalPlan] with ExpressionHel throw new RuntimeException("Correlated InSubquery is not supported") } - val executedPlan = QueryExecution.prepareExecutedPlan(spark, listQuery.plan) + val executedPlan = + SparkShimLoader.shim.classicApi.prepareExecutedPlan(spark, listQuery.plan) val physicalSubquery = execution.InSubqueryExec( expr, execution.SubqueryExec(s"subquery#${listQuery.exprId.id}", executedPlan), @@ -83,7 +85,7 @@ object EvalSubqueriesForDeleteTable extends Rule[LogicalPlan] with ExpressionHel evalPhysicalSubquery(physicalSubquery) physicalSubquery.values() match { - case Some(l) if l.length > 0 => In(expr, l.map(Literal(_, expr.dataType))) + case Some(l) if l.length > 0 => In(expr, l.map(Literal(_, expr.dataType)).toSeq) case _ => Literal(false, BooleanType) } @@ -92,7 +94,7 @@ object EvalSubqueriesForDeleteTable extends Rule[LogicalPlan] with ExpressionHel throw new RuntimeException("Correlated ScalarSubquery is not supported") } - val executedPlan = QueryExecution.prepareExecutedPlan(spark, s.plan) + val executedPlan = SparkShimLoader.shim.classicApi.prepareExecutedPlan(spark, s.plan) val physicalSubquery = execution.ScalarSubquery( execution.SubqueryExec .createForScalarSubquery(s"scalar-subquery#${s.exprId.id}", executedPlan), diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueriesBase.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueriesBase.scala index 95ee8e86b312..e42ac1cc426b 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueriesBase.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueriesBase.scala @@ -282,7 +282,7 @@ trait MergePaimonScalarSubqueriesBase extends Rule[LogicalPlan] with PredicateHe Some(scan2) } else { val mergedRequiredSchema = StructType( - (scan2.requiredSchema.fields.toSet ++ scan1.requiredSchema.fields.toSet).toSeq) + (scan2.requiredSchema.fields.toSet ++ scan1.requiredSchema.fields.toSet).toArray) Some(scan2.copy(requiredSchema = mergedRequiredSchema)) } } else { @@ -334,7 +334,7 @@ trait MergePaimonScalarSubqueriesBase extends Rule[LogicalPlan] with PredicateHe // Only allow aggregates of the same implementation because merging different implementations // could cause performance regression. - private def supportedAggregateMerge(newPlan: Aggregate, cachedPlan: Aggregate) = { + private def supportedAggregateMerge(newPlan: Aggregate, cachedPlan: Aggregate): Boolean = { val aggregateExpressionsSeq = Seq(newPlan, cachedPlan).map { plan => plan.aggregateExpressions.flatMap(_.collect { case a: AggregateExpression => a }) } @@ -343,7 +343,7 @@ trait MergePaimonScalarSubqueriesBase extends Rule[LogicalPlan] with PredicateHe val Seq(newPlanSupportsHashAggregate, cachedPlanSupportsHashAggregate) = aggregateExpressionsSeq.zip(groupByExpressionSeq).map { case (aggregateExpressions, groupByExpressions) => - SparkShimLoader.getSparkShim.supportsHashAggregate( + SparkShimLoader.shim.supportsHashAggregate( aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes), groupByExpressions) } @@ -351,8 +351,11 @@ trait MergePaimonScalarSubqueriesBase extends Rule[LogicalPlan] with PredicateHe newPlanSupportsHashAggregate && cachedPlanSupportsHashAggregate || newPlanSupportsHashAggregate == cachedPlanSupportsHashAggregate && { val Seq(newPlanSupportsObjectHashAggregate, cachedPlanSupportsObjectHashAggregate) = - aggregateExpressionsSeq.map( - aggregateExpressions => Aggregate.supportsObjectHashAggregate(aggregateExpressions)) + aggregateExpressionsSeq.zip(groupByExpressionSeq).map { + case (aggregateExpressions, groupByExpressions: Seq[Expression]) => + SparkShimLoader.shim + .supportsObjectHashAggregate(aggregateExpressions, groupByExpressions) + } newPlanSupportsObjectHashAggregate && cachedPlanSupportsObjectHashAggregate || newPlanSupportsObjectHashAggregate == cachedPlanSupportsObjectHashAggregate } @@ -371,7 +374,7 @@ trait MergePaimonScalarSubqueriesBase extends Rule[LogicalPlan] with PredicateHe val subqueryCTE = header.plan.asInstanceOf[CTERelationDef] GetStructField( createScalarSubquery( - SparkShimLoader.getSparkShim.createCTERelationRef( + SparkShimLoader.shim.createCTERelationRef( subqueryCTE.id, resolved = true, subqueryCTE.output, diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/UpdatePaimonTableCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/UpdatePaimonTableCommand.scala index 74c7e122cda1..4c132aae9f91 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/UpdatePaimonTableCommand.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/UpdatePaimonTableCommand.scala @@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.plans.logical.{Assignment, Filter, Project, SupportsSubquery} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.functions.lit -import org.apache.spark.sql.paimon.shims.SparkShimLoader case class UpdatePaimonTableCommand( relation: DataSourceV2Relation, @@ -132,7 +131,7 @@ case class UpdatePaimonTableCommand( touchedDataSplits: Array[DataSplit]): Seq[CommitMessage] = { val updateColumns = updateExpressions.zip(relation.output).map { case (update, origin) => - SparkShimLoader.getSparkShim.column(update).as(origin.name, origin.metadata) + toColumn(update).as(origin.name, origin.metadata) } val toUpdateScanRelation = createNewRelation(touchedDataSplits, relation) @@ -155,7 +154,7 @@ case class UpdatePaimonTableCommand( } else { If(condition, update, origin) } - SparkShimLoader.getSparkShim.column(updated).as(origin.name, origin.metadata) + toColumn(updated).as(origin.name, origin.metadata) } val data = createDataset(sparkSession, toUpdateScanRelation).select(updateColumns: _*) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkArrayData.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkArrayData.scala index c6539a493cee..790d273d0c93 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkArrayData.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkArrayData.scala @@ -113,6 +113,6 @@ abstract class AbstractSparkArrayData extends SparkArrayData { object SparkArrayData { def create(elementType: PaimonDataType): SparkArrayData = { - SparkShimLoader.getSparkShim.createSparkArrayData(elementType) + SparkShimLoader.shim.createSparkArrayData(elementType) } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkInternalRow.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkInternalRow.scala index f3e607e9d7d2..b0916447c06e 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkInternalRow.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkInternalRow.scala @@ -30,7 +30,7 @@ abstract class SparkInternalRow extends InternalRow { object SparkInternalRow { def create(rowType: RowType): SparkInternalRow = { - SparkShimLoader.getSparkShim.createSparkInternalRow(rowType) + SparkShimLoader.shim.createSparkInternalRow(rowType) } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonStrategy.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonStrategy.scala index f2bfc7846b90..ca4c356fa53d 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonStrategy.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonStrategy.scala @@ -23,19 +23,19 @@ import org.apache.paimon.spark.catalog.SupportView import org.apache.paimon.spark.catalyst.analysis.ResolvedPaimonView import org.apache.paimon.spark.catalyst.plans.logical.{CreateOrReplaceTagCommand, CreatePaimonView, DeleteTagCommand, DropPaimonView, PaimonCallCommand, RenameTagCommand, ResolvedIdentifier, ShowPaimonViews, ShowTagsCommand} -import org.apache.spark.sql.{SparkSession, Strategy} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.ResolvedNamespace import org.apache.spark.sql.catalyst.expressions.{Expression, GenericInternalRow, PredicateHelper} import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, DescribeRelation, LogicalPlan, ShowCreateTable} import org.apache.spark.sql.connector.catalog.{Identifier, PaimonLookupCatalog, TableCatalog} -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy} import org.apache.spark.sql.execution.shim.PaimonCreateTableAsSelectStrategy import scala.collection.JavaConverters._ case class PaimonStrategy(spark: SparkSession) - extends Strategy + extends SparkStrategy with PredicateHelper with PaimonLookupCatalog { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala index 4ecff93ea679..c68c91ca4587 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala @@ -33,16 +33,13 @@ class PaimonSparkSessionExtensions extends (SparkSessionExtensions => Unit) { override def apply(extensions: SparkSessionExtensions): Unit = { // parser extensions - extensions.injectParser { - case (_, parser) => SparkShimLoader.getSparkShim.createSparkParser(parser) - } + extensions.injectParser { case (_, parser) => SparkShimLoader.shim.createSparkParser(parser) } // analyzer extensions extensions.injectResolutionRule(spark => new PaimonAnalysis(spark)) extensions.injectResolutionRule(spark => PaimonProcedureResolver(spark)) extensions.injectResolutionRule(spark => PaimonViewResolver(spark)) - extensions.injectResolutionRule( - spark => SparkShimLoader.getSparkShim.createCustomResolution(spark)) + extensions.injectResolutionRule(spark => SparkShimLoader.shim.createCustomResolution(spark)) extensions.injectResolutionRule(spark => PaimonIncompatibleResolutionRules(spark)) extensions.injectPostHocResolutionRule(spark => ReplacePaimonFunctions(spark)) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/procedure/SparkOrphanFilesClean.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/procedure/SparkOrphanFilesClean.scala index 010a3e4edef8..9e435d8b4444 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/procedure/SparkOrphanFilesClean.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/procedure/SparkOrphanFilesClean.scala @@ -29,7 +29,7 @@ import org.apache.paimon.utils.FileStorePathFactory.BUCKET_PATH_PREFIX import org.apache.paimon.utils.SerializableConsumer import org.apache.spark.internal.Logging -import org.apache.spark.sql.{functions, Dataset, SparkSession} +import org.apache.spark.sql.{functions, Dataset, PaimonSparkSession, SparkSession} import org.apache.spark.sql.catalyst.SQLConfHelper import java.util @@ -199,7 +199,7 @@ object SparkOrphanFilesClean extends SQLConfHelper { olderThanMillis: Long, parallelismOpt: Integer, dryRun: Boolean): CleanOrphanFilesResult = { - val spark = SparkSession.active + val spark = PaimonSparkSession.active val parallelism = if (parallelismOpt == null) { Math.max(spark.sparkContext.defaultParallelism, conf.numShufflePartitions) } else { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/procedure/SparkRemoveUnexistingFiles.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/procedure/SparkRemoveUnexistingFiles.scala index f28a12824e62..5361d4eafa9c 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/procedure/SparkRemoveUnexistingFiles.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/procedure/SparkRemoveUnexistingFiles.scala @@ -27,7 +27,7 @@ import org.apache.paimon.table.FileStoreTable import org.apache.paimon.table.sink.{CommitMessage, CommitMessageImpl, CommitMessageSerializer} import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{PaimonSparkSession, SparkSession} import org.apache.spark.sql.catalyst.SQLConfHelper import java.util @@ -110,7 +110,7 @@ object SparkRemoveUnexistingFiles extends SQLConfHelper { tableName: String, dryRun: Boolean, parallelismOpt: Integer): Array[String] = { - val spark = SparkSession.active + val spark = PaimonSparkSession.active val parallelism = if (parallelismOpt == null) { Math.max(spark.sparkContext.defaultParallelism, conf.numShufflePartitions) } else { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonStatsUtils.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonStatsUtils.scala index 8f24700c2902..5553b6d8ef75 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonStatsUtils.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonStatsUtils.scala @@ -21,6 +21,7 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan} import org.apache.spark.sql.execution.command.CommandUtils +import org.apache.spark.sql.paimon.shims.SparkShimLoader import org.apache.spark.sql.types._ /** @@ -34,7 +35,7 @@ object PaimonStatsUtils { sparkSession: SparkSession, relation: LogicalPlan, columns: Seq[Attribute]): (Long, Map[Attribute, ColumnStat]) = { - CommandUtils.computeColumnStats(sparkSession, relation, columns) + SparkShimLoader.shim.classicApi.computeColumnStats(sparkSession, relation, columns) } /** [[IntegralType]] is private in spark, therefore we need add it here. */ diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonUtils.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonUtils.scala index 9023bfa64666..885910dbc2cc 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonUtils.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonUtils.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.connector.expressions.filter.Predicate import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy.translateFilterV2WithMapping import org.apache.spark.sql.internal.connector.PredicateUtils +import org.apache.spark.sql.paimon.shims.SparkShimLoader import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.util.PartitioningUtils @@ -54,11 +55,11 @@ object PaimonUtils { * [[org.apache.spark.sql.execution.streaming.Sink.addBatch]]. */ def createNewDataFrame(data: DataFrame): DataFrame = { - data.sqlContext.internalCreateDataFrame(data.queryExecution.toRdd, data.schema) + SparkShimLoader.shim.classicApi.createDataset(data) } def createDataset(sparkSession: SparkSession, logicalPlan: LogicalPlan): Dataset[Row] = { - Dataset.ofRows(sparkSession, logicalPlan) + SparkShimLoader.shim.classicApi.createDataset(sparkSession, logicalPlan) } def normalizeExprs(exprs: Seq[Expression], attributes: Seq[Attribute]): Seq[Expression] = { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/AbstractPaimonSparkSqlExtensionsParser.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/AbstractPaimonSparkSqlExtensionsParser.scala index 557b0735c74d..73e1ea3ec808 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/AbstractPaimonSparkSqlExtensionsParser.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/AbstractPaimonSparkSqlExtensionsParser.scala @@ -25,7 +25,7 @@ import org.antlr.v4.runtime.atn.PredictionMode import org.antlr.v4.runtime.misc.{Interval, ParseCancellationException} import org.antlr.v4.runtime.tree.TerminalNodeImpl import org.apache.spark.internal.Logging -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.{AnalysisException, PaimonSparkSession, SparkSession} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface} @@ -65,7 +65,7 @@ abstract class AbstractPaimonSparkSqlExtensionsParser(val delegate: ParserInterf parse(sqlTextAfterSubstitution)(parser => astBuilder.visit(parser.singleStatement())) .asInstanceOf[LogicalPlan] } else { - RewritePaimonViewCommands(SparkSession.active).apply(delegate.parsePlan(sqlText)) + RewritePaimonViewCommands(PaimonSparkSession.active).apply(delegate.parsePlan(sqlText)) } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala index 5db6894ba093..f330fed3f38d 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.connector.catalog import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkConf -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{PaimonSparkSession, SparkSession} import org.apache.spark.sql.catalyst.catalog.ExternalCatalog import org.apache.spark.sql.connector.catalog.CatalogV2Util import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION @@ -30,7 +30,7 @@ object PaimonCatalogUtils { def buildExternalCatalog(conf: SparkConf, hadoopConf: Configuration): ExternalCatalog = { val externalCatalogClassName = - if (SparkSession.active.conf.get(CATALOG_IMPLEMENTATION.key).equals("hive")) { + if (PaimonSparkSession.active.conf.get(CATALOG_IMPLEMENTATION.key).equals("hive")) { "org.apache.spark.sql.hive.HiveExternalCatalog" } else { "org.apache.spark.sql.catalyst.catalog.InMemoryCatalog" diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala index 8279a6de31dd..fd6627c09578 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala @@ -22,17 +22,17 @@ import org.apache.paimon.CoreOptions import org.apache.paimon.spark.SparkCatalog import org.apache.paimon.spark.catalog.FormatTableCatalog -import org.apache.spark.sql.{SparkSession, Strategy} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.ResolvedIdentifier import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, LogicalPlan, TableSpec} import org.apache.spark.sql.connector.catalog.StagingTableCatalog -import org.apache.spark.sql.execution.{PaimonStrategyHelper, SparkPlan} +import org.apache.spark.sql.execution.{PaimonStrategyHelper, SparkPlan, SparkStrategy} import org.apache.spark.sql.execution.datasources.v2.CreateTableAsSelectExec import scala.collection.JavaConverters._ case class PaimonCreateTableAsSelectStrategy(spark: SparkSession) - extends Strategy + extends SparkStrategy with PaimonStrategyHelper { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/PaimonSparkSession.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/PaimonSparkSession.scala new file mode 100644 index 000000000000..674a9196f71e --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/PaimonSparkSession.scala @@ -0,0 +1,25 @@ +/* + * 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 + +object PaimonSparkSession { + + def active: SparkSession = SparkSession.active + +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/ClassicApi.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/ClassicApi.scala new file mode 100644 index 000000000000..21381cca29f3 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/ClassicApi.scala @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.paimon.shims + +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan} +import org.apache.spark.sql.execution.SparkPlan + +trait ClassicApi { + + def column(expression: Expression): Column + + def expression(spark: SparkSession, column: Column): Expression + + def createDataset(data: DataFrame): DataFrame + + def createDataset(sparkSession: SparkSession, logicalPlan: LogicalPlan): Dataset[Row] + + def prepareExecutedPlan(spark: SparkSession, logicalPlan: LogicalPlan): SparkPlan + + def recacheByPlan(spark: SparkSession, plan: LogicalPlan): Unit + + def computeColumnStats( + sparkSession: SparkSession, + relation: LogicalPlan, + columns: Seq[Attribute]): (Long, Map[Attribute, ColumnStat]) + +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShim.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShim.scala index 6b771a333961..ef764bc0d1bb 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShim.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShim.scala @@ -22,9 +22,10 @@ import org.apache.paimon.data.variant.Variant import org.apache.paimon.spark.data.{SparkArrayData, SparkInternalRow} import org.apache.paimon.types.{DataType, RowType} -import org.apache.spark.sql.{Column, SparkSession} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{CTERelationRef, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule @@ -42,6 +43,8 @@ import java.util.{Map => JMap} */ trait SparkShim { + def classicApi: ClassicApi + def createSparkParser(delegate: ParserInterface): ParserInterface def createCustomResolution(spark: SparkSession): Rule[LogicalPlan] @@ -50,10 +53,6 @@ trait SparkShim { def createSparkArrayData(elementType: DataType): SparkArrayData - def supportsHashAggregate( - aggregateBufferAttributes: Seq[Attribute], - groupingExpression: Seq[Expression]): Boolean - def createTable( tableCatalog: TableCatalog, ident: Identifier, @@ -67,9 +66,13 @@ trait SparkShim { output: Seq[Attribute], isStreaming: Boolean): CTERelationRef - def column(expr: Expression): Column + def supportsHashAggregate( + aggregateBufferAttributes: Seq[Attribute], + groupingExpression: Seq[Expression]): Boolean - def convertToExpression(spark: SparkSession, column: Column): Expression + def supportsObjectHashAggregate( + aggregateExpressions: Seq[AggregateExpression], + groupByExpressions: Seq[Expression]): Boolean // for variant def toPaimonVariant(o: Object): Variant diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShimLoader.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShimLoader.scala index 920896547a1e..d6b5850b9d09 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShimLoader.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShimLoader.scala @@ -27,7 +27,7 @@ object SparkShimLoader { private lazy val sparkShim: SparkShim = loadSparkShim() - def getSparkShim: SparkShim = { + def shim: SparkShim = { sparkShim } diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java index 488913f14add..c7c07b1aef03 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java @@ -31,6 +31,7 @@ import org.junit.jupiter.api.io.TempDir; import java.io.FileNotFoundException; +import java.io.IOException; import java.util.stream.Collectors; import static org.assertj.core.api.Assertions.assertThat; @@ -56,8 +57,9 @@ public static void closeMetastore() throws Exception { } @Test - public void testCreateFormatTable() { - try (SparkSession spark = createSessionBuilder().getOrCreate()) { + public void testCreateFormatTable() throws IOException { + SparkSession spark = createSessionBuilder().getOrCreate(); + { spark.sql("CREATE DATABASE IF NOT EXISTS my_db1"); spark.sql("USE spark_catalog.my_db1"); @@ -107,10 +109,11 @@ public void testCreateFormatTable() { .collect(Collectors.toList())) .containsExactlyInAnyOrder("[1,1,1]", "[2,2,2]"); } + spark.stop(); } @Test - public void testSpecifyHiveConfDirInGenericCatalog() { + public void testSpecifyHiveConfDirInGenericCatalog() throws IOException { try (SparkSession spark = createSessionBuilder() .config("spark.sql.catalog.spark_catalog.hive-conf-dir", "nonExistentPath") @@ -126,7 +129,7 @@ public void testSpecifyHiveConfDirInGenericCatalog() { } @Test - public void testCreateExternalTable() { + public void testCreateExternalTable() throws IOException { try (SparkSession spark = createSessionBuilder().getOrCreate()) { String warehousePath = spark.sparkContext().conf().get("spark.sql.warehouse.dir"); spark.sql("CREATE DATABASE IF NOT EXISTS test_db"); diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java index 0ae0e9130627..b38995da14a3 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java @@ -28,6 +28,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import java.io.IOException; import java.util.List; import static org.assertj.core.api.Assertions.assertThat; @@ -82,7 +83,7 @@ public void testPaimonTable() { } @Test - public void testSparkSessionReload() { + public void testSparkSessionReload() throws IOException { spark.sql("CREATE DATABASE my_db"); spark.close(); diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java index 604e2ea27993..9e13dd7be525 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java @@ -28,6 +28,7 @@ import org.junit.jupiter.api.io.TempDir; import java.io.FileNotFoundException; +import java.io.IOException; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -48,7 +49,7 @@ public static void closeMetastore() throws Exception { } @Test - public void testBuildWithHive(@TempDir java.nio.file.Path tempDir) { + public void testBuildWithHive(@TempDir java.nio.file.Path tempDir) throws IOException { // firstly, we use hive metastore to create table, and check the result. Path warehousePath = new Path("file:" + tempDir.toString()); SparkSession spark = @@ -59,12 +60,15 @@ public void testBuildWithHive(@TempDir java.nio.file.Path tempDir) { .config( "spark.sql.catalog.spark_catalog", SparkGenericCatalog.class.getName()) + .config("spark.sql.catalog.paimon.warehouse", warehousePath.toString()) + .config("spark.sql.catalog.paimon", SparkCatalog.class.getName()) .config( "spark.sql.extensions", "org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions") .master("local[2]") .getOrCreate(); + spark.sql("USE spark_catalog"); spark.sql("CREATE DATABASE my_db"); spark.sql("USE my_db"); spark.sql( @@ -79,33 +83,26 @@ public void testBuildWithHive(@TempDir java.nio.file.Path tempDir) { .map(s -> s.get(1)) .map(Object::toString)) .containsExactlyInAnyOrder("t1"); - spark.close(); - // secondly, we close catalog with hive metastore, and start a filesystem metastore to check - // the result. - SparkSession spark2 = - SparkSession.builder() - .config("spark.sql.catalog.paimon.warehouse", warehousePath.toString()) - .config("spark.sql.catalogImplementation", "in-memory") - .config("spark.sql.catalog.paimon", SparkCatalog.class.getName()) - .config( - "spark.sql.extensions", - "org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions") - .master("local[2]") - .getOrCreate(); - spark2.sql("USE paimon"); - spark2.sql("USE my_db"); - assertThat(spark2.sql("SHOW NAMESPACES").collectAsList().stream().map(Object::toString)) + // secondly, use filesystem metastore to check the result. + spark.sql("USE paimon"); + spark.sql("USE my_db"); + assertThat(spark.sql("SHOW NAMESPACES").collectAsList().stream().map(Object::toString)) .containsExactlyInAnyOrder("[default]", "[my_db]"); assertThat( - spark2.sql("SHOW TABLES").collectAsList().stream() + spark.sql("SHOW TABLES").collectAsList().stream() .map(s -> s.get(1)) .map(Object::toString)) .containsExactlyInAnyOrder("t1"); + + spark.sql("USE spark_catalog"); + spark.sql("DROP TABLE my_db.t1"); + spark.sql("DROP DATABASE my_db"); + spark.close(); } @Test - public void testHiveCatalogOptions(@TempDir java.nio.file.Path tempDir) { + public void testHiveCatalogOptions(@TempDir java.nio.file.Path tempDir) throws IOException { Path warehousePath = new Path("file:" + tempDir.toString()); SparkSession spark = SparkSession.builder() diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/catalog/functions/BucketFunctionTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/catalog/functions/BucketFunctionTest.java index cf0265406386..b8fbcdae4248 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/catalog/functions/BucketFunctionTest.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/catalog/functions/BucketFunctionTest.java @@ -56,6 +56,7 @@ import org.apache.spark.sql.SparkSession; import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -190,6 +191,11 @@ public void after() { spark.sql(String.format("DROP TABLE IF EXISTS %s", TABLE_NAME)); } + @AfterAll + public static void tearDown() { + spark.stop(); + } + public static void setupTable(String... bucketColumns) { String commitUser = UUID.randomUUID().toString(); try { diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/extensions/CallStatementParserTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/extensions/CallStatementParserTest.java index e4e571e96bc9..9f77a93bf04e 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/extensions/CallStatementParserTest.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/extensions/CallStatementParserTest.java @@ -22,6 +22,7 @@ import org.apache.paimon.spark.catalyst.plans.logical.PaimonCallStatement; import org.apache.paimon.spark.catalyst.plans.logical.PaimonNamedArgument; import org.apache.paimon.spark.catalyst.plans.logical.PaimonPositionalArgument; +import org.apache.paimon.spark.sql.SparkVersionSupport$; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.expressions.Literal$; @@ -81,16 +82,20 @@ public void stopSparkSession() { @Test public void testDelegateUnsupportedProcedure() { - assertThatThrownBy(() -> parser.parsePlan("CALL cat.d.t()")) - .isInstanceOf(ParseException.class) - .satisfies( - exception -> { - ParseException parseException = (ParseException) exception; - assertThat(parseException.getErrorClass()) - .isEqualTo("PARSE_SYNTAX_ERROR"); - assertThat(parseException.getMessageParameters().get("error")) - .isEqualTo("'CALL'"); - }); + if (!SparkVersionSupport$.MODULE$.gteqSpark4_0()) { + // TODO: adapt spark 4.0 to make Paimon parser only apply own supported procedures. + + assertThatThrownBy(() -> parser.parsePlan("CALL cat.d.t()")) + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + assertThat(parseException.getErrorClass()) + .isEqualTo("PARSE_SYNTAX_ERROR"); + assertThat(parseException.getMessageParameters().get("error")) + .isEqualTo("'CALL'"); + }); + } } @Test diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTestBase.scala index a5f9f3ffa01b..df98026022b8 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTestBase.scala @@ -20,6 +20,7 @@ package org.apache.paimon.spark.procedure import org.apache.paimon.spark.PaimonSparkTestBase +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.parser.extensions.PaimonParseException import org.assertj.core.api.Assertions.assertThatThrownBy @@ -31,8 +32,13 @@ abstract class ProcedureTestBase extends PaimonSparkTestBase { |CREATE TABLE T (id INT, name STRING, dt STRING) |""".stripMargin) - assertThatThrownBy(() => spark.sql("CALL sys.unknown_procedure(table => 'test.T')")) - .isInstanceOf(classOf[ParseException]) + if (gteqSpark4_0) { + assertThatThrownBy(() => spark.sql("CALL sys.unknown_procedure(table => 'test.T')")) + .isInstanceOf(classOf[AnalysisException]) + } else { + assertThatThrownBy(() => spark.sql("CALL sys.unknown_procedure(table => 'test.T')")) + .isInstanceOf(classOf[ParseException]) + } } test(s"test parse exception") { diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala index 36315f022837..06c2eaf0497d 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala @@ -285,41 +285,45 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { } test("Paimon DDL with hive catalog: set default database") { - var reusedSpark = spark + if (!gteqSpark4_0) { + // TODO: This is skipped in Spark 4.0, because it would fail in afterAll method, not because the default database is not supported. - Seq("paimon", sparkCatalogName, paimonHiveCatalogName).foreach { - catalogName => - { - val dbName = s"${catalogName}_default_db" - val tblName = s"${dbName}_tbl" - - reusedSpark.sql(s"use $catalogName") - reusedSpark.sql(s"create database $dbName") - reusedSpark.sql(s"use $dbName") - reusedSpark.sql(s"create table $tblName (id int, name string, dt string) using paimon") - reusedSpark.stop() - - reusedSpark = SparkSession - .builder() - .master("local[2]") - .config(sparkConf) - .config("spark.sql.defaultCatalog", catalogName) - .config(s"spark.sql.catalog.$catalogName.defaultDatabase", dbName) - .getOrCreate() - - if (catalogName.equals(sparkCatalogName) && !gteqSpark3_4) { - checkAnswer(reusedSpark.sql("show tables").select("tableName"), Nil) + var reusedSpark = spark + + Seq("paimon", sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + { + val dbName = s"${catalogName}_default_db" + val tblName = s"${dbName}_tbl" + + reusedSpark.sql(s"use $catalogName") + reusedSpark.sql(s"create database $dbName") reusedSpark.sql(s"use $dbName") + reusedSpark.sql(s"create table $tblName (id int, name string, dt string) using paimon") + reusedSpark.stop() + + reusedSpark = SparkSession + .builder() + .master("local[2]") + .config(sparkConf) + .config("spark.sql.defaultCatalog", catalogName) + .config(s"spark.sql.catalog.$catalogName.defaultDatabase", dbName) + .getOrCreate() + + if (catalogName.equals(sparkCatalogName) && !gteqSpark3_4) { + checkAnswer(reusedSpark.sql("show tables").select("tableName"), Nil) + reusedSpark.sql(s"use $dbName") + } + checkAnswer(reusedSpark.sql("show tables").select("tableName"), Row(tblName) :: Nil) + + reusedSpark.sql(s"drop table $tblName") } - checkAnswer(reusedSpark.sql("show tables").select("tableName"), Row(tblName) :: Nil) + } - reusedSpark.sql(s"drop table $tblName") - } + // Since we created a new sparkContext, we need to stop it and reset the default sparkContext + reusedSpark.stop() + reset() } - - // Since we created a new sparkContext, we need to stop it and reset the default sparkContext - reusedSpark.stop() - reset() } test("Paimon DDL with hive catalog: drop database cascade which contains paimon table") { diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala index 87f4c9448619..e5f1d0e1319a 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, LogicalPlan, import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.functions._ import org.apache.spark.sql.paimon.Utils +import org.apache.spark.sql.paimon.shims.SparkShimLoader import org.junit.jupiter.api.Assertions import scala.collection.immutable diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala index 647b4cfdcab7..9dadb26e2500 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala @@ -21,7 +21,8 @@ package org.apache.paimon.spark.sql import org.apache.spark.SPARK_VERSION trait SparkVersionSupport { - lazy val sparkVersion: String = SPARK_VERSION + + val sparkVersion: String = SPARK_VERSION lazy val gteqSpark3_3: Boolean = sparkVersion >= "3.3" @@ -31,3 +32,5 @@ trait SparkVersionSupport { lazy val gteqSpark4_0: Boolean = sparkVersion >= "4.0" } + +object SparkVersionSupport extends SparkVersionSupport {} diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/spark/sql/paimon/Utils.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/spark/sql/paimon/Utils.scala index 03f1c7706efb..61a479b9f07d 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/spark/sql/paimon/Utils.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/spark/sql/paimon/Utils.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.paimon import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.paimon.shims.SparkShimLoader import org.apache.spark.util.{Utils => SparkUtils} import java.io.File @@ -36,7 +37,7 @@ object Utils { } def createDataFrame(sparkSession: SparkSession, plan: LogicalPlan): DataFrame = { - Dataset.ofRows(sparkSession, plan) + SparkShimLoader.shim.classicApi.createDataset(sparkSession, plan) } } diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Classic3Api.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Classic3Api.scala new file mode 100644 index 000000000000..b0782c59d648 --- /dev/null +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Classic3Api.scala @@ -0,0 +1,55 @@ +/* + * 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.paimon.shims + +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan} +import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} +import org.apache.spark.sql.execution.command.CommandUtils +class Classic3Api extends ClassicApi { + + override def column(expression: Expression): Column = new Column(expression) + + override def expression(spark: SparkSession, column: Column): Expression = column.expr + + override def createDataset(data: DataFrame): DataFrame = { + data.sqlContext + .internalCreateDataFrame(data.queryExecution.toRdd, data.schema) + } + + override def createDataset(spark: SparkSession, logicalPlan: LogicalPlan): Dataset[Row] = { + Dataset.ofRows(spark, logicalPlan) + } + + override def recacheByPlan(spark: SparkSession, plan: LogicalPlan): Unit = { + spark.sharedState.cacheManager.recacheByPlan(spark, plan) + } + + override def prepareExecutedPlan(spark: SparkSession, logicalPlan: LogicalPlan): SparkPlan = { + QueryExecution.prepareExecutedPlan(spark, logicalPlan) + } + + override def computeColumnStats( + spark: SparkSession, + relation: LogicalPlan, + columns: Seq[Attribute]): (Long, Map[Attribute, ColumnStat]) = + CommandUtils.computeColumnStats(spark, relation, columns) + +} diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala index b173a3ff5c6a..18ba22674d5a 100644 --- a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala @@ -27,18 +27,22 @@ import org.apache.paimon.types.{DataType, RowType} import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CTERelationRef, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.ArrayData import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.execution.command.CommandUtils import org.apache.spark.sql.types.StructType import java.util.{Map => JMap} class Spark3Shim extends SparkShim { + override def classicApi: ClassicApi = new Classic3Api + override def createSparkParser(delegate: ParserInterface): ParserInterface = { new PaimonSpark3SqlExtensionsParser(delegate) } @@ -55,12 +59,6 @@ class Spark3Shim extends SparkShim { new Spark3ArrayData(elementType) } - override def supportsHashAggregate( - aggregateBufferAttributes: Seq[Attribute], - groupingExpression: Seq[Expression]): Boolean = { - Aggregate.supportsHashAggregate(aggregateBufferAttributes) - } - override def createTable( tableCatalog: TableCatalog, ident: Identifier, @@ -77,9 +75,15 @@ class Spark3Shim extends SparkShim { isStreaming: Boolean): CTERelationRef = MinorVersionShim.createCTERelationRef(cteId, resolved, output, isStreaming) - override def column(expr: Expression): Column = new Column(expr) + override def supportsHashAggregate( + aggregateBufferAttributes: Seq[Attribute], + groupingExpression: Seq[Expression]): Boolean = + Aggregate.supportsHashAggregate(aggregateBufferAttributes) - override def convertToExpression(spark: SparkSession, column: Column): Expression = column.expr + override def supportsObjectHashAggregate( + aggregateExpressions: Seq[AggregateExpression], + groupByExpressions: Seq[Expression]): Boolean = + Aggregate.supportsObjectHashAggregate(aggregateExpressions) override def toPaimonVariant(o: Object): Variant = throw new UnsupportedOperationException() @@ -94,4 +98,5 @@ class Spark3Shim extends SparkShim { override def toPaimonVariant(array: ArrayData, pos: Int): Variant = throw new UnsupportedOperationException() + } diff --git a/paimon-spark/paimon-spark4-common/pom.xml b/paimon-spark/paimon-spark4-common/pom.xml index e839cd45dfe3..e8b7356eca3d 100644 --- a/paimon-spark/paimon-spark4-common/pom.xml +++ b/paimon-spark/paimon-spark4-common/pom.xml @@ -49,6 +49,10 @@ under the License. spark-sql-api_2.13 ${spark.version} + + org.apache.spark + spark-connect-shims_${scala.binary.version} + log4j log4j diff --git a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark4SqlExtensionsParser.scala b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark4SqlExtensionsParser.scala index ef1f5763d27b..9bd395f3336d 100644 --- a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark4SqlExtensionsParser.scala +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark4SqlExtensionsParser.scala @@ -18,11 +18,12 @@ package org.apache.paimon.spark.catalyst.parser.extensions -import org.apache.spark.sql.catalyst.parser.{CompoundBody, ParserInterface} +import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.parser.extensions.AbstractPaimonSparkSqlExtensionsParser +import org.apache.spark.sql.types.StructType class PaimonSpark4SqlExtensionsParser(override val delegate: ParserInterface) extends AbstractPaimonSparkSqlExtensionsParser(delegate) { - def parseScript(sqlScriptText: String): CompoundBody = delegate.parseScript(sqlScriptText) + override def parseRoutineParam(sqlText: String): StructType = delegate.parseRoutineParam(sqlText) } diff --git a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Classic4Api.scala b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Classic4Api.scala new file mode 100644 index 000000000000..8dff78f0bc6f --- /dev/null +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Classic4Api.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.paimon.shims + +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan} +import org.apache.spark.sql.classic.{ClassicConversions, Dataset => ClassicDataset, ExpressionUtils} +import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} +import org.apache.spark.sql.execution.command.CommandUtils + +/** + * This class is used to implement the conversion from sql-api to classic one. Make sure this is the + * only class that implements [[org.apache.spark.sql.classic.ClassicConversions]] in Paimon-Spark. + */ +class Classic4Api extends ClassicApi with ClassicConversions { + + override def column(expression: Expression): Column = ExpressionUtils.column(expression) + + override def expression(spark: SparkSession, column: Column): Expression = { + spark.expression(column) + } + + override def createDataset(data: DataFrame): DataFrame = { + data.sqlContext + .internalCreateDataFrame(data.queryExecution.toRdd, data.schema) + } + + override def createDataset(spark: SparkSession, logicalPlan: LogicalPlan): Dataset[Row] = { + ClassicDataset.ofRows(spark, logicalPlan) + } + + override def recacheByPlan(spark: SparkSession, plan: LogicalPlan): Unit = { + spark.sharedState.cacheManager.recacheByPlan(spark, plan) + } + + override def prepareExecutedPlan(spark: SparkSession, logicalPlan: LogicalPlan): SparkPlan = { + QueryExecution.prepareExecutedPlan(spark, logicalPlan) + } + + override def computeColumnStats( + spark: SparkSession, + relation: LogicalPlan, + columns: Seq[Attribute]): (Long, Map[Attribute, ColumnStat]) = { + CommandUtils.computeColumnStats(spark, relation, columns.toSeq) + } +} diff --git a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala index e8e47831388d..cb535ef86cc2 100644 --- a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala @@ -24,16 +24,16 @@ import org.apache.paimon.spark.catalyst.parser.extensions.PaimonSpark4SqlExtensi import org.apache.paimon.spark.data.{Spark4ArrayData, Spark4InternalRow, SparkArrayData, SparkInternalRow} import org.apache.paimon.types.{DataType, RowType} -import org.apache.spark.sql.{Column, SparkSession} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CTERelationRef, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.ArrayData import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, Table, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.internal.ExpressionUtils import org.apache.spark.sql.types.{DataTypes, StructType, VariantType} import org.apache.spark.unsafe.types.VariantVal @@ -41,6 +41,8 @@ import java.util.{Map => JMap} class Spark4Shim extends SparkShim { + override def classicApi: ClassicApi = new Classic4Api + override def createSparkParser(delegate: ParserInterface): ParserInterface = { new PaimonSpark4SqlExtensionsParser(delegate) } @@ -57,13 +59,7 @@ class Spark4Shim extends SparkShim { new Spark4ArrayData(elementType) } - def supportsHashAggregate( - aggregateBufferAttributes: Seq[Attribute], - groupingExpression: Seq[Expression]): Boolean = { - Aggregate.supportsHashAggregate(aggregateBufferAttributes, groupingExpression) - } - - def createTable( + override def createTable( tableCatalog: TableCatalog, ident: Identifier, schema: StructType, @@ -81,10 +77,16 @@ class Spark4Shim extends SparkShim { CTERelationRef(cteId, resolved, output.toSeq, isStreaming) } - def column(expr: Expression): Column = ExpressionUtils.column(expr) + override def supportsHashAggregate( + aggregateBufferAttributes: Seq[Attribute], + groupingExpression: Seq[Expression]): Boolean = { + Aggregate.supportsHashAggregate(aggregateBufferAttributes.toSeq, groupingExpression.toSeq) + } - def convertToExpression(spark: SparkSession, column: Column): Expression = - spark.expression(column) + override def supportsObjectHashAggregate( + aggregateExpressions: Seq[AggregateExpression], + groupByExpressions: Seq[Expression]): Boolean = + Aggregate.supportsObjectHashAggregate(aggregateExpressions.toSeq, groupByExpressions.toSeq) override def toPaimonVariant(o: Object): Variant = { val v = o.asInstanceOf[VariantVal] diff --git a/pom.xml b/pom.xml index bfdd189eff71..ef045dbcb96b 100644 --- a/pom.xml +++ b/pom.xml @@ -400,10 +400,10 @@ under the License. 4.13.1 2.13 ${scala213.version} - 4.0.0-preview2 + 4.0.0 paimon-spark4-common 4.0 - 4.0.0-preview2 + 4.0.0