This is an automated email from the ASF dual-hosted git repository.
lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-paimon.git
The following commit(s) were added to refs/heads/master by this push:
new 63f2b50d1 [core] Add validation that append table of -1 bucket do not
support full-compaction (#2158)
63f2b50d1 is described below
commit 63f2b50d1c9ea4724e85c20acf38f15998d81677
Author: wgcn <[email protected]>
AuthorDate: Mon Oct 23 10:01:38 2023 +0800
[core] Add validation that append table of -1 bucket do not support
full-compaction (#2158)
---
.../main/java/org/apache/paimon/schema/SchemaValidation.java | 8 ++++++++
.../java/org/apache/paimon/flink/AppendOnlyTableITCase.java | 12 ++++++++++++
2 files changed, 20 insertions(+)
diff --git
a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java
b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java
index ae050f6e7..440a8dc97 100644
--- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java
+++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java
@@ -44,6 +44,7 @@ import java.util.stream.Collectors;
import static org.apache.paimon.CoreOptions.BUCKET_KEY;
import static org.apache.paimon.CoreOptions.CHANGELOG_PRODUCER;
+import static org.apache.paimon.CoreOptions.FULL_COMPACTION_DELTA_COMMITS;
import static org.apache.paimon.CoreOptions.INCREMENTAL_BETWEEN;
import static org.apache.paimon.CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP;
import static org.apache.paimon.CoreOptions.SCAN_MODE;
@@ -124,6 +125,13 @@ public class SchemaValidation {
"Cannot define 'bucket-key' in unaware or dynamic bucket
mode.");
}
+ if (options.bucket() == -1
+ && schema.primaryKeys().isEmpty()
+ && options.toMap().get(FULL_COMPACTION_DELTA_COMMITS.key()) !=
null) {
+ throw new RuntimeException(
+ "AppendOnlyTable of unware or dynamic bucket does not
support 'full-compaction.delta-commits'");
+ }
+
if (schema.primaryKeys().isEmpty() &&
options.streamingReadOverwrite()) {
throw new RuntimeException(
"Doesn't support streaming read the changes from overwrite
when the primary keys are not defined.");
diff --git
a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java
b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java
index f2b0a6540..9e1408658 100644
---
a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java
+++
b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java
@@ -49,6 +49,18 @@ public class AppendOnlyTableITCase extends CatalogITCaseBase
{
"Cannot define 'bucket-key' in unaware or dynamic
bucket mode.");
}
+ @Test
+ public void testCreateUnawareBucketTableWithFullCompaction() {
+ assertThatThrownBy(
+ () ->
+ batchSql(
+ "CREATE TABLE pk_table (id INT, data
STRING) "
+ + "WITH ('bucket' =
'-1','full-compaction.delta-commits'='10')"))
+ .hasRootCauseInstanceOf(RuntimeException.class)
+ .hasRootCauseMessage(
+ "AppendOnlyTable of unware or dynamic bucket does not
support 'full-compaction.delta-commits'");
+ }
+
@Test
public void testReadEmpty() {
assertThat(batchSql("SELECT * FROM append_table")).isEmpty();