From c1913468b33c99ecf09bc6a1f580f699f49a73e2 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Mon, 29 Dec 2025 11:54:20 +0800 Subject: [PATCH 1/3] [core][flink] Remove legacy codes for 0.2,0.3 and logsystem --- docs/content/maintenance/rescale-bucket.md | 4 - .../java/org/apache/paimon/CoreOptions.java | 156 --- .../main/java/org/apache/paimon/Snapshot.java | 63 +- .../java/org/apache/paimon/Changelog.java | 9 +- .../paimon/manifest/ManifestCommittable.java | 31 +- .../ManifestCommittableSerializer.java | 30 +- .../apache/paimon/manifest/ManifestEntry.java | 10 + .../paimon/operation/FileStoreCommitImpl.java | 45 +- .../paimon/operation/FileStoreScan.java | 13 - .../paimon/operation/ManifestsReader.java | 4 - .../operation/commit/CommitScanner.java | 4 - .../table/format/TwoPhaseCommitMessage.java | 3 +- .../table/sink/KeyAndBucketExtractor.java | 7 +- .../paimon/table/sink/RowKeyExtractor.java | 18 +- .../paimon/table/sink/TableWriteImpl.java | 9 - .../main/java/org/apache/paimon/tag/Tag.java | 15 +- .../java/org/apache/paimon/TestFileStore.java | 17 +- ...ommittableSerializerCompatibilityTest.java | 174 ++-- .../ManifestCommittableSerializerTest.java | 6 - .../paimon/operation/FileDeletionTest.java | 1 - .../paimon/operation/FileStoreCommitTest.java | 77 +- .../paimon/table/IncrementalTableTest.java | 22 +- .../apache/paimon/tag/TagAutoManagerTest.java | 10 +- .../java/org/apache/paimon/tag/TagTest.java | 15 +- .../paimon/utils/SnapshotManagerTest.java | 25 +- .../compatibility/manifest-committable-v11-v5 | Bin 0 -> 4130 bytes .../apache/paimon/tests/LogStoreE2eTest.java | 135 --- .../paimon/flink/sink/FlinkTableSink.java | 10 +- .../paimon/flink/sink/FlinkTableSink.java | 10 +- .../kafka/KafkaLogDeserializationSchema.java | 169 ---- .../paimon/flink/kafka/KafkaLogOptions.java | 38 - .../kafka/KafkaLogSerializationSchema.java | 87 -- .../flink/kafka/KafkaLogSinkProvider.java | 88 -- .../flink/kafka/KafkaLogSourceProvider.java | 153 --- .../flink/kafka/KafkaLogStoreFactory.java | 171 ---- .../flink/kafka/KafkaLogStoreRegister.java | 166 ---- .../paimon/flink/kafka/KafkaSinkFunction.java | 95 -- .../paimon/flink/sink/FlinkTableSink.java | 10 +- .../cdc/CdcDynamicBucketWriteOperator.java | 5 - .../cdc/CdcRecordKeyAndBucketExtractor.java | 5 - .../sink/cdc/CdcRecordStoreWriteOperator.java | 5 - .../org.apache.paimon.factories.Factory | 2 - ...ltiPartitionedTableWithKafkaLogITCase.java | 936 ------------------ .../kafka/KafkaLogSerializationTest.java | 221 ----- .../flink/kafka/KafkaLogStoreFactoryTest.java | 149 --- .../kafka/KafkaLogStoreRegisterITCase.java | 192 ---- .../paimon/flink/kafka/KafkaLogTestUtils.java | 275 ----- .../flink/kafka/KafkaTableTestBase.java | 306 ------ .../paimon/flink/kafka/LogSystemITCase.java | 441 --------- ...amingReadWriteTableWithKafkaLogITCase.java | 831 ---------------- .../flink/kafka/StreamingWarehouseITCase.java | 245 ----- .../source/LogHybridSourceFactoryTest.java | 44 - .../flink/AbstractFlinkTableFactory.java | 72 +- .../org/apache/paimon/flink/FlinkCatalog.java | 44 +- .../paimon/flink/FlinkCatalogOptions.java | 21 - .../paimon/flink/FlinkConnectorOptions.java | 46 - .../paimon/flink/action/CompactAction.java | 6 +- .../action/RemoveUnexistingFilesAction.java | 14 +- .../RemoveClusterBeforeFilesOperator.java | 8 +- ...IncrementalClusterCommittableOperator.java | 10 +- ...ndPreCommitCompactCoordinatorOperator.java | 15 +- .../AppendPreCommitCompactWorkerOperator.java | 4 +- .../flink/compact/AppendTableCompactor.java | 2 +- .../compact/IncrementalClusterCompact.java | 4 +- .../ChangelogCompactCoordinateOperator.java | 13 +- .../ChangelogCompactSortOperator.java | 13 +- .../changelog/ChangelogCompactTask.java | 2 +- .../flink/factories/FlinkFactoryUtil.java | 498 ---------- .../paimon/flink/log/LogSinkProvider.java | 30 - .../paimon/flink/log/LogSourceProvider.java | 50 - .../paimon/flink/log/LogStoreRegister.java | 88 -- .../flink/log/LogStoreTableFactory.java | 167 ---- .../paimon/flink/log/LogWriteCallback.java | 51 - .../PostponeBucketCommittableRewriter.java | 1 - .../RemovePostponeBucketFilesOperator.java | 4 +- ...ritePostponeBucketCommittableOperator.java | 6 +- ...nlyMultiTableCompactionWorkerOperator.java | 3 +- .../paimon/flink/sink/AppendTableSink.java | 3 - .../apache/paimon/flink/sink/Committable.java | 54 +- .../flink/sink/CommittableSerializer.java | 41 +- ...DataEvolutionCompactionWorkerOperator.java | 5 +- .../sink/DynamicBucketRowWriteOperator.java | 5 - .../paimon/flink/sink/FixedBucketSink.java | 14 +- .../paimon/flink/sink/FlinkSinkBuilder.java | 12 +- .../paimon/flink/sink/FlinkTableSink.java | 10 +- .../paimon/flink/sink/FlinkTableSinkBase.java | 41 +- .../paimon/flink/sink/FlinkWriteSink.java | 6 +- .../flink/sink/LogFlinkSinkBuilder.java | 36 - .../flink/sink/LogOffsetCommittable.java | 69 -- .../paimon/flink/sink/LogSinkFunction.java | 46 - .../flink/sink/MultiTableCommittable.java | 61 +- .../sink/MultiTableCommittableSerializer.java | 41 +- .../sink/PostponeBatchWriteOperator.java | 27 +- .../paimon/flink/sink/PostponeBucketSink.java | 2 +- .../flink/sink/PostponeFixedBucketSink.java | 2 +- .../flink/sink/RewriteFileIndexSink.java | 6 +- .../paimon/flink/sink/RowAppendTableSink.java | 10 +- .../flink/sink/RowDataChannelComputer.java | 10 +- .../sink/RowDataKeyAndBucketExtractor.java | 5 - .../flink/sink/RowDataStoreWriteOperator.java | 204 +--- .../StatelessRowDataStoreWriteOperator.java | 5 +- .../paimon/flink/sink/StoreCommitter.java | 14 +- .../flink/sink/StoreMultiCommitter.java | 16 +- .../paimon/flink/sink/StoreSinkWrite.java | 2 - .../paimon/flink/sink/StoreSinkWriteImpl.java | 8 +- ...pportsRowLevelOperationFlinkTableSink.java | 11 +- .../paimon/flink/sink/TableWriteOperator.java | 12 +- .../flink/source/BaseDataTableSource.java | 52 +- .../paimon/flink/source/DataTableSource.java | 20 +- .../flink/source/FlinkSourceBuilder.java | 51 +- .../flink/source/LogHybridSourceFactory.java | 169 ---- .../paimon/flink/ChangelogModeTest.java | 24 +- ...ComputedColumnAndWatermarkTableITCase.java | 122 +-- .../apache/paimon/flink/FlinkCatalogTest.java | 151 +-- .../paimon/flink/ReadWriteTableITCase.java | 3 +- ...eCommitCompactCoordinatorOperatorTest.java | 4 +- ...hangelogCompactCoordinateOperatorTest.java | 4 +- .../ChangelogCompactSortOperatorTest.java | 17 +- .../ChangelogCompactTaskSerializerTest.java | 2 +- .../changelog/ChangelogCompactTaskTest.java | 2 +- ...ultiTableCompactionWorkerOperatorTest.java | 2 +- ...ngleTableCompactionWorkerOperatorTest.java | 2 +- ...oTagForSavepointCommitterOperatorTest.java | 3 +- .../flink/sink/CommittableSerializerTest.java | 30 +- .../flink/sink/CommitterOperatorTest.java | 40 +- .../CompactionTaskSimpleSerializerTest.java | 3 +- .../flink/sink/LogOffsetCommittableTest.java | 35 - .../MultiTableCommittableSerializerTest.java | 7 +- .../sink/RowDataChannelComputerTest.java | 22 +- .../flink/sink/StoreCompactOperatorTest.java | 5 - .../flink/sink/StoreMultiCommitterTest.java | 67 +- ...ppedManifestCommittableSerializerTest.java | 10 - .../paimon/flink/sink/WriterOperatorTest.java | 12 +- .../flink/source/DataTableSourceTest.java | 12 +- .../source/FileStoreSourceMetricsTest.java | 14 - .../flink/source/FlinkTableSourceTest.java | 4 +- .../FileStoreTableStatisticsTestBase.java | 7 +- .../PrimaryKeyTableStatisticsTest.java | 1 - .../org.apache.paimon.factories.Factory | 2 - .../flink/table/hive/LegacyHiveClasses.java | 36 - .../org/apache/paimon/hive/HiveCatalog.java | 7 +- .../hive/TableStoreHiveStorageHandler.java | 25 - .../table/store/hive/TableStoreSerDe.java | 25 - .../store/mapred/TableStoreInputFormat.java | 25 - .../store/mapred/TableStoreOutputFormat.java | 25 - .../apache/paimon/spark/SparkReadITCase.java | 2 +- .../spark/sql/TableValuedFunctionsTest.scala | 21 +- 147 files changed, 456 insertions(+), 8116 deletions(-) create mode 100644 paimon-core/src/test/resources/compatibility/manifest-committable-v11-v5 delete mode 100644 paimon-e2e-tests/src/test/java/org/apache/paimon/tests/LogStoreE2eTest.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogDeserializationSchema.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogOptions.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogSerializationSchema.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogSinkProvider.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogSourceProvider.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactory.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogStoreRegister.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/CompositePkAndMultiPartitionedTableWithKafkaLogITCase.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogSerializationTest.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactoryTest.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreRegisterITCase.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogTestUtils.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaTableTestBase.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/LogSystemITCase.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/StreamingReadWriteTableWithKafkaLogITCase.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/StreamingWarehouseITCase.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/source/LogHybridSourceFactoryTest.java delete mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/factories/FlinkFactoryUtil.java delete mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogSinkProvider.java delete mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogSourceProvider.java delete mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogStoreRegister.java delete mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogStoreTableFactory.java delete mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogWriteCallback.java delete mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LogFlinkSinkBuilder.java delete mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LogOffsetCommittable.java delete mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LogSinkFunction.java delete mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/LogHybridSourceFactory.java rename paimon-flink/{paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka => paimon-flink-common/src/test/java/org/apache/paimon/flink}/ComputedColumnAndWatermarkTableITCase.java (62%) delete mode 100644 paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/LogOffsetCommittableTest.java delete mode 100644 paimon-hive/paimon-hive-catalog/src/main/java/org/apache/flink/table/hive/LegacyHiveClasses.java delete mode 100644 paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/hive/TableStoreHiveStorageHandler.java delete mode 100644 paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/hive/TableStoreSerDe.java delete mode 100644 paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/mapred/TableStoreInputFormat.java delete mode 100644 paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/mapred/TableStoreOutputFormat.java diff --git a/docs/content/maintenance/rescale-bucket.md b/docs/content/maintenance/rescale-bucket.md index 96d6e1ee86e5..22b134c2f307 100644 --- a/docs/content/maintenance/rescale-bucket.md +++ b/docs/content/maintenance/rescale-bucket.md @@ -66,10 +66,6 @@ Please note that ``` - During overwrite period, make sure there are no other jobs writing the same table/partition. -{{< hint info >}} -__Note:__ For the table which enables log system(*e.g.* Kafka), please rescale the topic's partition as well to keep consistency. -{{< /hint >}} - ## Use Case Rescale bucket helps to handle sudden spikes in throughput. Suppose there is a daily streaming ETL task to sync transaction data. The table's DDL and pipeline 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 a5e0e1919e4f..78270d7523bc 100644 --- a/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java @@ -21,7 +21,6 @@ import org.apache.paimon.annotation.Documentation; import org.apache.paimon.annotation.Documentation.ExcludeFromDocumentation; import org.apache.paimon.annotation.Documentation.Immutable; -import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.compression.CompressOptions; import org.apache.paimon.fileindex.FileIndexOptions; import org.apache.paimon.fs.Path; @@ -969,42 +968,6 @@ public InlineElement getDescription() { .withDescription( "The delay duration of stream read when scan incremental snapshots."); - @ExcludeFromDocumentation("Confused without log system") - public static final ConfigOption LOG_CONSISTENCY = - key("log.consistency") - .enumType(LogConsistency.class) - .defaultValue(LogConsistency.TRANSACTIONAL) - .withDescription("Specify the log consistency mode for table."); - - @ExcludeFromDocumentation("Confused without log system") - public static final ConfigOption LOG_CHANGELOG_MODE = - key("log.changelog-mode") - .enumType(LogChangelogMode.class) - .defaultValue(LogChangelogMode.AUTO) - .withDescription("Specify the log changelog mode for table."); - - @ExcludeFromDocumentation("Confused without log system") - public static final ConfigOption LOG_KEY_FORMAT = - key("log.key.format") - .stringType() - .defaultValue("json") - .withDescription( - "Specify the key message format of log system with primary key."); - - @ExcludeFromDocumentation("Confused without log system") - public static final ConfigOption LOG_FORMAT = - key("log.format") - .stringType() - .defaultValue("debezium-json") - .withDescription("Specify the message format of log system."); - - @ExcludeFromDocumentation("Confused without log system") - public static final ConfigOption LOG_IGNORE_DELETE = - key("log.ignore-delete") - .booleanType() - .defaultValue(false) - .withDescription("Specify whether the log system ignores delete records."); - public static final ConfigOption AUTO_CREATE = key("auto-create") .booleanType() @@ -1272,13 +1235,6 @@ public InlineElement getDescription() { "Only used to force TableScan to construct suitable 'StartingUpScanner' and 'FollowUpScanner' " + "dedicated internal streaming scan."); - public static final ConfigOption STREAMING_READ_MODE = - key("streaming-read-mode") - .enumType(StreamingReadMode.class) - .noDefaultValue() - .withDescription( - "The mode of streaming read that specifies to read the data of table file or log."); - @ExcludeFromDocumentation("Internal use only") public static final ConfigOption BATCH_SCAN_MODE = key("batch-scan-mode") @@ -2770,10 +2726,6 @@ public boolean scanPlanSortPartition() { } public StartupMode startupMode() { - return startupMode(options); - } - - public static StartupMode startupMode(Options options) { StartupMode mode = options.get(SCAN_MODE); if (mode == StartupMode.DEFAULT) { if (options.getOptional(SCAN_TIMESTAMP_MILLIS).isPresent() @@ -3001,10 +2953,6 @@ public Integer fullCompactionDeltaCommits() { return options.get(FULL_COMPACTION_DELTA_COMMITS); } - public static StreamingReadMode streamReadType(Options options) { - return options.get(STREAMING_READ_MODE); - } - public Duration consumerExpireTime() { return options.get(CONSUMER_EXPIRATION_TIME); } @@ -3471,67 +3419,6 @@ public InlineElement getDescription() { } } - /** Specifies the log consistency mode for table. */ - public enum LogConsistency implements DescribedEnum { - TRANSACTIONAL( - "transactional", - "Only the data after the checkpoint can be seen by readers, the latency depends on checkpoint interval."), - - EVENTUAL( - "eventual", - "Immediate data visibility, you may see some intermediate states, " - + "but eventually the right results will be produced, only works for table with primary key."); - - private final String value; - private final String description; - - LogConsistency(String value, String description) { - this.value = value; - this.description = description; - } - - @Override - public String toString() { - return value; - } - - @Override - public InlineElement getDescription() { - return text(description); - } - } - - /** Specifies the log changelog mode for table. */ - public enum LogChangelogMode implements DescribedEnum { - AUTO("auto", "Upsert for table with primary key, all for table without primary key."), - - ALL("all", "The log system stores all changes including UPDATE_BEFORE."), - - UPSERT( - "upsert", - "The log system does not store the UPDATE_BEFORE changes, the log consumed job" - + " will automatically add the normalized node, relying on the state" - + " to generate the required update_before."); - - private final String value; - private final String description; - - LogChangelogMode(String value, String description) { - this.value = value; - this.description = description; - } - - @Override - public String toString() { - return value; - } - - @Override - public InlineElement getDescription() { - return text(description); - } - } - /** Specifies the changelog producer for table. */ public enum ChangelogProducer implements DescribedEnum { NONE("none", "No changelog file."), @@ -3563,49 +3450,6 @@ public InlineElement getDescription() { } } - /** Specifies the type for streaming read. */ - public enum StreamingReadMode implements DescribedEnum { - LOG("log", "Read from the data of table log store."), - FILE("file", "Read from the data of table file store."); - - private final String value; - private final String description; - - StreamingReadMode(String value, String description) { - this.value = value; - this.description = description; - } - - @Override - public String toString() { - return value; - } - - @Override - public InlineElement getDescription() { - return text(description); - } - - public String getValue() { - return value; - } - - @VisibleForTesting - public static StreamingReadMode fromValue(String value) { - for (StreamingReadMode formatType : StreamingReadMode.values()) { - if (formatType.value.equals(value)) { - return formatType; - } - } - throw new IllegalArgumentException( - String.format( - "Invalid format type %s, only support [%s]", - value, - StringUtils.join( - Arrays.stream(StreamingReadMode.values()).iterator(), ","))); - } - } - /** Inner stream scan mode for some internal requirements. */ public enum StreamScanMode implements DescribedEnum { NONE("none", "No requirement."), diff --git a/paimon-api/src/main/java/org/apache/paimon/Snapshot.java b/paimon-api/src/main/java/org/apache/paimon/Snapshot.java index a318a72e40a3..894b8724484e 100644 --- a/paimon-api/src/main/java/org/apache/paimon/Snapshot.java +++ b/paimon-api/src/main/java/org/apache/paimon/Snapshot.java @@ -64,7 +64,6 @@ public class Snapshot implements Serializable { public static final long FIRST_SNAPSHOT_ID = 1; - public static final int TABLE_STORE_02_VERSION = 1; protected static final int CURRENT_VERSION = 3; protected static final String FIELD_VERSION = "version"; @@ -81,7 +80,6 @@ public class Snapshot implements Serializable { protected static final String FIELD_COMMIT_IDENTIFIER = "commitIdentifier"; protected static final String FIELD_COMMIT_KIND = "commitKind"; protected static final String FIELD_TIME_MILLIS = "timeMillis"; - protected static final String FIELD_LOG_OFFSETS = "logOffsets"; protected static final String FIELD_TOTAL_RECORD_COUNT = "totalRecordCount"; protected static final String FIELD_DELTA_RECORD_COUNT = "deltaRecordCount"; protected static final String FIELD_CHANGELOG_RECORD_COUNT = "changelogRecordCount"; @@ -91,10 +89,8 @@ public class Snapshot implements Serializable { protected static final String FIELD_NEXT_ROW_ID = "nextRowId"; // version of snapshot - // null for paimon <= 0.2 @JsonProperty(FIELD_VERSION) - @Nullable - protected final Integer version; + protected final int version; @JsonProperty(FIELD_ID) protected final long id; @@ -106,6 +102,7 @@ public class Snapshot implements Serializable { @JsonProperty(FIELD_BASE_MANIFEST_LIST) protected final String baseManifestList; + // null for paimon <= 1.0 @JsonProperty(FIELD_BASE_MANIFEST_LIST_SIZE) @JsonInclude(JsonInclude.Include.NON_NULL) @Nullable @@ -116,17 +113,20 @@ public class Snapshot implements Serializable { @JsonProperty(FIELD_DELTA_MANIFEST_LIST) protected final String deltaManifestList; + // null for paimon <= 1.0 @JsonProperty(FIELD_DELTA_MANIFEST_LIST_SIZE) @JsonInclude(JsonInclude.Include.NON_NULL) @Nullable protected final Long deltaManifestListSize; // a manifest list recording all changelog produced in this snapshot - // null if no changelog is produced, or for paimon <= 0.2 + // null if no changelog is produced @JsonProperty(FIELD_CHANGELOG_MANIFEST_LIST) + @JsonInclude(JsonInclude.Include.NON_NULL) @Nullable protected final String changelogManifestList; + // null for paimon <= 1.0 or no changelog @JsonProperty(FIELD_CHANGELOG_MANIFEST_LIST_SIZE) @JsonInclude(JsonInclude.Include.NON_NULL) @Nullable @@ -157,32 +157,22 @@ public class Snapshot implements Serializable { @JsonProperty(FIELD_TIME_MILLIS) protected final long timeMillis; - @JsonProperty(FIELD_LOG_OFFSETS) - @JsonInclude(JsonInclude.Include.NON_NULL) - @Nullable - protected final Map logOffsets; - // record count of all changes occurred in this snapshot - // null for paimon <= 0.3 @JsonProperty(FIELD_TOTAL_RECORD_COUNT) - @Nullable - protected final Long totalRecordCount; + protected final long totalRecordCount; // record count of all new changes occurred in this snapshot - // null for paimon <= 0.3 @JsonProperty(FIELD_DELTA_RECORD_COUNT) - @Nullable - protected final Long deltaRecordCount; + protected final long deltaRecordCount; // record count of all changelog produced in this snapshot - // null for paimon <= 0.3 + // null if no changelog @JsonProperty(FIELD_CHANGELOG_RECORD_COUNT) @JsonInclude(JsonInclude.Include.NON_NULL) @Nullable protected final Long changelogRecordCount; // watermark for input records - // null for paimon <= 0.3 // null if there is no watermark in new committing, and the previous snapshot does not have a // watermark @JsonProperty(FIELD_WATERMARK) @@ -198,15 +188,15 @@ public class Snapshot implements Serializable { protected final String statistics; // properties - // null for paimon <= 1.1 or empty properties + // null for empty properties @JsonInclude(JsonInclude.Include.NON_NULL) @JsonProperty(FIELD_PROPERTIES) @Nullable protected final Map properties; - @Nullable @JsonInclude(JsonInclude.Include.NON_NULL) @JsonProperty(FIELD_NEXT_ROW_ID) + @Nullable protected final Long nextRowId; public Snapshot( @@ -223,9 +213,8 @@ public Snapshot( long commitIdentifier, CommitKind commitKind, long timeMillis, - Map logOffsets, - @Nullable Long totalRecordCount, - @Nullable Long deltaRecordCount, + long totalRecordCount, + long deltaRecordCount, @Nullable Long changelogRecordCount, @Nullable Long watermark, @Nullable String statistics, @@ -246,7 +235,6 @@ public Snapshot( commitIdentifier, commitKind, timeMillis, - logOffsets, totalRecordCount, deltaRecordCount, changelogRecordCount, @@ -258,7 +246,7 @@ public Snapshot( @JsonCreator public Snapshot( - @JsonProperty(FIELD_VERSION) @Nullable Integer version, + @JsonProperty(FIELD_VERSION) int version, @JsonProperty(FIELD_ID) long id, @JsonProperty(FIELD_SCHEMA_ID) long schemaId, @JsonProperty(FIELD_BASE_MANIFEST_LIST) String baseManifestList, @@ -273,9 +261,8 @@ public Snapshot( @JsonProperty(FIELD_COMMIT_IDENTIFIER) long commitIdentifier, @JsonProperty(FIELD_COMMIT_KIND) CommitKind commitKind, @JsonProperty(FIELD_TIME_MILLIS) long timeMillis, - @JsonProperty(FIELD_LOG_OFFSETS) @Nullable Map logOffsets, - @JsonProperty(FIELD_TOTAL_RECORD_COUNT) @Nullable Long totalRecordCount, - @JsonProperty(FIELD_DELTA_RECORD_COUNT) @Nullable Long deltaRecordCount, + @JsonProperty(FIELD_TOTAL_RECORD_COUNT) long totalRecordCount, + @JsonProperty(FIELD_DELTA_RECORD_COUNT) long deltaRecordCount, @JsonProperty(FIELD_CHANGELOG_RECORD_COUNT) @Nullable Long changelogRecordCount, @JsonProperty(FIELD_WATERMARK) @Nullable Long watermark, @JsonProperty(FIELD_STATISTICS) @Nullable String statistics, @@ -295,7 +282,6 @@ public Snapshot( this.commitIdentifier = commitIdentifier; this.commitKind = commitKind; this.timeMillis = timeMillis; - this.logOffsets = logOffsets; this.totalRecordCount = totalRecordCount; this.deltaRecordCount = deltaRecordCount; this.changelogRecordCount = changelogRecordCount; @@ -307,8 +293,7 @@ public Snapshot( @JsonGetter(FIELD_VERSION) public int version() { - // there is no version field for paimon <= 0.2 - return version == null ? TABLE_STORE_02_VERSION : version; + return version; } @JsonGetter(FIELD_ID) @@ -381,21 +366,13 @@ public long timeMillis() { return timeMillis; } - @JsonGetter(FIELD_LOG_OFFSETS) - @Nullable - public Map logOffsets() { - return logOffsets; - } - @JsonGetter(FIELD_TOTAL_RECORD_COUNT) - @Nullable - public Long totalRecordCount() { + public long totalRecordCount() { return totalRecordCount; } @JsonGetter(FIELD_DELTA_RECORD_COUNT) - @Nullable - public Long deltaRecordCount() { + public long deltaRecordCount() { return deltaRecordCount; } @@ -450,7 +427,6 @@ public int hashCode() { commitIdentifier, commitKind, timeMillis, - logOffsets, totalRecordCount, deltaRecordCount, changelogRecordCount, @@ -483,7 +459,6 @@ public boolean equals(Object o) { && commitIdentifier == that.commitIdentifier && commitKind == that.commitKind && timeMillis == that.timeMillis - && Objects.equals(logOffsets, that.logOffsets) && Objects.equals(totalRecordCount, that.totalRecordCount) && Objects.equals(deltaRecordCount, that.deltaRecordCount) && Objects.equals(changelogRecordCount, that.changelogRecordCount) diff --git a/paimon-core/src/main/java/org/apache/paimon/Changelog.java b/paimon-core/src/main/java/org/apache/paimon/Changelog.java index a715ab8234c3..0ab3429dfc1f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/Changelog.java +++ b/paimon-core/src/main/java/org/apache/paimon/Changelog.java @@ -57,7 +57,6 @@ public Changelog(Snapshot snapshot) { snapshot.commitIdentifier(), snapshot.commitKind(), snapshot.timeMillis(), - snapshot.logOffsets(), snapshot.totalRecordCount(), snapshot.deltaRecordCount(), snapshot.changelogRecordCount(), @@ -69,7 +68,7 @@ public Changelog(Snapshot snapshot) { @JsonCreator public Changelog( - @JsonProperty(FIELD_VERSION) @Nullable Integer version, + @JsonProperty(FIELD_VERSION) int version, @JsonProperty(FIELD_ID) long id, @JsonProperty(FIELD_SCHEMA_ID) long schemaId, @JsonProperty(FIELD_BASE_MANIFEST_LIST) String baseManifestList, @@ -84,9 +83,8 @@ public Changelog( @JsonProperty(FIELD_COMMIT_IDENTIFIER) long commitIdentifier, @JsonProperty(FIELD_COMMIT_KIND) CommitKind commitKind, @JsonProperty(FIELD_TIME_MILLIS) long timeMillis, - @JsonProperty(FIELD_LOG_OFFSETS) Map logOffsets, - @JsonProperty(FIELD_TOTAL_RECORD_COUNT) @Nullable Long totalRecordCount, - @JsonProperty(FIELD_DELTA_RECORD_COUNT) @Nullable Long deltaRecordCount, + @JsonProperty(FIELD_TOTAL_RECORD_COUNT) long totalRecordCount, + @JsonProperty(FIELD_DELTA_RECORD_COUNT) long deltaRecordCount, @JsonProperty(FIELD_CHANGELOG_RECORD_COUNT) @Nullable Long changelogRecordCount, @JsonProperty(FIELD_WATERMARK) @Nullable Long watermark, @JsonProperty(FIELD_STATISTICS) @Nullable String statistics, @@ -107,7 +105,6 @@ public Changelog( commitIdentifier, commitKind, timeMillis, - logOffsets, totalRecordCount, deltaRecordCount, changelogRecordCount, diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittable.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittable.java index e8e12a668d25..18712efaaed3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittable.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittable.java @@ -33,7 +33,6 @@ public class ManifestCommittable { private final long identifier; @Nullable private final Long watermark; - private final Map logOffsets; private final Map properties; private final List commitMessages; @@ -44,28 +43,22 @@ public ManifestCommittable(long identifier) { public ManifestCommittable(long identifier, @Nullable Long watermark) { this.identifier = identifier; this.watermark = watermark; - this.logOffsets = new HashMap<>(); this.commitMessages = new ArrayList<>(); this.properties = new HashMap<>(); } public ManifestCommittable( - long identifier, - @Nullable Long watermark, - Map logOffsets, - List commitMessages) { - this(identifier, watermark, logOffsets, commitMessages, new HashMap<>()); + long identifier, @Nullable Long watermark, List commitMessages) { + this(identifier, watermark, commitMessages, new HashMap<>()); } public ManifestCommittable( long identifier, @Nullable Long watermark, - Map logOffsets, List commitMessages, Map properties) { this.identifier = identifier; this.watermark = watermark; - this.logOffsets = logOffsets; this.commitMessages = commitMessages; this.properties = properties; } @@ -74,16 +67,6 @@ public void addFileCommittable(CommitMessage commitMessage) { commitMessages.add(commitMessage); } - public void addLogOffset(int bucket, long offset, boolean allowDuplicate) { - if (!allowDuplicate && logOffsets.containsKey(bucket)) { - throw new RuntimeException( - String.format( - "bucket-%d appears multiple times, which is not possible.", bucket)); - } - long newOffset = Math.max(logOffsets.getOrDefault(bucket, offset), offset); - logOffsets.put(bucket, newOffset); - } - public void addProperty(String key, String value) { properties.put(key, value); } @@ -97,10 +80,6 @@ public Long watermark() { return watermark; } - public Map logOffsets() { - return logOffsets; - } - public List fileCommittables() { return commitMessages; } @@ -120,14 +99,13 @@ public boolean equals(Object o) { ManifestCommittable that = (ManifestCommittable) o; return Objects.equals(identifier, that.identifier) && Objects.equals(watermark, that.watermark) - && Objects.equals(logOffsets, that.logOffsets) && Objects.equals(commitMessages, that.commitMessages) && Objects.equals(properties, that.properties); } @Override public int hashCode() { - return Objects.hash(identifier, watermark, logOffsets, commitMessages, properties); + return Objects.hash(identifier, watermark, commitMessages, properties); } @Override @@ -136,9 +114,8 @@ public String toString() { "ManifestCommittable {" + "identifier = %s, " + "watermark = %s, " - + "logOffsets = %s, " + "commitMessages = %s, " + "properties = %s}", - identifier, watermark, logOffsets, commitMessages, properties); + identifier, watermark, commitMessages, properties); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittableSerializer.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittableSerializer.java index 67845099e424..7cb6dc0e0fa4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittableSerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittableSerializer.java @@ -34,7 +34,7 @@ /** {@link VersionedSerializer} for {@link ManifestCommittable}. */ public class ManifestCommittableSerializer implements VersionedSerializer { - private static final int CURRENT_VERSION = 4; + private static final int CURRENT_VERSION = 5; private final CommitMessageSerializer commitMessageSerializer; @@ -61,22 +61,12 @@ public byte[] serialize(ManifestCommittable obj) throws IOException { view.writeBoolean(false); view.writeLong(watermark); } - serializeOffsets(view, obj.logOffsets()); serializeProperties(view, obj.properties()); view.writeInt(commitMessageSerializer.getVersion()); commitMessageSerializer.serializeList(obj.fileCommittables(), view); return out.toByteArray(); } - private void serializeOffsets(DataOutputViewStreamWrapper view, Map offsets) - throws IOException { - view.writeInt(offsets.size()); - for (Map.Entry entry : offsets.entrySet()) { - view.writeInt(entry.getKey()); - view.writeLong(entry.getValue()); - } - } - private void serializeProperties( DataOutputViewStreamWrapper view, Map properties) throws IOException { view.writeInt(properties.size()); @@ -100,9 +90,11 @@ public ManifestCommittable deserialize(int version, byte[] serialized) throws IO DataInputDeserializer view = new DataInputDeserializer(serialized); long identifier = view.readLong(); Long watermark = view.readBoolean() ? null : view.readLong(); - Map offsets = deserializeOffsets(view); + if (version <= 4) { + skipLegacyLogOffsets(view); + } Map properties = - version == CURRENT_VERSION ? deserializeProperties(view) : new HashMap<>(); + version >= 4 ? deserializeProperties(view) : new HashMap<>(); int fileCommittableSerializerVersion = view.readInt(); List fileCommittables; try { @@ -119,7 +111,7 @@ public ManifestCommittable deserialize(int version, byte[] serialized) throws IO if (!view.readBoolean()) { view.readLong(); } - deserializeOffsets(view); + skipLegacyLogOffsets(view); view.readInt(); if (legacyV2CommitMessageSerializer == null) { @@ -128,17 +120,15 @@ public ManifestCommittable deserialize(int version, byte[] serialized) throws IO fileCommittables = legacyV2CommitMessageSerializer.deserializeList(view); } - return new ManifestCommittable( - identifier, watermark, offsets, fileCommittables, properties); + return new ManifestCommittable(identifier, watermark, fileCommittables, properties); } - private Map deserializeOffsets(DataInputDeserializer view) throws IOException { + private void skipLegacyLogOffsets(DataInputDeserializer view) throws IOException { int size = view.readInt(); - Map offsets = new HashMap<>(size); for (int i = 0; i < size; i++) { - offsets.put(view.readInt(), view.readLong()); + view.readInt(); + view.readLong(); } - return offsets; } private Map deserializeProperties(DataInputDeserializer view) diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java index 756611b07605..f95ba19221a2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java @@ -26,6 +26,8 @@ import org.apache.paimon.types.RowType; import org.apache.paimon.types.TinyIntType; +import javax.annotation.Nullable; + import java.util.Arrays; import java.util.List; @@ -68,6 +70,14 @@ static long recordCount(List manifestEntries) { return manifestEntries.stream().mapToLong(manifest -> manifest.file().rowCount()).sum(); } + @Nullable + static Long nullableRecordCount(List manifestEntries) { + if (manifestEntries.isEmpty()) { + return null; + } + return manifestEntries.stream().mapToLong(manifest -> manifest.file().rowCount()).sum(); + } + static long recordCountAdd(List manifestEntries) { return manifestEntries.stream() .filter(manifestEntry -> FileKind.ADD.equals(manifestEntry.kind())) diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java index e5d4cd2ff20c..1355bb4a3acd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java @@ -95,7 +95,7 @@ import static java.util.Collections.emptyList; import static org.apache.paimon.deletionvectors.DeletionVectorsIndexFile.DELETION_VECTORS_INDEX; import static org.apache.paimon.format.blob.BlobFileFormat.isBlobFile; -import static org.apache.paimon.manifest.ManifestEntry.recordCount; +import static org.apache.paimon.manifest.ManifestEntry.nullableRecordCount; import static org.apache.paimon.manifest.ManifestEntry.recordCountAdd; import static org.apache.paimon.manifest.ManifestEntry.recordCountDelete; import static org.apache.paimon.operation.commit.ConflictDetection.hasConflictChecked; @@ -383,7 +383,6 @@ public int commit(ManifestCommittable committable, boolean checkAppendFiles) { changes.appendIndexFiles), committable.identifier(), committable.watermark(), - committable.logOffsets(), committable.properties(), CommitKindProvider.provider(commitKind), conflictCheck, @@ -421,7 +420,6 @@ public int commit(ManifestCommittable committable, boolean checkAppendFiles) { changes.compactIndexFiles), committable.identifier(), committable.watermark(), - committable.logOffsets(), committable.properties(), CommitKindProvider.provider(CommitKind.COMPACT), hasConflictChecked(safeLatestSnapshotId), @@ -574,7 +572,6 @@ public int overwritePartition( changes.appendIndexFiles, committable.identifier(), committable.watermark(), - committable.logOffsets(), committable.properties()); generatedSnapshot += 1; } @@ -588,7 +585,6 @@ public int overwritePartition( changes.compactIndexFiles), committable.identifier(), committable.watermark(), - committable.logOffsets(), committable.properties(), CommitKindProvider.provider(CommitKind.COMPACT), mustConflictCheck(), @@ -688,25 +684,13 @@ public void dropPartitions(List> partitions, long commitIden } tryOverwritePartition( - partitionFilter, - emptyList(), - emptyList(), - commitIdentifier, - null, - new HashMap<>(), - new HashMap<>()); + partitionFilter, emptyList(), emptyList(), commitIdentifier, null, new HashMap<>()); } @Override public void truncateTable(long commitIdentifier) { tryOverwritePartition( - null, - emptyList(), - emptyList(), - commitIdentifier, - null, - new HashMap<>(), - new HashMap<>()); + null, emptyList(), emptyList(), commitIdentifier, null, new HashMap<>()); } @Override @@ -741,7 +725,6 @@ public void commitStatistics(Statistics stats, long commitIdentifier) { commitIdentifier, null, Collections.emptyMap(), - Collections.emptyMap(), CommitKindProvider.provider(CommitKind.ANALYZE), noConflictCheck(), statsFileName); @@ -768,7 +751,6 @@ private int tryCommit( CommitChangesProvider changesProvider, long identifier, @Nullable Long watermark, - Map logOffsets, Map properties, CommitKindProvider commitKindProvider, ConflictCheck conflictCheck, @@ -788,7 +770,6 @@ private int tryCommit( changes.indexFiles, identifier, watermark, - logOffsets, properties, commitKind, latestSnapshot, @@ -828,7 +809,6 @@ private int tryOverwritePartition( List indexFiles, long identifier, @Nullable Long watermark, - Map logOffsets, Map properties) { CommitKindProvider commitKindProvider = commitChanges -> @@ -842,7 +822,6 @@ private int tryOverwritePartition( numBucket, changes, indexFiles, latestSnapshot, partitionFilter), identifier, watermark, - logOffsets, properties, commitKindProvider, mustConflictCheck(), @@ -857,7 +836,6 @@ CommitResult tryCommitOnce( List indexFiles, long identifier, @Nullable Long watermark, - Map logOffsets, Map properties, CommitKind commitKind, @Nullable Snapshot latestSnapshot, @@ -966,15 +944,9 @@ CommitResult tryCommitOnce( long previousTotalRecordCount = 0L; Long currentWatermark = watermark; if (latestSnapshot != null) { - previousTotalRecordCount = scanner.totalRecordCount(latestSnapshot); + previousTotalRecordCount = latestSnapshot.totalRecordCount(); // read all previous manifest files mergeBeforeManifests = manifestList.readDataManifests(latestSnapshot); - // read the last snapshot to complete the bucket's offsets when logOffsets does not - // contain all buckets - Map latestLogOffsets = latestSnapshot.logOffsets(); - if (latestLogOffsets != null) { - latestLogOffsets.forEach(logOffsets::putIfAbsent); - } Long latestWatermark = latestSnapshot.watermark(); if (latestWatermark != null) { currentWatermark = @@ -1060,10 +1032,9 @@ CommitResult tryCommitOnce( identifier, commitKind, System.currentTimeMillis(), - logOffsets, totalRecordCount, deltaRecordCount, - recordCount(changelogFiles), + nullableRecordCount(changelogFiles), currentWatermark, statsFileName, // if empty properties, just set to null @@ -1148,10 +1119,9 @@ public boolean replaceManifestList( Long.MAX_VALUE, CommitKind.OVERWRITE, System.currentTimeMillis(), - latest.logOffsets(), totalRecordCount, 0L, - 0L, + null, latest.watermark(), latest.statistics(), // if empty properties, just set to null @@ -1276,10 +1246,9 @@ private boolean compactManifestOnce() { Long.MAX_VALUE, CommitKind.COMPACT, System.currentTimeMillis(), - latestSnapshot.logOffsets(), latestSnapshot.totalRecordCount(), 0L, - 0L, + null, latestSnapshot.watermark(), latestSnapshot.statistics(), latestSnapshot.properties(), diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java index b315d1d89b14..129658cc3645 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java @@ -44,8 +44,6 @@ import java.util.Map; import java.util.stream.Collectors; -import static org.apache.paimon.manifest.ManifestEntry.recordCount; - /** Scan operation which produces a plan. */ public interface FileStoreScan { @@ -107,17 +105,6 @@ public interface FileStoreScan { /** Produce a {@link Plan}. */ Plan plan(); - /** - * Return record count of all changes occurred in this snapshot given the scan. - * - * @return total record count of Snapshot. - */ - default Long totalRecordCount(Snapshot snapshot) { - return snapshot.totalRecordCount() == null - ? (Long) recordCount(withSnapshot(snapshot.id()).plan().files()) - : snapshot.totalRecordCount(); - } - /** * Read {@link SimpleFileEntry}s, SimpleFileEntry only retains some critical information, so it * cannot perform filtering based on statistical information. diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/ManifestsReader.java b/paimon-core/src/main/java/org/apache/paimon/operation/ManifestsReader.java index fed68d699758..144121ca6e83 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/ManifestsReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/ManifestsReader.java @@ -143,10 +143,6 @@ private List readManifests(Snapshot snapshot, ScanMode scanMod case DELTA: return manifestList.readDeltaManifests(snapshot); case CHANGELOG: - if (snapshot.version() <= Snapshot.TABLE_STORE_02_VERSION) { - throw new UnsupportedOperationException( - "Unsupported snapshot version: " + snapshot.version()); - } return manifestList.readChangelogManifests(snapshot); default: throw new UnsupportedOperationException("Unknown scan kind " + scanMode.name()); diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/commit/CommitScanner.java b/paimon-core/src/main/java/org/apache/paimon/operation/commit/CommitScanner.java index 0a93d2b92023..da8b5c7563c9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/commit/CommitScanner.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/commit/CommitScanner.java @@ -122,8 +122,4 @@ public CommitChanges readOverwriteChanges( indexChangesWithOverwrite.addAll(indexFiles); return new CommitChanges(changesWithOverwrite, emptyList(), indexChangesWithOverwrite); } - - public long totalRecordCount(Snapshot snapshot) { - return scan.totalRecordCount(snapshot); - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/format/TwoPhaseCommitMessage.java b/paimon-core/src/main/java/org/apache/paimon/table/format/TwoPhaseCommitMessage.java index cc43636d2feb..ffb08064dd17 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/format/TwoPhaseCommitMessage.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/format/TwoPhaseCommitMessage.java @@ -26,7 +26,8 @@ /** {@link CommitMessage} implementation for format table. */ public class TwoPhaseCommitMessage implements CommitMessage { - private TwoPhaseOutputStream.Committer committer; + + private final TwoPhaseOutputStream.Committer committer; public TwoPhaseCommitMessage(TwoPhaseOutputStream.Committer committer) { this.committer = committer; diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/KeyAndBucketExtractor.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/KeyAndBucketExtractor.java index f6ba262db0cc..c92f3c1f8195 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/KeyAndBucketExtractor.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/KeyAndBucketExtractor.java @@ -24,13 +24,12 @@ import org.slf4j.LoggerFactory; /** - * Utility interface to extract partition keys, bucket id, primary keys for file store ({@code - * trimmedPrimaryKey}) and primary keys for external log system ({@code logPrimaryKey}) from the - * given record. + * Utility interface to extract partition keys, bucket id, primary keys. * * @param type of record */ public interface KeyAndBucketExtractor { + Logger LOG = LoggerFactory.getLogger(KeyAndBucketExtractor.class); void setRecord(T record); @@ -40,6 +39,4 @@ public interface KeyAndBucketExtractor { int bucket(); BinaryRow trimmedPrimaryKey(); - - BinaryRow logPrimaryKey(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/RowKeyExtractor.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/RowKeyExtractor.java index 04c3c0bc3402..455aaa4aa5e9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/RowKeyExtractor.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/RowKeyExtractor.java @@ -18,8 +18,6 @@ package org.apache.paimon.table.sink; -import org.apache.paimon.codegen.CodeGenUtils; -import org.apache.paimon.codegen.Projection; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.schema.TableSchema; @@ -28,19 +26,14 @@ public abstract class RowKeyExtractor implements KeyAndBucketExtractor { private final RowPartitionKeyExtractor partitionKeyExtractor; - private final Projection logPrimaryKeyProjection; protected InternalRow record; private BinaryRow partition; private BinaryRow trimmedPrimaryKey; - private BinaryRow logPrimaryKey; public RowKeyExtractor(TableSchema schema) { - partitionKeyExtractor = new RowPartitionKeyExtractor(schema); - logPrimaryKeyProjection = - CodeGenUtils.newProjection( - schema.logicalRowType(), schema.projection(schema.primaryKeys())); + this.partitionKeyExtractor = new RowPartitionKeyExtractor(schema); } @Override @@ -48,7 +41,6 @@ public void setRecord(InternalRow record) { this.record = record; this.partition = null; this.trimmedPrimaryKey = null; - this.logPrimaryKey = null; } @Override @@ -66,12 +58,4 @@ public BinaryRow trimmedPrimaryKey() { } return trimmedPrimaryKey; } - - @Override - public BinaryRow logPrimaryKey() { - if (logPrimaryKey == null) { - logPrimaryKey = logPrimaryKeyProjection.apply(record); - } - return logPrimaryKey; - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/TableWriteImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/TableWriteImpl.java index 82b0b87439bd..5f91c07bb029 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/TableWriteImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/TableWriteImpl.java @@ -224,15 +224,6 @@ private SinkRecord toSinkRecord(InternalRow row, int bucket) { row); } - public SinkRecord toLogRecord(SinkRecord record) { - keyAndBucketExtractor.setRecord(record.row()); - return new SinkRecord( - record.partition(), - bucketMode == BucketMode.BUCKET_UNAWARE ? -1 : record.bucket(), - keyAndBucketExtractor.logPrimaryKey(), - record.row()); - } - @Override public void compact(BinaryRow partition, int bucket, boolean fullCompaction) throws Exception { write.compact(partition, bucket, fullCompaction); diff --git a/paimon-core/src/main/java/org/apache/paimon/tag/Tag.java b/paimon-core/src/main/java/org/apache/paimon/tag/Tag.java index 73a2e19cc842..192df2d4705a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/tag/Tag.java +++ b/paimon-core/src/main/java/org/apache/paimon/tag/Tag.java @@ -58,7 +58,7 @@ public class Tag extends Snapshot { @JsonCreator public Tag( - @JsonProperty(FIELD_VERSION) @Nullable Integer version, + @JsonProperty(FIELD_VERSION) int version, @JsonProperty(FIELD_ID) long id, @JsonProperty(FIELD_SCHEMA_ID) long schemaId, @JsonProperty(FIELD_BASE_MANIFEST_LIST) String baseManifestList, @@ -73,9 +73,8 @@ public Tag( @JsonProperty(FIELD_COMMIT_IDENTIFIER) long commitIdentifier, @JsonProperty(FIELD_COMMIT_KIND) CommitKind commitKind, @JsonProperty(FIELD_TIME_MILLIS) long timeMillis, - @JsonProperty(FIELD_LOG_OFFSETS) Map logOffsets, - @JsonProperty(FIELD_TOTAL_RECORD_COUNT) @Nullable Long totalRecordCount, - @JsonProperty(FIELD_DELTA_RECORD_COUNT) @Nullable Long deltaRecordCount, + @JsonProperty(FIELD_TOTAL_RECORD_COUNT) long totalRecordCount, + @JsonProperty(FIELD_DELTA_RECORD_COUNT) long deltaRecordCount, @JsonProperty(FIELD_CHANGELOG_RECORD_COUNT) @Nullable Long changelogRecordCount, @JsonProperty(FIELD_WATERMARK) @Nullable Long watermark, @JsonProperty(FIELD_STATISTICS) @Nullable String statistics, @@ -98,7 +97,6 @@ public Tag( commitIdentifier, commitKind, timeMillis, - logOffsets, totalRecordCount, deltaRecordCount, changelogRecordCount, @@ -120,11 +118,6 @@ public Tag( return tagTimeRetained; } - @Override - public String toJson() { - return JsonSerdeUtil.toJson(this); - } - public static Tag fromSnapshotAndTagTtl( Snapshot snapshot, Duration tagTimeRetained, LocalDateTime tagCreateTime) { return new Tag( @@ -142,7 +135,6 @@ public static Tag fromSnapshotAndTagTtl( snapshot.commitIdentifier(), snapshot.commitKind(), snapshot.timeMillis(), - snapshot.logOffsets(), snapshot.totalRecordCount(), snapshot.deltaRecordCount(), snapshot.changelogRecordCount(), @@ -170,7 +162,6 @@ public Snapshot trimToSnapshot() { commitIdentifier, commitKind, timeMillis, - logOffsets, totalRecordCount, deltaRecordCount, changelogRecordCount, diff --git a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java index aa405060f422..f904cde0819f 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java @@ -205,14 +205,6 @@ public ExpireSnapshots newChangelogExpire(ExpireConfig config) { return impl; } - public List commitData( - List kvs, - Function partitionCalculator, - Function bucketCalculator) - throws Exception { - return commitData(kvs, partitionCalculator, bucketCalculator, new HashMap<>()); - } - public List commitDataWatermark( List kvs, Function partitionCalculator, Long watermark) throws Exception { @@ -230,8 +222,7 @@ public List commitDataWatermark( public List commitData( List kvs, Function partitionCalculator, - Function bucketCalculator, - Map logOffsets) + Function bucketCalculator) throws Exception { return commitDataImpl( kvs, @@ -241,11 +232,7 @@ public List commitData( null, null, Collections.emptyList(), - (commit, committable) -> { - logOffsets.forEach( - (bucket, offset) -> committable.addLogOffset(bucket, offset, false)); - commit.commit(committable, false); - }); + (commit, committable) -> commit.commit(committable, false)); } public List overwriteData( diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerCompatibilityTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerCompatibilityTest.java index 98023723b057..bc7775d4af36 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerCompatibilityTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerCompatibilityTest.java @@ -45,6 +45,105 @@ /** Compatibility Test for {@link ManifestCommittableSerializer}. */ public class ManifestCommittableSerializerCompatibilityTest { + @Test + public void testCompatibilityToV5CommitV11() throws IOException { + String fileName = "manifest-committable-v11-v5"; + + SimpleStats keyStats = + new SimpleStats( + singleColumn("min_key"), + singleColumn("max_key"), + fromLongArray(new Long[] {0L})); + SimpleStats valueStats = + new SimpleStats( + singleColumn("min_value"), + singleColumn("max_value"), + fromLongArray(new Long[] {0L})); + DataFileMeta dataFile = + DataFileMeta.create( + "my_file", + 1024 * 1024, + 1024, + singleColumn("min_key"), + singleColumn("max_key"), + keyStats, + valueStats, + 15, + 200, + 5, + 3, + Arrays.asList("extra1", "extra2"), + Timestamp.fromLocalDateTime(LocalDateTime.parse("2022-03-02T20:20:12")), + 11L, + new byte[] {1, 2, 4}, + FileSource.COMPACT, + Arrays.asList("field1", "field2", "field3"), + "hdfs://localhost:9000/path/to/file", + 1L, + Arrays.asList("asdf", "qwer", "zxcv")); + List dataFiles = Collections.singletonList(dataFile); + GlobalIndexMeta globalIndexMeta = + new GlobalIndexMeta(1L, 2L, 3, new int[] {5, 6, 7}, new byte[] {0x23, 0x45}); + IndexFileMeta hashIndexFile = + new IndexFileMeta( + "my_index_type", + "my_index_file", + 1024 * 100, + 1002, + null, + null, + globalIndexMeta); + + LinkedHashMap dvRanges = new LinkedHashMap<>(); + dvRanges.put("dv_key1", new DeletionVectorMeta("dv_key1", 1, 2, 3L)); + dvRanges.put("dv_key2", new DeletionVectorMeta("dv_key2", 3, 4, 5L)); + IndexFileMeta devIndexFile = + new IndexFileMeta( + "my_index_type", + "my_index_file", + 1024 * 100, + 1002, + dvRanges, + "external_path"); + + CommitMessageImpl commitMessage = + new CommitMessageImpl( + singleColumn("my_partition"), + 11, + 16, + new DataIncrement( + dataFiles, + dataFiles, + dataFiles, + Collections.singletonList(hashIndexFile), + Collections.singletonList(hashIndexFile)), + new CompactIncrement( + dataFiles, + dataFiles, + dataFiles, + Collections.singletonList(devIndexFile), + Collections.emptyList())); + + ManifestCommittable manifestCommittable = + new ManifestCommittable(5, 202020L, Collections.singletonList(commitMessage)); + manifestCommittable.addProperty("k1", "v1"); + manifestCommittable.addProperty("k2", "v2"); + + ManifestCommittableSerializer serializer = new ManifestCommittableSerializer(); + byte[] bytes = serializer.serialize(manifestCommittable); + ManifestCommittable deserialized = serializer.deserialize(serializer.getVersion(), bytes); + assertThat(deserialized).isEqualTo(manifestCommittable); + + byte[] oldBytes = + IOUtils.readFully( + ManifestCommittableSerializerCompatibilityTest.class + .getClassLoader() + .getResourceAsStream("compatibility/" + fileName), + true); + deserialized = serializer.deserialize(5, oldBytes); + assertThat(deserialized).isEqualTo(manifestCommittable); + } + @Test public void testCompatibilityToV4CommitV11() throws IOException { String fileName = "manifest-committable-v11"; @@ -125,11 +224,7 @@ public void testCompatibilityToV4CommitV11() throws IOException { Collections.emptyList())); ManifestCommittable manifestCommittable = - new ManifestCommittable( - 5, - 202020L, - Collections.singletonMap(5, 555L), - Collections.singletonList(commitMessage)); + new ManifestCommittable(5, 202020L, Collections.singletonList(commitMessage)); manifestCommittable.addProperty("k1", "v1"); manifestCommittable.addProperty("k2", "v2"); @@ -219,11 +314,7 @@ public void testCompatibilityToV4CommitV10() throws IOException { Collections.emptyList())); ManifestCommittable manifestCommittable = - new ManifestCommittable( - 5, - 202020L, - Collections.singletonMap(5, 555L), - Collections.singletonList(commitMessage)); + new ManifestCommittable(5, 202020L, Collections.singletonList(commitMessage)); manifestCommittable.addProperty("k1", "v1"); manifestCommittable.addProperty("k2", "v2"); @@ -305,11 +396,8 @@ public void testCompatibilityToV4CommitV9() throws IOException { Collections.emptyList())); ManifestCommittable manifestCommittable = - new ManifestCommittable( - 5, - 202020L, - Collections.singletonMap(5, 555L), - Collections.singletonList(commitMessage)); + new ManifestCommittable(5, 202020L, Collections.singletonList(commitMessage)); + manifestCommittable.addProperty("k1", "v1"); manifestCommittable.addProperty("k2", "v2"); @@ -387,11 +475,8 @@ public void testCompatibilityToV4CommitV8() throws IOException { Collections.emptyList())); ManifestCommittable manifestCommittable = - new ManifestCommittable( - 5, - 202020L, - Collections.singletonMap(5, 555L), - Collections.singletonList(commitMessage)); + new ManifestCommittable(5, 202020L, Collections.singletonList(commitMessage)); + manifestCommittable.addProperty("k1", "v1"); manifestCommittable.addProperty("k2", "v2"); @@ -469,11 +554,8 @@ public void testCompatibilityToV4CommitV7() throws IOException { Collections.emptyList())); ManifestCommittable manifestCommittable = - new ManifestCommittable( - 5, - 202020L, - Collections.singletonMap(5, 555L), - Collections.singletonList(commitMessage)); + new ManifestCommittable(5, 202020L, Collections.singletonList(commitMessage)); + manifestCommittable.addProperty("k1", "v1"); manifestCommittable.addProperty("k2", "v2"); @@ -550,11 +632,7 @@ public void testCompatibilityToV3CommitV7() throws IOException { Collections.emptyList())); ManifestCommittable manifestCommittable = - new ManifestCommittable( - 5, - 202020L, - Collections.singletonMap(5, 555L), - Collections.singletonList(commitMessage)); + new ManifestCommittable(5, 202020L, Collections.singletonList(commitMessage)); ManifestCommittableSerializer serializer = new ManifestCommittableSerializer(); byte[] bytes = serializer.serialize(manifestCommittable); @@ -629,11 +707,7 @@ public void testCompatibilityToV3CommitV6() throws IOException { Collections.emptyList())); ManifestCommittable manifestCommittable = - new ManifestCommittable( - 5, - 202020L, - Collections.singletonMap(5, 555L), - Collections.singletonList(commitMessage)); + new ManifestCommittable(5, 202020L, Collections.singletonList(commitMessage)); ManifestCommittableSerializer serializer = new ManifestCommittableSerializer(); byte[] bytes = serializer.serialize(manifestCommittable); @@ -708,11 +782,7 @@ public void testCompatibilityToV3CommitV5() throws IOException { Collections.emptyList())); ManifestCommittable manifestCommittable = - new ManifestCommittable( - 5, - 202020L, - Collections.singletonMap(5, 555L), - Collections.singletonList(commitMessage)); + new ManifestCommittable(5, 202020L, Collections.singletonList(commitMessage)); ManifestCommittableSerializer serializer = new ManifestCommittableSerializer(); byte[] bytes = serializer.serialize(manifestCommittable); @@ -786,11 +856,7 @@ public void testCompatibilityToV3CommitV4() throws IOException { Collections.emptyList())); ManifestCommittable manifestCommittable = - new ManifestCommittable( - 5, - 202020L, - Collections.singletonMap(5, 555L), - Collections.singletonList(commitMessage)); + new ManifestCommittable(5, 202020L, Collections.singletonList(commitMessage)); ManifestCommittableSerializer serializer = new ManifestCommittableSerializer(); byte[] bytes = serializer.serialize(manifestCommittable); @@ -865,11 +931,7 @@ public void testCompatibilityToV3CommitV3() throws IOException { Collections.emptyList())); ManifestCommittable manifestCommittable = - new ManifestCommittable( - 5, - 202020L, - Collections.singletonMap(5, 555L), - Collections.singletonList(commitMessage)); + new ManifestCommittable(5, 202020L, Collections.singletonList(commitMessage)); ManifestCommittableSerializer serializer = new ManifestCommittableSerializer(); byte[] bytes = serializer.serialize(manifestCommittable); @@ -944,11 +1006,7 @@ public void testCompatibilityToV2CommitV2() throws IOException { Collections.emptyList())); ManifestCommittable manifestCommittable = - new ManifestCommittable( - 5, - 202020L, - Collections.singletonMap(5, 555L), - Collections.singletonList(commitMessage)); + new ManifestCommittable(5, 202020L, Collections.singletonList(commitMessage)); ManifestCommittableSerializer serializer = new ManifestCommittableSerializer(); byte[] bytes = serializer.serialize(manifestCommittable); @@ -1019,11 +1077,7 @@ public void testCompatibilityToVersion2PaimonV07() throws IOException { Collections.emptyList())); ManifestCommittable manifestCommittable = - new ManifestCommittable( - 5, - 202020L, - Collections.singletonMap(5, 555L), - Collections.singletonList(commitMessage)); + new ManifestCommittable(5, 202020L, Collections.singletonList(commitMessage)); ManifestCommittableSerializer serializer = new ManifestCommittableSerializer(); byte[] bytes = serializer.serialize(manifestCommittable); diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java index 4ff378a5ac52..a59013229c10 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java @@ -78,12 +78,6 @@ private static void addFileCommittables( partition, bucket, totalBuckets, dataIncrement, compactIncrement); committable.addFileCommittable(commitMessage); } - - if (!committable.logOffsets().containsKey(bucket)) { - int offset = ID.incrementAndGet(); - committable.addLogOffset(bucket, offset, false); - assertThat(committable.logOffsets().get(bucket)).isEqualTo(offset); - } } public static DataIncrement randomNewFilesIncrement() { diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java index 984424610d47..26c10d6d2e76 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java @@ -929,7 +929,6 @@ private void cleanBucket(TestFileStore store, BinaryRow partition, int bucket) { commitIdentifier++, null, Collections.emptyMap(), - Collections.emptyMap(), Snapshot.CommitKind.APPEND, store.snapshotManager().latestSnapshot(), mustConflictCheck(), diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java index ec17d7e4c671..19a1e3414d26 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java @@ -445,85 +445,42 @@ public void testOverwritePartialCommit() throws Exception { assertThat(actual).isEqualTo(expected); } - @Test - public void testSnapshotAddLogOffset() throws Exception { - TestFileStore store = createStore(false, 2); - - // commit 1 - Map offsets = new HashMap<>(); - offsets.put(0, 1L); - offsets.put(1, 3L); - Snapshot snapshot = - store.commitData(generateDataList(10), gen::getPartition, kv -> 0, offsets).get(0); - assertThat(snapshot.logOffsets()).isEqualTo(offsets); - - // commit 2 - offsets = new HashMap<>(); - offsets.put(1, 8L); - snapshot = - store.commitData(generateDataList(10), gen::getPartition, kv -> 0, offsets).get(0); - Map expected = new HashMap<>(); - expected.put(0, 1L); - expected.put(1, 8L); - assertThat(snapshot.logOffsets()).isEqualTo(expected); - } - @Test public void testSnapshotRecordCount() throws Exception { TestFileStore store = createStore(false); // commit 1 Snapshot snapshot1 = - store.commitData( - generateDataList(10), - gen::getPartition, - kv -> 0, - Collections.emptyMap()) - .get(0); + store.commitData(generateDataList(10), gen::getPartition, kv -> 0).get(0); long deltaRecordCount1 = snapshot1.deltaRecordCount(); assertThat(deltaRecordCount1).isNotEqualTo(0L); assertThat(snapshot1.totalRecordCount()).isEqualTo(deltaRecordCount1); - assertThat(snapshot1.changelogRecordCount()).isEqualTo(0L); + assertThat(snapshot1.changelogRecordCount()).isNull(); // commit 2 Snapshot snapshot2 = - store.commitData( - generateDataList(20), - gen::getPartition, - kv -> 0, - Collections.emptyMap()) - .get(0); + store.commitData(generateDataList(20), gen::getPartition, kv -> 0).get(0); long deltaRecordCount2 = snapshot2.deltaRecordCount(); assertThat(deltaRecordCount2).isNotEqualTo(0L); assertThat(snapshot2.totalRecordCount()) .isEqualTo(snapshot1.totalRecordCount() + deltaRecordCount2); - assertThat(snapshot2.changelogRecordCount()).isEqualTo(0L); + assertThat(snapshot2.changelogRecordCount()).isNull(); // commit 3 Snapshot snapshot3 = - store.commitData( - generateDataList(30), - gen::getPartition, - kv -> 0, - Collections.emptyMap()) - .get(0); + store.commitData(generateDataList(30), gen::getPartition, kv -> 0).get(0); long deltaRecordCount3 = snapshot3.deltaRecordCount(); assertThat(deltaRecordCount3).isNotEqualTo(0L); assertThat(snapshot3.totalRecordCount()) .isEqualTo(snapshot2.totalRecordCount() + deltaRecordCount3); - assertThat(snapshot3.changelogRecordCount()).isEqualTo(0L); + assertThat(snapshot3.changelogRecordCount()).isNull(); } @Test public void testCommitEmpty() throws Exception { TestFileStore store = createStore(false, 2); Snapshot snapshot = - store.commitData( - generateDataList(10), - gen::getPartition, - kv -> 0, - Collections.emptyMap()) - .get(0); + store.commitData(generateDataList(10), gen::getPartition, kv -> 0).get(0); // not commit empty new files store.commitDataImpl( @@ -634,8 +591,7 @@ public void testDropPartitions() throws Exception { store.commitData( data.values().stream().flatMap(Collection::stream).collect(Collectors.toList()), gen::getPartition, - kv -> 0, - Collections.singletonMap(0, 1L)); + kv -> 0); // generate partitions to be dropped ThreadLocalRandom random = ThreadLocalRandom.current(); @@ -820,7 +776,7 @@ public void testWriteStats() throws Exception { TestFileStore store = createStore(false, 1, CoreOptions.ChangelogProducer.NONE); StatsFileHandler statsFileHandler = store.newStatsFileHandler(); FileStoreCommitImpl fileStoreCommit = store.newCommit(); - store.commitData(generateDataList(10), gen::getPartition, kv -> 0, Collections.emptyMap()); + store.commitData(generateDataList(10), gen::getPartition, kv -> 0); Snapshot latestSnapshot = store.snapshotManager().latestSnapshot(); // Analyze and check @@ -839,7 +795,7 @@ public void testWriteStats() throws Exception { assertThat(readStats.get()).isEqualTo(fakeStats); // New snapshot will inherit last snapshot's stats - store.commitData(generateDataList(10), gen::getPartition, kv -> 0, Collections.emptyMap()); + store.commitData(generateDataList(10), gen::getPartition, kv -> 0); readStats = statsFileHandler.readStats(); assertThat(readStats).isPresent(); assertThat(readStats.get()).isEqualTo(fakeStats); @@ -849,7 +805,7 @@ public void testWriteStats() throws Exception { new ArrayList<>(TestKeyValueGenerator.DEFAULT_ROW_TYPE.getFields()); newFields.add(new DataField(-1, "newField", DataTypes.INT())); store.mergeSchema(new RowType(false, newFields), true); - store.commitData(generateDataList(10), gen::getPartition, kv -> 0, Collections.emptyMap()); + store.commitData(generateDataList(10), gen::getPartition, kv -> 0); readStats = statsFileHandler.readStats(); assertThat(readStats).isEmpty(); @@ -937,8 +893,7 @@ public void testManifestCompact() throws Exception { List keyValues = generateDataList(1); BinaryRow partition = gen.getPartition(keyValues.get(0)); // commit 1 - Snapshot snapshot1 = - store.commitData(keyValues, s -> partition, kv -> 0, Collections.emptyMap()).get(0); + Snapshot snapshot1 = store.commitData(keyValues, s -> partition, kv -> 0).get(0); // commit 2 Snapshot snapshot2 = store.overwriteData(keyValues, s -> partition, kv -> 0, Collections.emptyMap()) @@ -970,8 +925,7 @@ public void testDropStatsForOverwrite() throws Exception { List keyValues = generateDataList(1); BinaryRow partition = gen.getPartition(keyValues.get(0)); // commit 1 - Snapshot snapshot1 = - store.commitData(keyValues, s -> partition, kv -> 0, Collections.emptyMap()).get(0); + Snapshot snapshot1 = store.commitData(keyValues, s -> partition, kv -> 0).get(0); // overwrite commit 2 Snapshot snapshot2 = store.overwriteData(keyValues, s -> partition, kv -> 0, Collections.emptyMap()) @@ -1001,8 +955,7 @@ public void testManifestCompactFull() throws Exception { List keyValues = generateDataList(1); BinaryRow partition = gen.getPartition(keyValues.get(0)); // commit 1 - Snapshot snapshot = - store.commitData(keyValues, s -> partition, kv -> 0, Collections.emptyMap()).get(0); + Snapshot snapshot = store.commitData(keyValues, s -> partition, kv -> 0).get(0); for (int i = 0; i < 100; i++) { snapshot = @@ -1066,7 +1019,6 @@ public void testCommitTwiceWithDifferentKind() throws Exception { 0, null, Collections.emptyMap(), - Collections.emptyMap(), Snapshot.CommitKind.APPEND, firstLatest, mustConflictCheck(), @@ -1080,7 +1032,6 @@ public void testCommitTwiceWithDifferentKind() throws Exception { 0, null, Collections.emptyMap(), - Collections.emptyMap(), Snapshot.CommitKind.COMPACT, store.snapshotManager().latestSnapshot(), mustConflictCheck(), diff --git a/paimon-core/src/test/java/org/apache/paimon/table/IncrementalTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/IncrementalTableTest.java index 2bc33cb7b327..4c218bd32a04 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/IncrementalTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/IncrementalTableTest.java @@ -42,7 +42,6 @@ import org.junit.jupiter.api.Test; import java.time.LocalDateTime; -import java.util.Collections; import java.util.List; import static org.apache.paimon.CoreOptions.INCREMENTAL_BETWEEN; @@ -367,30 +366,15 @@ public void testIncrementalToAutoTag() throws Exception { write.write(GenericRow.of(1, BinaryString.fromString("a"))); List commitMessages = write.prepareCommit(false, 0); - commit.commit( - new ManifestCommittable( - 0, - utcMills("2024-12-02T10:00:00"), - Collections.emptyMap(), - commitMessages)); + commit.commit(new ManifestCommittable(0, utcMills("2024-12-02T10:00:00"), commitMessages)); write.write(GenericRow.of(2, BinaryString.fromString("b"))); commitMessages = write.prepareCommit(false, 1); - commit.commit( - new ManifestCommittable( - 1, - utcMills("2024-12-03T10:00:00"), - Collections.emptyMap(), - commitMessages)); + commit.commit(new ManifestCommittable(1, utcMills("2024-12-03T10:00:00"), commitMessages)); write.write(GenericRow.of(3, BinaryString.fromString("c"))); commitMessages = write.prepareCommit(false, 2); - commit.commit( - new ManifestCommittable( - 2, - utcMills("2024-12-05T10:00:00"), - Collections.emptyMap(), - commitMessages)); + commit.commit(new ManifestCommittable(2, utcMills("2024-12-05T10:00:00"), commitMessages)); assertIncrementalToAutoTag(table); } diff --git a/paimon-core/src/test/java/org/apache/paimon/tag/TagAutoManagerTest.java b/paimon-core/src/test/java/org/apache/paimon/tag/TagAutoManagerTest.java index 306820fa7d99..c8954f7b31aa 100644 --- a/paimon-core/src/test/java/org/apache/paimon/tag/TagAutoManagerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/tag/TagAutoManagerTest.java @@ -403,9 +403,8 @@ public void testExpireTagsByTimeRetained() throws Exception { 0L, Snapshot.CommitKind.APPEND, 1000, - null, - null, - null, + 0L, + 0L, null, null, null, @@ -433,9 +432,8 @@ public void testExpireTagsByTimeRetained() throws Exception { 0L, Snapshot.CommitKind.APPEND, 1000, - null, - null, - null, + 0L, + 0L, null, null, null, diff --git a/paimon-core/src/test/java/org/apache/paimon/tag/TagTest.java b/paimon-core/src/test/java/org/apache/paimon/tag/TagTest.java index 27b3a575b668..d7dbeacb7f57 100644 --- a/paimon-core/src/test/java/org/apache/paimon/tag/TagTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/tag/TagTest.java @@ -45,9 +45,8 @@ public class TagTest { 0L, Snapshot.CommitKind.APPEND, 1000, - null, - null, - null, + 0L, + 0L, null, null, null, @@ -64,13 +63,12 @@ public void testFromJson() { + " \"schemaId\" : 0,\n" + " \"baseManifestList\" : null,\n" + " \"deltaManifestList\" : null,\n" - + " \"changelogManifestList\" : null,\n" + " \"commitUser\" : null,\n" + " \"commitIdentifier\" : 0,\n" + " \"commitKind\" : \"APPEND\",\n" + " \"timeMillis\" : 1000,\n" - + " \"totalRecordCount\" : null,\n" - + " \"deltaRecordCount\" : null\n" + + " \"totalRecordCount\" : 0,\n" + + " \"deltaRecordCount\" : 0\n" + "}", tag.toJson()); } @@ -90,13 +88,12 @@ public void testFromSnapshotAndTagTtl() { + " \"schemaId\" : 0,\n" + " \"baseManifestList\" : null,\n" + " \"deltaManifestList\" : null,\n" - + " \"changelogManifestList\" : null,\n" + " \"commitUser\" : null,\n" + " \"commitIdentifier\" : 0,\n" + " \"commitKind\" : \"APPEND\",\n" + " \"timeMillis\" : 1000,\n" - + " \"totalRecordCount\" : null,\n" - + " \"deltaRecordCount\" : null,\n" + + " \"totalRecordCount\" : 0,\n" + + " \"deltaRecordCount\" : 0,\n" + " \"tagCreateTime\" : [ 1969, 1, 1, 0, 0, 0, 123456789 ],\n" + " \"tagTimeRetained\" : 5.000000000\n" + "}", diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/SnapshotManagerTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/SnapshotManagerTest.java index 6b659db370b9..19ae86569246 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/SnapshotManagerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/SnapshotManagerTest.java @@ -270,9 +270,8 @@ public static Snapshot createSnapshotWithMillis(long id, long millis) { 0L, Snapshot.CommitKind.APPEND, millis, - null, - null, - null, + 0L, + 0L, null, null, null, @@ -295,9 +294,8 @@ private Snapshot createSnapshotWithMillis(long id, long millis, long watermark) 0L, Snapshot.CommitKind.APPEND, millis, - null, - null, - null, + 0L, + 0L, null, watermark, null, @@ -321,9 +319,8 @@ private Changelog createChangelogWithMillis(long id, long millis) { 0L, Snapshot.CommitKind.APPEND, millis, - null, - null, - null, + 0L, + 0L, null, null, null, @@ -353,9 +350,8 @@ public void testLatestSnapshotOfUser() throws IOException, InterruptedException 0L, Snapshot.CommitKind.APPEND, i * 1000, - null, - null, - null, + 0L, + 0L, null, null, null, @@ -407,9 +403,8 @@ public void testTraversalSnapshotsFromLatestSafely() throws IOException, Interru 0L, Snapshot.CommitKind.APPEND, i * 1000, - null, - null, - null, + 0L, + 0L, null, null, null, diff --git a/paimon-core/src/test/resources/compatibility/manifest-committable-v11-v5 b/paimon-core/src/test/resources/compatibility/manifest-committable-v11-v5 new file mode 100644 index 0000000000000000000000000000000000000000..0ff85b2662c8d41d0cbc55b0a268de5c90eaa9b2 GIT binary patch literal 4130 zcmeHK%}&BV5MB#v3>rOX!jY&4Zw0)lCoU$&c<>FNp(QkwAE6?6(Wrr|kKrSD^ytx} z&)~^N@cVYV1rZHiOiXr?uU~g&x}9#P(^(;;%qvQ4Y~#cdRxKy2E>O#Z^U$S$Ie>Pu zlw};(fO@~!a@wIE`pt&2h!mY@16zBMl8zNW@P^6+ZBkj*K@R}=2`B-tz%{Uk8CJoR zXGJ0kS0tiw8vF>q(#i436`0Xd<}7)GTd1UDaD2gF6Z{Sy*QYJ%qE({qH;Of{k60)4 zPH$W|K+ofJ+zK>zXBS?ZeAz2^$pAt!`Ga+sDIX`_AFrdB{88&*>d(*h%;e9s z`h#CnR^`tMr~r^p0dcQZ>`fOk>c)h%x&$Ea{D$lGiebOS(ie|Mr4ZUtG1qlhPs{%s zL)bc`m$=dTc?=eSMQW8Wpnn#xsqcJsv-BJ0`<(WDn*1^OWBNO?h587Y%37`G)L# s.split(",")[0], "A, 10", "B, 2", "C, 30", "D, 40"); - - // prepare second part of test data - sendKafkaMessage("2.csv", "A,100\nD,400", testTopicName); - - // check that we can receive data from log store quickly - checkResult(s -> s.split(",")[0], "A, 100", "B, 2", "C, 30", "D, 400"); - } -} diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java index 054e48a207d7..73af189bb3ba 100644 --- a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java +++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java @@ -18,22 +18,16 @@ package org.apache.paimon.flink.sink; -import org.apache.paimon.flink.log.LogStoreTableFactory; import org.apache.paimon.table.Table; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.factories.DynamicTableFactory; -import javax.annotation.Nullable; - /** Table sink to create sink. */ public class FlinkTableSink extends FlinkTableSinkBase { public FlinkTableSink( - ObjectIdentifier tableIdentifier, - Table table, - DynamicTableFactory.Context context, - @Nullable LogStoreTableFactory logStoreTableFactory) { - super(tableIdentifier, table, context, logStoreTableFactory); + ObjectIdentifier tableIdentifier, Table table, DynamicTableFactory.Context context) { + super(tableIdentifier, table, context); } } diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java index a12093f4f6ee..70862d0e4e2f 100644 --- a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java +++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java @@ -18,22 +18,16 @@ package org.apache.paimon.flink.sink; -import org.apache.paimon.flink.log.LogStoreTableFactory; import org.apache.paimon.table.Table; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.factories.DynamicTableFactory; -import javax.annotation.Nullable; - /** Table sink to create sink. */ public class FlinkTableSink extends SupportsRowLevelOperationFlinkTableSink { public FlinkTableSink( - ObjectIdentifier tableIdentifier, - Table table, - DynamicTableFactory.Context context, - @Nullable LogStoreTableFactory logStoreTableFactory) { - super(tableIdentifier, table, context, logStoreTableFactory); + ObjectIdentifier tableIdentifier, Table table, DynamicTableFactory.Context context) { + super(tableIdentifier, table, context); } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogDeserializationSchema.java deleted file mode 100644 index 22ae1dd86583..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogDeserializationSchema.java +++ /dev/null @@ -1,169 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.flink.ProjectedRowData; -import org.apache.paimon.flink.Projection; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.Collector; -import org.apache.kafka.clients.consumer.ConsumerRecord; - -import javax.annotation.Nullable; - -import java.util.Objects; -import java.util.stream.IntStream; - -/** A {@link KafkaDeserializationSchema} for the table with primary key in log store. */ -public class KafkaLogDeserializationSchema implements KafkaDeserializationSchema { - - private static final long serialVersionUID = 1L; - - private final TypeInformation producedType; - private final int fieldCount; - private final int[] primaryKey; - @Nullable private final DeserializationSchema primaryKeyDeserializer; - private final DeserializationSchema valueDeserializer; - private final RowData.FieldGetter[] keyFieldGetters; - @Nullable private final int[][] projectFields; - - private transient ProjectCollector projectCollector; - - public KafkaLogDeserializationSchema( - DataType physicalType, - int[] primaryKey, - @Nullable DeserializationSchema primaryKeyDeserializer, - DeserializationSchema valueDeserializer, - @Nullable int[][] projectFields) { - this.primaryKey = primaryKey; - this.primaryKeyDeserializer = primaryKeyDeserializer; - this.valueDeserializer = valueDeserializer; - RowType logicalType = (RowType) physicalType.getLogicalType(); - this.producedType = - InternalTypeInfo.of( - projectFields == null - ? logicalType - : Projection.of(projectFields).project(logicalType)); - this.fieldCount = physicalType.getChildren().size(); - this.projectFields = projectFields; - this.keyFieldGetters = - IntStream.range(0, primaryKey.length) - .mapToObj( - i -> - createNullCheckingFieldGetter( - physicalType - .getChildren() - .get(primaryKey[i]) - .getLogicalType(), - i)) - .toArray(RowData.FieldGetter[]::new); - } - - private static RowData.FieldGetter createNullCheckingFieldGetter( - LogicalType dataType, int index) { - RowData.FieldGetter getter = RowData.createFieldGetter(dataType, index); - if (dataType.isNullable()) { - return getter; - } else { - return row -> { - if (row.isNullAt(index)) { - return null; - } - return getter.getFieldOrNull(row); - }; - } - } - - @Override - public void open(DeserializationSchema.InitializationContext context) throws Exception { - if (primaryKeyDeserializer != null) { - primaryKeyDeserializer.open(context); - } - valueDeserializer.open(context); - projectCollector = new ProjectCollector(); - } - - @Override - public boolean isEndOfStream(RowData nextElement) { - return false; - } - - @Override - public RowData deserialize(ConsumerRecord record) { - throw new RuntimeException( - "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); - } - - @Override - public void deserialize( - ConsumerRecord record, Collector underCollector) - throws Exception { - Collector collector = projectCollector.project(underCollector); - - if (primaryKey.length > 0 && record.value() == null) { - RowData key = Objects.requireNonNull(primaryKeyDeserializer).deserialize(record.key()); - GenericRowData value = new GenericRowData(RowKind.DELETE, fieldCount); - for (int i = 0; i < primaryKey.length; i++) { - value.setField(primaryKey[i], keyFieldGetters[i].getFieldOrNull(key)); - } - collector.collect(value); - } else { - valueDeserializer.deserialize(record.value(), collector); - } - } - - @Override - public TypeInformation getProducedType() { - return producedType; - } - - private class ProjectCollector implements Collector { - - private final ProjectedRowData projectedRow = - projectFields == null ? null : ProjectedRowData.from(projectFields); - - private Collector underCollector; - - private Collector project(Collector underCollector) { - if (projectedRow == null) { - return underCollector; - } - - this.underCollector = underCollector; - return this; - } - - @Override - public void collect(RowData rowData) { - underCollector.collect(projectedRow.replaceRow(rowData)); - } - - @Override - public void close() {} - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogOptions.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogOptions.java deleted file mode 100644 index e76a7a9129fa..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogOptions.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.options.ConfigOption; -import org.apache.paimon.options.ConfigOptions; - -/** Options for kafka log. */ -public class KafkaLogOptions { - - public static final ConfigOption BOOTSTRAP_SERVERS = - ConfigOptions.key("kafka.bootstrap.servers") - .stringType() - .noDefaultValue() - .withDescription("Required Kafka server connection string."); - - public static final ConfigOption TOPIC = - ConfigOptions.key("kafka.topic") - .stringType() - .noDefaultValue() - .withDescription("Topic of this kafka table."); -} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogSerializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogSerializationSchema.java deleted file mode 100644 index 1fd80d356e55..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogSerializationSchema.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.CoreOptions.LogChangelogMode; -import org.apache.paimon.flink.FlinkRowData; -import org.apache.paimon.table.sink.SinkRecord; -import org.apache.paimon.types.RowKind; - -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema; -import org.apache.flink.streaming.connectors.kafka.KafkaSerializationSchema; -import org.apache.flink.table.data.RowData; -import org.apache.kafka.clients.producer.ProducerRecord; - -import javax.annotation.Nullable; - -/** A {@link KafkaRecordSerializationSchema} for the table in log store. */ -public class KafkaLogSerializationSchema implements KafkaSerializationSchema { - - private static final long serialVersionUID = 1L; - - private final String topic; - @Nullable private final SerializationSchema primaryKeySerializer; - private final SerializationSchema valueSerializer; - private final LogChangelogMode changelogMode; - - public KafkaLogSerializationSchema( - String topic, - @Nullable SerializationSchema primaryKeySerializer, - SerializationSchema valueSerializer, - LogChangelogMode changelogMode) { - this.topic = topic; - this.primaryKeySerializer = primaryKeySerializer; - this.valueSerializer = valueSerializer; - this.changelogMode = changelogMode; - if (changelogMode == LogChangelogMode.UPSERT && primaryKeySerializer == null) { - throw new IllegalArgumentException( - "Can not use upsert changelog mode for non-pk table."); - } - } - - @Override - public void open(SerializationSchema.InitializationContext context) throws Exception { - if (primaryKeySerializer != null) { - primaryKeySerializer.open(context); - } - valueSerializer.open(context); - } - - @Override - public ProducerRecord serialize(SinkRecord element, @Nullable Long timestamp) { - RowKind kind = element.row().getRowKind(); - - byte[] primaryKeyBytes = null; - byte[] valueBytes = null; - if (primaryKeySerializer != null) { - primaryKeyBytes = - primaryKeySerializer.serialize(new FlinkRowData(element.primaryKey())); - if (changelogMode == LogChangelogMode.ALL - || kind == RowKind.INSERT - || kind == RowKind.UPDATE_AFTER) { - valueBytes = valueSerializer.serialize(new FlinkRowData(element.row())); - } - } else { - valueBytes = valueSerializer.serialize(new FlinkRowData(element.row())); - } - Integer partition = element.bucket() < 0 ? null : element.bucket(); - return new ProducerRecord<>(topic, partition, primaryKeyBytes, valueBytes); - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogSinkProvider.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogSinkProvider.java deleted file mode 100644 index 09c9c823fd06..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogSinkProvider.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.CoreOptions.LogChangelogMode; -import org.apache.paimon.CoreOptions.LogConsistency; -import org.apache.paimon.annotation.VisibleForTesting; -import org.apache.paimon.flink.log.LogSinkProvider; -import org.apache.paimon.flink.sink.LogSinkFunction; - -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.Semantic; -import org.apache.flink.table.data.RowData; - -import javax.annotation.Nullable; - -import java.util.Properties; - -/** A Kafka {@link LogSinkProvider}. */ -public class KafkaLogSinkProvider implements LogSinkProvider { - - private static final long serialVersionUID = 1L; - - private final String topic; - - private final Properties properties; - - @Nullable private final SerializationSchema primaryKeySerializer; - - private final SerializationSchema valueSerializer; - - private final LogConsistency consistency; - - private final LogChangelogMode changelogMode; - - public KafkaLogSinkProvider( - String topic, - Properties properties, - @Nullable SerializationSchema primaryKeySerializer, - SerializationSchema valueSerializer, - LogConsistency consistency, - LogChangelogMode changelogMode) { - this.topic = topic; - this.properties = properties; - this.primaryKeySerializer = primaryKeySerializer; - this.valueSerializer = valueSerializer; - this.consistency = consistency; - this.changelogMode = changelogMode; - } - - @Override - public LogSinkFunction createSink() { - Semantic semantic; - switch (consistency) { - case TRANSACTIONAL: - semantic = Semantic.EXACTLY_ONCE; - break; - case EVENTUAL: - semantic = Semantic.AT_LEAST_ONCE; - break; - default: - throw new IllegalArgumentException("Unsupported: " + consistency); - } - return new KafkaSinkFunction(topic, createSerializationSchema(), properties, semantic); - } - - @VisibleForTesting - KafkaLogSerializationSchema createSerializationSchema() { - return new KafkaLogSerializationSchema( - topic, primaryKeySerializer, valueSerializer, changelogMode); - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogSourceProvider.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogSourceProvider.java deleted file mode 100644 index 40d5f06385a8..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogSourceProvider.java +++ /dev/null @@ -1,153 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.CoreOptions.LogConsistency; -import org.apache.paimon.CoreOptions.StartupMode; -import org.apache.paimon.annotation.VisibleForTesting; -import org.apache.paimon.flink.log.LogSourceProvider; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.connector.kafka.source.KafkaSource; -import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; -import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.kafka.common.TopicPartition; - -import javax.annotation.Nullable; - -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; -import java.util.UUID; - -import static org.apache.kafka.clients.consumer.ConsumerConfig.ISOLATION_LEVEL_CONFIG; - -/** A Kafka {@link LogSourceProvider}. */ -public class KafkaLogSourceProvider implements LogSourceProvider { - - private static final long serialVersionUID = 1L; - - private final String topic; - - private final Properties properties; - - private final DataType physicalType; - - private final int[] primaryKey; - - @Nullable private final DeserializationSchema primaryKeyDeserializer; - - private final DeserializationSchema valueDeserializer; - - @Nullable private final int[][] projectFields; - - private final LogConsistency consistency; - - private final StartupMode scanMode; - - @Nullable private final Long timestampMills; - - public KafkaLogSourceProvider( - String topic, - Properties properties, - DataType physicalType, - int[] primaryKey, - @Nullable DeserializationSchema primaryKeyDeserializer, - DeserializationSchema valueDeserializer, - @Nullable int[][] projectFields, - LogConsistency consistency, - StartupMode scanMode, - @Nullable Long timestampMills) { - this.topic = topic; - this.properties = properties; - this.physicalType = physicalType; - this.primaryKey = primaryKey; - this.primaryKeyDeserializer = primaryKeyDeserializer; - this.valueDeserializer = valueDeserializer; - this.projectFields = projectFields; - this.consistency = consistency; - this.scanMode = scanMode; - this.timestampMills = timestampMills; - } - - @Override - public void preCreateSource() { - // nothing to do before log source creating - } - - @Override - public KafkaSource createSource(@Nullable Map bucketOffsets) { - switch (consistency) { - case TRANSACTIONAL: - // Add read committed for transactional consistency mode. - properties.setProperty(ISOLATION_LEVEL_CONFIG, "read_committed"); - break; - case EVENTUAL: - properties.setProperty(ISOLATION_LEVEL_CONFIG, "read_uncommitted"); - break; - } - - return KafkaSource.builder() - .setTopics(topic) - .setStartingOffsets(toOffsetsInitializer(bucketOffsets)) - .setDeserializer(createDeserializationSchema()) - .setGroupId(UUID.randomUUID().toString()) - .setProperties(properties) - .build(); - } - - @VisibleForTesting - KafkaRecordDeserializationSchema createDeserializationSchema() { - return KafkaRecordDeserializationSchema.of( - new KafkaLogDeserializationSchema( - physicalType, - primaryKey, - primaryKeyDeserializer, - valueDeserializer, - projectFields)); - } - - private OffsetsInitializer toOffsetsInitializer(@Nullable Map bucketOffsets) { - switch (scanMode) { - case LATEST_FULL: - return bucketOffsets == null - ? OffsetsInitializer.earliest() - : OffsetsInitializer.offsets(toKafkaOffsets(bucketOffsets)); - case LATEST: - return OffsetsInitializer.latest(); - case FROM_TIMESTAMP: - if (timestampMills == null) { - throw new NullPointerException( - "Must specify a timestamp if you choose timestamp startup mode."); - } - return OffsetsInitializer.timestamp(timestampMills); - default: - throw new UnsupportedOperationException("Unsupported mode: " + scanMode); - } - } - - private Map toKafkaOffsets(Map bucketOffsets) { - Map offsets = new HashMap<>(); - bucketOffsets.forEach( - (bucket, offset) -> offsets.put(new TopicPartition(topic, bucket), offset)); - return offsets; - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactory.java deleted file mode 100644 index f382a8cf18f2..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactory.java +++ /dev/null @@ -1,171 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.CoreOptions; -import org.apache.paimon.flink.factories.FlinkFactoryUtil.FlinkTableFactoryHelper; -import org.apache.paimon.flink.log.LogStoreRegister; -import org.apache.paimon.flink.log.LogStoreTableFactory; -import org.apache.paimon.options.Options; -import org.apache.paimon.utils.DateTimeUtils; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.catalog.ResolvedSchema; -import org.apache.flink.table.catalog.UniqueConstraint; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.DynamicTableFactory.Context; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.utils.DataTypeUtils; - -import javax.annotation.Nullable; - -import java.util.List; -import java.util.Properties; -import java.util.TimeZone; - -import static org.apache.kafka.clients.consumer.ConsumerConfig.ISOLATION_LEVEL_CONFIG; -import static org.apache.paimon.CoreOptions.LOG_CHANGELOG_MODE; -import static org.apache.paimon.CoreOptions.LOG_CONSISTENCY; -import static org.apache.paimon.CoreOptions.LogConsistency; -import static org.apache.paimon.CoreOptions.SCAN_TIMESTAMP; -import static org.apache.paimon.CoreOptions.SCAN_TIMESTAMP_MILLIS; -import static org.apache.paimon.flink.factories.FlinkFactoryUtil.createFlinkTableFactoryHelper; -import static org.apache.paimon.flink.kafka.KafkaLogOptions.TOPIC; -import static org.apache.paimon.options.OptionsUtils.convertToPropertiesPrefixKey; - -/** The Kafka {@link LogStoreTableFactory} implementation. */ -public class KafkaLogStoreFactory implements LogStoreTableFactory { - - public static final String IDENTIFIER = "kafka"; - - public static final String KAFKA_PREFIX = IDENTIFIER + "."; - - @Override - public String identifier() { - return IDENTIFIER; - } - - private String topic(Context context) { - return context.getCatalogTable().getOptions().get(TOPIC.key()); - } - - @Override - public KafkaLogSourceProvider createSourceProvider( - Context context, - DynamicTableSource.Context sourceContext, - @Nullable int[][] projectFields) { - FlinkTableFactoryHelper helper = createFlinkTableFactoryHelper(this, context); - ResolvedSchema schema = context.getCatalogTable().getResolvedSchema(); - DataType physicalType = schema.toPhysicalRowDataType(); - DeserializationSchema primaryKeyDeserializer = null; - int[] primaryKey = getPrimaryKeyIndexes(schema); - if (primaryKey.length > 0) { - DataType keyType = DataTypeUtils.projectRow(physicalType, primaryKey); - primaryKeyDeserializer = - LogStoreTableFactory.getKeyDecodingFormat(helper) - .createRuntimeDecoder(sourceContext, keyType); - } - DeserializationSchema valueDeserializer = - LogStoreTableFactory.getValueDecodingFormat(helper, primaryKey.length != 0) - .createRuntimeDecoder(sourceContext, physicalType); - Options options = toOptions(helper.getOptions()); - Long timestampMills = options.get(SCAN_TIMESTAMP_MILLIS); - String timestampString = options.get(SCAN_TIMESTAMP); - - if (timestampMills == null && timestampString != null) { - timestampMills = - DateTimeUtils.parseTimestampData(timestampString, 3, TimeZone.getDefault()) - .getMillisecond(); - } - return new KafkaLogSourceProvider( - topic(context), - toKafkaProperties(options), - physicalType, - primaryKey, - primaryKeyDeserializer, - valueDeserializer, - projectFields, - options.get(LOG_CONSISTENCY), - // TODO visit all options through CoreOptions - CoreOptions.startupMode(options), - timestampMills); - } - - @Override - public KafkaLogSinkProvider createSinkProvider( - Context context, DynamicTableSink.Context sinkContext) { - FlinkTableFactoryHelper helper = createFlinkTableFactoryHelper(this, context); - ResolvedSchema schema = context.getCatalogTable().getResolvedSchema(); - DataType physicalType = schema.toPhysicalRowDataType(); - SerializationSchema primaryKeySerializer = null; - int[] primaryKey = getPrimaryKeyIndexes(schema); - if (primaryKey.length > 0) { - DataType keyType = DataTypeUtils.projectRow(physicalType, primaryKey); - primaryKeySerializer = - LogStoreTableFactory.getKeyEncodingFormat(helper) - .createRuntimeEncoder(sinkContext, keyType); - } - SerializationSchema valueSerializer = - LogStoreTableFactory.getValueEncodingFormat(helper, primaryKey.length != 0) - .createRuntimeEncoder(sinkContext, physicalType); - Options options = toOptions(helper.getOptions()); - return new KafkaLogSinkProvider( - topic(context), - toKafkaProperties(options), - primaryKeySerializer, - valueSerializer, - options.get(LOG_CONSISTENCY), - options.get(LOG_CHANGELOG_MODE)); - } - - @Override - public LogStoreRegister createRegister(RegisterContext context) { - return new KafkaLogStoreRegister(context); - } - - private int[] getPrimaryKeyIndexes(ResolvedSchema schema) { - final List columns = schema.getColumnNames(); - return schema.getPrimaryKey() - .map(UniqueConstraint::getColumns) - .map(pkColumns -> pkColumns.stream().mapToInt(columns::indexOf).toArray()) - .orElseGet(() -> new int[] {}); - } - - public static Properties toKafkaProperties(Options options) { - Properties properties = new Properties(); - properties.putAll(convertToPropertiesPrefixKey(options.toMap(), KAFKA_PREFIX)); - - // Add read committed for transactional consistency mode. - if (options.get(LOG_CONSISTENCY) == LogConsistency.TRANSACTIONAL) { - properties.setProperty(ISOLATION_LEVEL_CONFIG, "read_committed"); - } - return properties; - } - - private Options toOptions(ReadableConfig config) { - Options options = new Options(); - ((Configuration) config).toMap().forEach(options::setString); - return options; - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogStoreRegister.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogStoreRegister.java deleted file mode 100644 index c93889b66e7b..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogStoreRegister.java +++ /dev/null @@ -1,166 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.flink.log.LogStoreRegister; -import org.apache.paimon.flink.log.LogStoreTableFactory; - -import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; - -import org.apache.kafka.clients.admin.AdminClient; -import org.apache.kafka.clients.admin.NewTopic; - -import java.time.Duration; -import java.util.Collections; -import java.util.Map; -import java.util.Properties; -import java.util.UUID; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -import static org.apache.paimon.CoreOptions.BUCKET; -import static org.apache.paimon.flink.FlinkCatalogOptions.REGISTER_TIMEOUT; -import static org.apache.paimon.flink.FlinkConnectorOptions.LOG_SYSTEM_PARTITIONS; -import static org.apache.paimon.flink.FlinkConnectorOptions.LOG_SYSTEM_REPLICATION; -import static org.apache.paimon.flink.kafka.KafkaLogOptions.BOOTSTRAP_SERVERS; -import static org.apache.paimon.flink.kafka.KafkaLogOptions.TOPIC; -import static org.apache.paimon.flink.kafka.KafkaLogStoreFactory.toKafkaProperties; -import static org.apache.paimon.utils.Preconditions.checkNotNull; - -/** KafkaLogStoreRegister is used to register/unregister topics in Kafka for paimon table. */ -public class KafkaLogStoreRegister implements LogStoreRegister { - - private final String bootstrapServers; - - private final String topic; - - private final int partition; - - private final int replicationFactor; - - private final Duration timeout; - - private final Properties properties; - - private final Identifier identifier; - - public KafkaLogStoreRegister(LogStoreTableFactory.RegisterContext context) { - this.bootstrapServers = context.getOptions().get(BOOTSTRAP_SERVERS); - this.identifier = context.getIdentifier(); - this.topic = - context.getOptions().getOptional(TOPIC).isPresent() - ? context.getOptions().get(TOPIC) - : String.format( - "%s_%s_%s", - this.identifier.getDatabaseName(), - this.identifier.getObjectName(), - UUID.randomUUID().toString().replace("-", "")); - - checkNotNull(context.getOptions().get(BOOTSTRAP_SERVERS)); - checkNotNull(this.topic); - checkNotNull(this.identifier); - - // handle the type information missing when Map is converted to Options - if (context.getOptions().get(REGISTER_TIMEOUT.key()) == null) { - this.timeout = REGISTER_TIMEOUT.defaultValue(); - } else { - this.timeout = Duration.parse(context.getOptions().get(REGISTER_TIMEOUT.key())); - } - - // handle bucket=-1 - int bucketNum = - context.getOptions().get(BUCKET) == -1 ? 1 : context.getOptions().get(BUCKET); - this.partition = - context.getOptions().getOptional(LOG_SYSTEM_PARTITIONS).isPresent() - ? context.getOptions().get(LOG_SYSTEM_PARTITIONS) - : bucketNum; - - this.replicationFactor = context.getOptions().get(LOG_SYSTEM_REPLICATION); - - this.properties = toKafkaProperties(context.getOptions()); - } - - @Override - public Map registerTopic() { - try (AdminClient admin = AdminClient.create(properties)) { - NewTopic newTopic = - new NewTopic(this.topic, this.partition, (short) this.replicationFactor); - - // Since the call is Async, let's wait for it to complete. - admin.createTopics(Collections.singleton(newTopic)) - .all() - .get(this.timeout.getSeconds(), TimeUnit.SECONDS); - } catch (TimeoutException e) { - throw new IllegalStateException( - String.format( - "Register topic for table %s timeout with properties %s", - this.identifier.getFullName(), properties), - e); - } catch (Exception e) { - throw new IllegalStateException( - String.format( - "Register topic for table %s failed with properties %s", - this.identifier.getFullName(), properties), - e); - } - - return ImmutableMap.of( - TOPIC.key(), - this.topic, - LOG_SYSTEM_PARTITIONS.key(), - String.valueOf(this.partition), - LOG_SYSTEM_REPLICATION.key(), - String.valueOf(this.replicationFactor)); - } - - @Override - public void unRegisterTopic() { - try (AdminClient admin = AdminClient.create(properties)) { - admin.deleteTopics(Collections.singleton(this.topic)) - .all() - .get(this.timeout.getSeconds(), TimeUnit.SECONDS); - } catch (ExecutionException e) { - if (e.getCause() - instanceof org.apache.kafka.common.errors.UnknownTopicOrPartitionException) { - // ignore - return; - } else { - throw new IllegalStateException( - String.format( - "Unregister topic for table %s failed with properties %s", - this.identifier.getFullName(), properties), - e); - } - } catch (TimeoutException e) { - throw new RuntimeException( - String.format( - "Unregister topic for table %s timeout with properties %s", - this.identifier.getFullName(), properties), - e); - } catch (Exception e) { - throw new RuntimeException( - String.format( - "Unregister topic for table %s failed with properties %s", - this.identifier.getFullName(), properties), - e); - } - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java deleted file mode 100644 index 41e7141cf48a..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.flink.sink.LogSinkFunction; -import org.apache.paimon.table.sink.SinkRecord; - -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaException; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer; -import org.apache.flink.streaming.connectors.kafka.KafkaSerializationSchema; -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.ProducerRecord; - -import java.util.Properties; - -import static java.util.Objects.requireNonNull; - -/** - * A {@link FlinkKafkaProducer} which implements {@link LogSinkFunction} to register {@link - * WriteCallback}. - */ -public class KafkaSinkFunction extends FlinkKafkaProducer implements LogSinkFunction { - - private WriteCallback writeCallback; - - /** - * Creates a {@link KafkaSinkFunction} for a given topic. The sink produces its input to the - * topic. It accepts a {@link KafkaSerializationSchema} for serializing records to a {@link - * ProducerRecord}, including partitioning information. - * - * @param defaultTopic The default topic to write data to - * @param serializationSchema A serializable serialization schema for turning user objects into - * a kafka-consumable byte[] supporting key/value messages - * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is - * the only required argument. - * @param semantic Defines semantic that will be used by this producer (see {@link - * KafkaSinkFunction.Semantic}). - */ - public KafkaSinkFunction( - String defaultTopic, - KafkaSerializationSchema serializationSchema, - Properties producerConfig, - KafkaSinkFunction.Semantic semantic) { - super(defaultTopic, serializationSchema, producerConfig, semantic); - } - - public void setWriteCallback(WriteCallback writeCallback) { - this.writeCallback = writeCallback; - } - - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); - } - - /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. - */ - public void open(Configuration configuration) throws Exception { - super.open(configuration); - Callback baseCallback = requireNonNull(callback); - callback = - (metadata, exception) -> { - if (writeCallback != null && metadata != null && metadata.hasOffset()) { - writeCallback.onCompletion(metadata.partition(), metadata.offset()); - } - baseCallback.onCompletion(metadata, exception); - }; - } - - @Override - public void flush() throws FlinkKafkaException { - super.preCommit(super.currentTransaction()); - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java index a12093f4f6ee..70862d0e4e2f 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java @@ -18,22 +18,16 @@ package org.apache.paimon.flink.sink; -import org.apache.paimon.flink.log.LogStoreTableFactory; import org.apache.paimon.table.Table; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.factories.DynamicTableFactory; -import javax.annotation.Nullable; - /** Table sink to create sink. */ public class FlinkTableSink extends SupportsRowLevelOperationFlinkTableSink { public FlinkTableSink( - ObjectIdentifier tableIdentifier, - Table table, - DynamicTableFactory.Context context, - @Nullable LogStoreTableFactory logStoreTableFactory) { - super(tableIdentifier, table, context, logStoreTableFactory); + ObjectIdentifier tableIdentifier, Table table, DynamicTableFactory.Context context) { + super(tableIdentifier, table, context); } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java index 8dc225bfd951..3462173dbfec 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java @@ -75,11 +75,6 @@ public void initializeState(StateInitializationContext context) throws Exception super.initializeState(context); } - @Override - protected boolean containLogSystem() { - return false; - } - @Override public void processElement(StreamRecord> element) throws Exception { Tuple2 record = element.getValue(); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordKeyAndBucketExtractor.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordKeyAndBucketExtractor.java index cb983d309243..235b8efac162 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordKeyAndBucketExtractor.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordKeyAndBucketExtractor.java @@ -114,9 +114,4 @@ public BinaryRow trimmedPrimaryKey() { } return trimmedPK; } - - @Override - public BinaryRow logPrimaryKey() { - throw new UnsupportedOperationException(); - } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java index 0f0351370fbf..a37ca5499403 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java @@ -101,11 +101,6 @@ public void initializeState(StateInitializationContext context) throws Exception super.initializeState(context); } - @Override - protected boolean containLogSystem() { - return false; - } - @Override public void processElement(StreamRecord element) throws Exception { CdcRecord record = element.getValue(); diff --git a/paimon-flink/paimon-flink-cdc/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-flink/paimon-flink-cdc/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index 1f44ef5cdd17..753d6cf1585a 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-flink/paimon-flink-cdc/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -13,8 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.paimon.flink.kafka.KafkaLogStoreFactory - ### cdc action factories org.apache.paimon.flink.action.cdc.mysql.MySqlSyncTableActionFactory org.apache.paimon.flink.action.cdc.mysql.MySqlSyncDatabaseActionFactory diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/CompositePkAndMultiPartitionedTableWithKafkaLogITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/CompositePkAndMultiPartitionedTableWithKafkaLogITCase.java deleted file mode 100644 index e9e0c34e95c9..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/CompositePkAndMultiPartitionedTableWithKafkaLogITCase.java +++ /dev/null @@ -1,936 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.utils.BlockingIterator; - -import org.apache.flink.types.Row; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -import static org.apache.flink.table.planner.factories.TestValuesTableFactory.changelogRow; -import static org.apache.paimon.flink.kafka.KafkaLogTestUtils.createTableWithKafkaLog; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.SCAN_LATEST; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.assertNoMoreRecords; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.bEnv; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.buildQuery; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.buildQueryWithTableOptions; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.buildSimpleQuery; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.checkFileStorePath; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.createTemporaryTable; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.init; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.insertInto; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.insertIntoFromTable; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.insertIntoPartition; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.testBatchRead; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.testStreamingRead; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.testStreamingReadWithReadFirst; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.validateStreamingReadResult; - -/** - * IT cases of streaming reading and writing tables which have composite primary keys and multiple - * partition fields with Kafka log. - */ -public class CompositePkAndMultiPartitionedTableWithKafkaLogITCase extends KafkaTableTestBase { - - @BeforeEach - public void setUp() { - init(getTempDirPath()); - } - - // ---------------------------------------------------------------------------------------------------------------- - // Non latest - // ---------------------------------------------------------------------------------------------------------------- - - @Test - public void testStreamingReadWriteMultiPartitionedRecordsWithMultiPk() throws Exception { - List initialRecords = - Arrays.asList( - // to_currency is USD - changelogRow("+I", "US Dollar", "US Dollar", 1.0d, "2022-01-02", "20"), - changelogRow("+I", "Euro", "US Dollar", null, "2022-01-02", "20"), - changelogRow("+I", "HK Dollar", "US Dollar", 0.13d, "2022-01-02", "20"), - changelogRow("+I", "Yen", "US Dollar", 0.0082d, "2022-01-02", "20"), - changelogRow( - "+I", "Singapore Dollar", "US Dollar", 0.74d, "2022-01-02", "20"), - changelogRow("+U", "Yen", "US Dollar", 0.0081d, "2022-01-02", "20"), - changelogRow("-D", "US Dollar", "US Dollar", 1.0d, "2022-01-02", "20"), - changelogRow("-D", "Euro", "US Dollar", null, "2022-01-02", "20"), - changelogRow( - "+U", "Singapore Dollar", "US Dollar", 0.76d, "2022-01-02", "20"), - // to_currency is Euro - changelogRow("+I", "US Dollar", "Euro", 0.9d, "2022-01-02", "20"), - changelogRow("+I", "Singapore Dollar", "Euro", 0.67d, "2022-01-02", "20"), - changelogRow("+U", "Singapore Dollar", "Euro", null, "2022-01-02", "20"), - // to_currency is Yen - changelogRow("+I", "Yen", "Yen", 1.0d, "2022-01-02", "20"), - changelogRow("+I", "Chinese Yuan", "Yen", 19.25d, "2022-01-02", "20"), - changelogRow("+U", "Chinese Yuan", "Yen", 25.6d, "2022-01-02", "20"), - changelogRow("+I", "Singapore Dollar", "Yen", 90.32d, "2022-01-02", "20"), - changelogRow("+I", "US Dollar", "Yen", 122.46d, "2022-01-02", "21"), - changelogRow("+U", "Singapore Dollar", "Yen", 90.1d, "2022-01-02", "20")); - - String temporaryTable = - createTemporaryTable( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING", - "hh STRING"), - Arrays.asList("from_currency", "to_currency", "dt", "hh"), - Arrays.asList("dt", "hh"), - initialRecords, - "dt:2022-01-02,hh:20;dt:2022-01-02,hh:21", - false, - "I,UA,D"); - - String table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING", - "hh STRING"), - Arrays.asList("from_currency", "to_currency", "dt", "hh"), - Arrays.asList("dt", "hh"), - false); - - insertIntoFromTable(temporaryTable, table); - - checkFileStorePath(table, Arrays.asList("dt=2022-01-02,hh=20", "dt=2022-01-02,hh=21")); - - BlockingIterator streamingItr = - testStreamingRead( - buildSimpleQuery(table), - Arrays.asList( - changelogRow( - "+I", "HK Dollar", "US Dollar", 0.13d, "2022-01-02", "20"), - changelogRow("+I", "Yen", "US Dollar", 0.0081d, "2022-01-02", "20"), - changelogRow( - "+I", - "Singapore Dollar", - "US Dollar", - 0.76d, - "2022-01-02", - "20"), - changelogRow("+I", "US Dollar", "Euro", 0.9d, "2022-01-02", "20"), - changelogRow( - "+I", "Singapore Dollar", "Euro", null, "2022-01-02", "20"), - changelogRow("+I", "Yen", "Yen", 1.0d, "2022-01-02", "20"), - changelogRow( - "+I", "Chinese Yuan", "Yen", 25.6d, "2022-01-02", "20"), - changelogRow("+I", "US Dollar", "Yen", 122.46d, "2022-01-02", "21"), - changelogRow( - "+I", - "Singapore Dollar", - "Yen", - 90.1d, - "2022-01-02", - "20"))); - - // test streaming consume changelog - insertInto(table, "('Chinese Yuan', 'HK Dollar', 1.231, '2022-01-03', '15')"); - - validateStreamingReadResult( - streamingItr, - Collections.singletonList( - changelogRow( - "+I", "Chinese Yuan", "HK Dollar", 1.231d, "2022-01-03", "15"))); - - // dynamic overwrite the whole table - bEnv.executeSql( - String.format( - "INSERT OVERWRITE `%s` SELECT 'US Dollar', 'US Dollar', 1, '2022-04-02', '10' FROM `%s`", - table, table)) - .await(); - - checkFileStorePath(table, Collections.singletonList("dt=2022-04-02,hh=10")); - - // batch read to check data refresh - testBatchRead( - buildSimpleQuery(table), - Collections.singletonList( - changelogRow("+I", "US Dollar", "US Dollar", 1.0d, "2022-04-02", "10"))); - - // check no changelog generated for streaming read - assertNoMoreRecords(streamingItr); - streamingItr.close(); - - // reset table - table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING", - "hh STRING"), - Arrays.asList("from_currency", "to_currency", "dt", "hh"), - Arrays.asList("dt", "hh"), - false); - - insertIntoFromTable(temporaryTable, table); - - // filter on partition and field filter - testStreamingRead( - buildQuery( - table, - "*", - "WHERE dt = '2022-01-02' AND from_currency = 'US Dollar'"), - Arrays.asList( - changelogRow("+I", "US Dollar", "Euro", 0.9d, "2022-01-02", "20"), - changelogRow( - "+I", "US Dollar", "Yen", 122.46d, "2022-01-02", "21"))) - .close(); - - // test projection and filter - testStreamingRead( - buildQuery( - table, - "from_currency, to_currency", - "WHERE dt = '2022-01-02' AND from_currency = 'US Dollar'"), - Arrays.asList( - changelogRow("+I", "US Dollar", "Euro"), - changelogRow("+I", "US Dollar", "Yen"))) - .close(); - } - - @Test - public void testStreamingReadWriteSinglePartitionedRecordsWithMultiPk() throws Exception { - List initialRecords = - Arrays.asList( - // to_currency is USD - changelogRow("+I", "US Dollar", "US Dollar", null, "2022-01-01"), - changelogRow("+I", "Euro", "US Dollar", null, "2022-01-01"), - changelogRow("+I", "HK Dollar", "US Dollar", 0.13d, "2022-01-01"), - changelogRow("+I", "Yen", "US Dollar", 0.0082d, "2022-01-01"), - changelogRow("-D", "Yen", "US Dollar", 0.0082d, "2022-01-01"), - changelogRow("+I", "Singapore Dollar", "US Dollar", 0.74d, "2022-01-01"), - changelogRow("+I", "Yen", "US Dollar", 0.0081d, "2022-01-01"), - changelogRow("+U", "Euro", "US Dollar", 1.11d, "2022-01-01"), - changelogRow("+U", "US Dollar", "US Dollar", 1.0d, "2022-01-01"), - // to_currency is Euro - changelogRow("+I", "US Dollar", "Euro", 0.9d, "2022-01-02"), - changelogRow("+I", "Singapore Dollar", "Euro", 0.67d, "2022-01-02"), - changelogRow("-D", "Singapore Dollar", "Euro", 0.67d, "2022-01-02"), - // to_currency is Yen - changelogRow("+I", "Yen", "Yen", 1.0d, "2022-01-02"), - changelogRow("+I", "Chinese Yuan", "Yen", 19.25d, "2022-01-02"), - changelogRow("+I", "Singapore Dollar", "Yen", 90.32d, "2022-01-02"), - changelogRow("+U", "Singapore Dollar", "Yen", 122.46d, "2022-01-02")); - - String temporaryTable = - createTemporaryTable( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING"), - Arrays.asList("from_currency", "to_currency", "dt"), - Collections.singletonList("dt"), - initialRecords, - "dt:2022-01-01;dt:2022-01-02", - false, - "I,UA,D"); - - String table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING"), - Arrays.asList("from_currency", "to_currency", "dt"), - Collections.singletonList("dt"), - false); - - insertIntoFromTable(temporaryTable, table); - - checkFileStorePath(table, Arrays.asList("dt=2022-01-01", "dt=2022-01-02")); - - BlockingIterator streamingItr = - testStreamingRead( - buildSimpleQuery(table), - Arrays.asList( - changelogRow("+I", "HK Dollar", "US Dollar", 0.13d, "2022-01-01"), - changelogRow( - "+I", "Singapore Dollar", "US Dollar", 0.74d, "2022-01-01"), - changelogRow("+I", "Yen", "US Dollar", 0.0081d, "2022-01-01"), - changelogRow("+I", "Euro", "US Dollar", 1.11d, "2022-01-01"), - changelogRow("+I", "US Dollar", "US Dollar", 1.0d, "2022-01-01"), - changelogRow("+I", "US Dollar", "Euro", 0.9d, "2022-01-02"), - changelogRow("+I", "Yen", "Yen", 1.0d, "2022-01-02"), - changelogRow("+I", "Chinese Yuan", "Yen", 19.25d, "2022-01-02"), - changelogRow( - "+I", "Singapore Dollar", "Yen", 122.46d, "2022-01-02"))); - - // test streaming consume changelog - insertIntoPartition( - table, "PARTITION (dt = '2022-01-03')", "('Chinese Yuan', 'HK Dollar', 1.231)"); - - validateStreamingReadResult( - streamingItr, - Collections.singletonList( - changelogRow("+I", "Chinese Yuan", "HK Dollar", 1.231d, "2022-01-03"))); - streamingItr.close(); - - // filter on partition and field filter - testStreamingRead( - buildQuery( - table, - "*", - "WHERE dt = '2022-01-02' AND from_currency = 'US Dollar'"), - Collections.singletonList( - changelogRow("+I", "US Dollar", "Euro", 0.9d, "2022-01-02"))) - .close(); - - // test projection and filter - testStreamingRead( - buildQuery( - table, - "from_currency, to_currency", - "WHERE dt = '2022-01-01' AND rate_by_to_currency IS NULL"), - Collections.emptyList()) - .close(); - } - - // ---------------------------------------------------------------------------------------------------------------- - // Latest - // ---------------------------------------------------------------------------------------------------------------- - @Test - public void testReadLatestChangelogOfMultiPartitionedRecordsWithMultiPk() throws Exception { - List initialRecords = - Arrays.asList( - // to_currency is USD - changelogRow("+I", "US Dollar", "US Dollar", 1.0d, "2022-01-02", "20"), - changelogRow("+I", "Euro", "US Dollar", null, "2022-01-02", "20"), - changelogRow("+I", "HK Dollar", "US Dollar", 0.13d, "2022-01-02", "20"), - changelogRow("+I", "Yen", "US Dollar", 0.0082d, "2022-01-02", "20"), - changelogRow( - "+I", "Singapore Dollar", "US Dollar", 0.74d, "2022-01-02", "20"), - changelogRow("+U", "Yen", "US Dollar", 0.0081d, "2022-01-02", "20"), - changelogRow("-D", "US Dollar", "US Dollar", 1.0d, "2022-01-02", "20"), - changelogRow("-D", "Euro", "US Dollar", null, "2022-01-02", "20"), - changelogRow( - "+U", "Singapore Dollar", "US Dollar", 0.76d, "2022-01-02", "20"), - // to_currency is Euro - changelogRow("+I", "US Dollar", "Euro", 0.9d, "2022-01-02", "20"), - changelogRow("+I", "Singapore Dollar", "Euro", 0.67d, "2022-01-02", "20"), - changelogRow("+U", "Singapore Dollar", "Euro", null, "2022-01-02", "20"), - // to_currency is Yen - changelogRow("+I", "Yen", "Yen", 1.0d, "2022-01-02", "20"), - changelogRow("+I", "Chinese Yuan", "Yen", 19.25d, "2022-01-02", "20"), - changelogRow("+U", "Chinese Yuan", "Yen", 25.6d, "2022-01-02", "20"), - changelogRow("+I", "Singapore Dollar", "Yen", 90.32d, "2022-01-02", "20"), - changelogRow("+I", "US Dollar", "Yen", 122.46d, "2022-01-02", "21"), - changelogRow("+U", "Singapore Dollar", "Yen", 90.1d, "2022-01-02", "20")); - - String temporaryTable = - createTemporaryTable( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING", - "hh STRING"), - Arrays.asList("from_currency", "to_currency", "dt", "hh"), - Arrays.asList("dt", "hh"), - initialRecords, - "dt:2022-01-02,hh:20;dt:2022-01-02,hh:21", - false, - "I,UA,D"); - - String table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING", - "hh STRING"), - Arrays.asList("from_currency", "to_currency", "dt", "hh"), - Arrays.asList("dt", "hh"), - true); - - List expectedRecords = new ArrayList<>(initialRecords); - expectedRecords.add(changelogRow("-U", "Yen", "US Dollar", 0.0082d, "2022-01-02", "20")); - expectedRecords.add( - changelogRow("-U", "Singapore Dollar", "US Dollar", 0.74d, "2022-01-02", "20")); - expectedRecords.add( - changelogRow("-U", "Singapore Dollar", "Euro", 0.67d, "2022-01-02", "20")); - expectedRecords.add(changelogRow("-U", "Chinese Yuan", "Yen", 19.25d, "2022-01-02", "20")); - expectedRecords.add( - changelogRow("-U", "Singapore Dollar", "Yen", 90.32d, "2022-01-02", "20")); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions(table, "*", "", SCAN_LATEST), - expectedRecords) - .close(); - - // test partition filter - table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING", - "hh STRING"), - Arrays.asList("from_currency", "to_currency", "dt", "hh"), - Arrays.asList("dt", "hh"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, "*", "WHERE dt = '2022-01-02' AND hh = '21'", SCAN_LATEST), - Collections.singletonList( - changelogRow( - "+I", "US Dollar", "Yen", 122.46d, "2022-01-02", "21"))) - .close(); - - // test field filter - table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING", - "hh STRING"), - Arrays.asList("from_currency", "to_currency", "dt", "hh"), - Arrays.asList("dt", "hh"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, - "*", - "WHERE rate_by_to_currency IS NOT NULL AND from_currency = 'US Dollar'", - SCAN_LATEST), - Arrays.asList( - // to_currency is USD - changelogRow( - "+I", "US Dollar", "US Dollar", 1.0d, "2022-01-02", "20"), - changelogRow( - "-D", "US Dollar", "US Dollar", 1.0d, "2022-01-02", "20"), - // to_currency is Euro - changelogRow("+I", "US Dollar", "Euro", 0.9d, "2022-01-02", "20"), - // to_currency is Yen - changelogRow( - "+I", "US Dollar", "Yen", 122.46d, "2022-01-02", "21"))) - .close(); - - // test partition and field filter - table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING", - "hh STRING"), - Arrays.asList("from_currency", "to_currency", "dt", "hh"), - Arrays.asList("dt", "hh"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, - "*", - "WHERE hh = '21' AND from_currency = 'US Dollar'", - SCAN_LATEST), - Collections.singletonList( - changelogRow( - "+I", "US Dollar", "Yen", 122.46d, "2022-01-02", "21"))) - .close(); - - // test projection - table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING", - "hh STRING"), - Arrays.asList("from_currency", "to_currency", "dt", "hh"), - Arrays.asList("dt", "hh"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, "from_currency, to_currency", "", SCAN_LATEST), - Arrays.asList( - // to_currency is USD - changelogRow("+I", "US Dollar", "US Dollar"), - changelogRow("+I", "Euro", "US Dollar"), - changelogRow("+I", "HK Dollar", "US Dollar"), - changelogRow("+I", "Yen", "US Dollar"), - changelogRow("+I", "Singapore Dollar", "US Dollar"), - changelogRow("-D", "US Dollar", "US Dollar"), - changelogRow("-D", "Euro", "US Dollar"), - // to_currency is Euro - changelogRow("+I", "US Dollar", "Euro"), - changelogRow("+I", "Singapore Dollar", "Euro"), - // to_currency is Yen - changelogRow("+I", "Yen", "Yen"), - changelogRow("+I", "Chinese Yuan", "Yen"), - changelogRow("+I", "Singapore Dollar", "Yen"), - changelogRow("+I", "US Dollar", "Yen"))) - .close(); - - // test projection and filter - table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING", - "hh STRING"), - Arrays.asList("from_currency", "to_currency", "dt", "hh"), - Arrays.asList("dt", "hh"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, - "from_currency, to_currency", - "WHERE rate_by_to_currency > 100", - SCAN_LATEST), - Collections.singletonList(changelogRow("+I", "US Dollar", "Yen"))) - .close(); - } - - @Test - public void testReadLatestChangelogOfSinglePartitionedRecordsWithMultiPk() throws Exception { - List initialRecords = - Arrays.asList( - // to_currency is USD - changelogRow("+I", "US Dollar", "US Dollar", null, "2022-01-01"), - changelogRow("+I", "Euro", "US Dollar", null, "2022-01-01"), - changelogRow("+I", "HK Dollar", "US Dollar", 0.13d, "2022-01-01"), - changelogRow("+I", "Yen", "US Dollar", 0.0082d, "2022-01-01"), - changelogRow("-D", "Yen", "US Dollar", 0.0082d, "2022-01-01"), - changelogRow("+I", "Singapore Dollar", "US Dollar", 0.74d, "2022-01-01"), - changelogRow("+I", "Yen", "US Dollar", 0.0081d, "2022-01-01"), - changelogRow("+U", "Euro", "US Dollar", 1.11d, "2022-01-01"), - changelogRow("+U", "US Dollar", "US Dollar", 1.0d, "2022-01-01"), - // to_currency is Euro - changelogRow("+I", "US Dollar", "Euro", 0.9d, "2022-01-02"), - changelogRow("+I", "Singapore Dollar", "Euro", 0.67d, "2022-01-02"), - changelogRow("-D", "Singapore Dollar", "Euro", 0.67d, "2022-01-02"), - // to_currency is Yen - changelogRow("+I", "Yen", "Yen", 1.0d, "2022-01-02"), - changelogRow("+I", "Chinese Yuan", "Yen", 19.25d, "2022-01-02"), - changelogRow("+I", "Singapore Dollar", "Yen", 90.32d, "2022-01-02"), - changelogRow("+U", "Singapore Dollar", "Yen", 122.46d, "2022-01-02")); - - String temporaryTable = - createTemporaryTable( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING"), - Arrays.asList("from_currency", "to_currency", "dt"), - Collections.singletonList("dt"), - initialRecords, - "dt:2022-01-01;dt:2022-01-02", - false, - "I,UA,D"); - - String table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING"), - Arrays.asList("from_currency", "to_currency", "dt"), - Collections.singletonList("dt"), - true); - - List expectedRecords = new ArrayList<>(initialRecords); - expectedRecords.add(changelogRow("-U", "Euro", "US Dollar", null, "2022-01-01")); - expectedRecords.add(changelogRow("-U", "US Dollar", "US Dollar", null, "2022-01-01")); - expectedRecords.add(changelogRow("-U", "Singapore Dollar", "Yen", 90.32d, "2022-01-02")); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions(table, "*", "", SCAN_LATEST), - expectedRecords) - .close(); - - // test partition filter - table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING"), - Arrays.asList("from_currency", "to_currency", "dt"), - Collections.singletonList("dt"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, "*", "WHERE dt = '2022-01-02'", SCAN_LATEST), - Arrays.asList( - changelogRow("+I", "US Dollar", "Euro", 0.9d, "2022-01-02"), - changelogRow("+I", "Singapore Dollar", "Euro", 0.67d, "2022-01-02"), - changelogRow("-D", "Singapore Dollar", "Euro", 0.67d, "2022-01-02"), - changelogRow("+I", "Yen", "Yen", 1.0d, "2022-01-02"), - changelogRow("+I", "Chinese Yuan", "Yen", 19.25d, "2022-01-02"), - changelogRow("+I", "Singapore Dollar", "Yen", 90.32d, "2022-01-02"), - changelogRow( - "+U", "Singapore Dollar", "Yen", 122.46d, "2022-01-02"), - changelogRow( - "-U", "Singapore Dollar", "Yen", 90.32d, "2022-01-02"))) - .close(); - - // test field filter - table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING"), - Arrays.asList("from_currency", "to_currency", "dt"), - Collections.singletonList("dt"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, "*", "WHERE rate_by_to_currency IS NULL", SCAN_LATEST), - Arrays.asList( - changelogRow("+I", "US Dollar", "US Dollar", null, "2022-01-01"), - changelogRow("+I", "Euro", "US Dollar", null, "2022-01-01"), - changelogRow("-U", "Euro", "US Dollar", null, "2022-01-01"), - changelogRow("-U", "US Dollar", "US Dollar", null, "2022-01-01"))) - .close(); - - // test partition and field filter - table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING"), - Arrays.asList("from_currency", "to_currency", "dt"), - Collections.singletonList("dt"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, - "*", - "WHERE dt = '2022-01-02' AND from_currency = 'Yen'", - SCAN_LATEST), - Collections.singletonList( - changelogRow("+I", "Yen", "Yen", 1.0d, "2022-01-02"))) - .close(); - - // test projection and filter - table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE", - "dt STRING"), - Arrays.asList("from_currency", "to_currency", "dt"), - Collections.singletonList("dt"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, - "from_currency, to_currency", - "WHERE rate_by_to_currency > 100", - SCAN_LATEST), - Collections.singletonList(changelogRow("+U", "Singapore Dollar", "Yen"))) - .close(); - } - - @Test - public void testReadLatestChangelogOfNonPartitionedRecordsWithMultiPk() throws Exception { - List initialRecords = - Arrays.asList( - // to_currency is USD - changelogRow("+I", "US Dollar", "US Dollar", 1.0d), - changelogRow("+I", "Euro", "US Dollar", 1.11d), - changelogRow("+I", "HK Dollar", "US Dollar", 0.13d), - changelogRow("+U", "Euro", "US Dollar", 1.12d), - changelogRow("+I", "Yen", "US Dollar", 0.0082d), - changelogRow("+I", "Singapore Dollar", "US Dollar", 0.74d), - changelogRow("+U", "Yen", "US Dollar", 0.0081d), - changelogRow("-D", "US Dollar", "US Dollar", 1.0d), - changelogRow("-D", "Yen", "US Dollar", 0.0081d), - // to_currency is Euro - changelogRow("+I", "US Dollar", "Euro", 0.9d), - changelogRow("+I", "Singapore Dollar", "Euro", 0.67d), - changelogRow("+U", "Singapore Dollar", "Euro", 0.69d), - // to_currency is Yen - changelogRow("+I", "Yen", "Yen", 1.0d), - changelogRow("+I", "Chinese Yuan", "Yen", 19.25d), - changelogRow("+I", "Singapore Dollar", "Yen", 90.32d), - changelogRow("-D", "Yen", "Yen", 1.0d), - changelogRow("+U", "Singapore Dollar", "Yen", 122.46d), - changelogRow("+U", "Singapore Dollar", "Yen", 122d)); - - String temporaryTable = - createTemporaryTable( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE"), - Arrays.asList("from_currency", "to_currency"), - Collections.emptyList(), - initialRecords, - null, - false, - "I,UA,D"); - - String table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE"), - Arrays.asList("from_currency", "to_currency"), - Collections.emptyList(), - true); - - List expectedRecords = new ArrayList<>(initialRecords); - expectedRecords.add(changelogRow("-U", "Yen", "US Dollar", 0.0082d)); - expectedRecords.add(changelogRow("-U", "Euro", "US Dollar", 1.11d)); - expectedRecords.add(changelogRow("-U", "Singapore Dollar", "Euro", 0.67d)); - expectedRecords.add(changelogRow("-U", "Singapore Dollar", "Yen", 90.32d)); - expectedRecords.add(changelogRow("-U", "Singapore Dollar", "Yen", 122.46d)); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions(table, "*", "", SCAN_LATEST), - expectedRecords) - .close(); - - // test field filter - table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE"), - Arrays.asList("from_currency", "to_currency"), - Collections.emptyList(), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, - "*", - "WHERE rate_by_to_currency < 1 OR rate_by_to_currency > 100", - SCAN_LATEST), - Arrays.asList( - // to_currency is USD - changelogRow("+I", "HK Dollar", "US Dollar", 0.13d), - changelogRow("+I", "Yen", "US Dollar", 0.0082d), - changelogRow("-U", "Yen", "US Dollar", 0.0082d), - changelogRow("+I", "Singapore Dollar", "US Dollar", 0.74d), - changelogRow("+U", "Yen", "US Dollar", 0.0081d), - changelogRow("-D", "Yen", "US Dollar", 0.0081d), - // to_currency is Euro - changelogRow("+I", "US Dollar", "Euro", 0.9d), - changelogRow("+I", "Singapore Dollar", "Euro", 0.67d), - changelogRow("-U", "Singapore Dollar", "Euro", 0.67d), - changelogRow("+U", "Singapore Dollar", "Euro", 0.69d), - // to_currency is Yen - changelogRow("+U", "Singapore Dollar", "Yen", 122.46d), - changelogRow("-U", "Singapore Dollar", "Yen", 122.46d), - changelogRow("+U", "Singapore Dollar", "Yen", 122d))) - .close(); - - // test projection and filter - table = - createTableWithKafkaLog( - Arrays.asList( - "from_currency STRING", - "to_currency STRING", - "rate_by_to_currency DOUBLE"), - Arrays.asList("from_currency", "to_currency"), - Collections.emptyList(), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, - "from_currency, to_currency", - "WHERE rate_by_to_currency < 1 OR rate_by_to_currency > 100", - SCAN_LATEST), - Arrays.asList( - // to_currency is USD - changelogRow("+I", "HK Dollar", "US Dollar"), - changelogRow("+I", "Yen", "US Dollar"), - changelogRow("-U", "Yen", "US Dollar"), - changelogRow("+I", "Singapore Dollar", "US Dollar"), - changelogRow("+U", "Yen", "US Dollar"), - changelogRow("-D", "Yen", "US Dollar"), - // to_currency is Euro - changelogRow("+I", "US Dollar", "Euro"), - changelogRow("+I", "Singapore Dollar", "Euro"), - changelogRow("-U", "Singapore Dollar", "Euro"), - changelogRow("+U", "Singapore Dollar", "Euro"), - // to_currency is Yen - changelogRow("+U", "Singapore Dollar", "Yen"), - changelogRow("-U", "Singapore Dollar", "Yen"), - changelogRow("+U", "Singapore Dollar", "Yen"))) - .close(); - } - - @Test - public void testReadLatestChangelogOfMultiPartitionedRecordsWithOnePk() throws Exception { - List initialRecords = - Arrays.asList( - // dt = 2022-01-01, hh = "15" - changelogRow("+I", "US Dollar", 102L, "2022-01-01", "15"), - changelogRow("+I", "Euro", 114L, "2022-01-01", "15"), - changelogRow("+I", "Yen", 1L, "2022-01-01", "15"), - changelogRow("+U", "Euro", 116L, "2022-01-01", "15"), - changelogRow("-D", "Yen", 1L, "2022-01-01", "15"), - changelogRow("-D", "Euro", 116L, "2022-01-01", "15"), - // dt = 2022-01-02, hh = "23" - changelogRow("+I", "Euro", 119L, "2022-01-02", "23"), - changelogRow("+U", "Euro", 119L, "2022-01-02", "23")); - - String temporaryTable = - createTemporaryTable( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING", "hh STRING"), - Arrays.asList("currency", "dt", "hh"), - Arrays.asList("dt", "hh"), - initialRecords, - "dt:2022-01-01,hh:15;dt:2022-01-02,hh:23", - false, - "I,UA,D"); - - String table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING", "hh STRING"), - Arrays.asList("currency", "dt", "hh"), - Arrays.asList("dt", "hh"), - true); - - List expectedRecords = new ArrayList<>(initialRecords); - expectedRecords.remove(changelogRow("+U", "Euro", 119L, "2022-01-02", "23")); - expectedRecords.add(changelogRow("-U", "Euro", 114L, "2022-01-01", "15")); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions(table, "*", "", SCAN_LATEST), - expectedRecords) - .close(); - - // test partition filter - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING", "hh STRING"), - Arrays.asList("currency", "dt", "hh"), - Arrays.asList("dt", "hh"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, "*", "WHERE dt >= '2022-01-02'", SCAN_LATEST), - Collections.singletonList( - changelogRow("+I", "Euro", 119L, "2022-01-02", "23"))) - .close(); - - // test field filter - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING", "hh STRING"), - Arrays.asList("currency", "dt", "hh"), - Arrays.asList("dt", "hh"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions(table, "*", "WHERE rate = 1", SCAN_LATEST), - Arrays.asList( - changelogRow("+I", "Yen", 1L, "2022-01-01", "15"), - changelogRow("-D", "Yen", 1L, "2022-01-01", "15"))) - .close(); - - // test projection and filter - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING", "hh STRING"), - Arrays.asList("currency", "dt", "hh"), - Arrays.asList("dt", "hh"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, "currency", "WHERE rate = 1", SCAN_LATEST), - Arrays.asList(changelogRow("+I", "Yen"), changelogRow("-D", "Yen"))) - .close(); - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogSerializationTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogSerializationTest.java deleted file mode 100644 index 28a746c307f9..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogSerializationTest.java +++ /dev/null @@ -1,221 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.CoreOptions.LogChangelogMode; -import org.apache.paimon.table.sink.SinkRecord; -import org.apache.paimon.types.RowKind; - -import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.DynamicTableFactory; -import org.apache.flink.util.Collector; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.junit.jupiter.api.Test; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.atomic.AtomicReference; - -import static org.apache.paimon.CoreOptions.LOG_FORMAT; -import static org.apache.paimon.CoreOptions.LOG_IGNORE_DELETE; -import static org.apache.paimon.flink.FlinkRowData.toFlinkRowKind; -import static org.apache.paimon.flink.kafka.KafkaLogTestUtils.discoverKafkaLogFactory; -import static org.apache.paimon.flink.kafka.KafkaLogTestUtils.testContext; -import static org.apache.paimon.flink.kafka.KafkaLogTestUtils.testRecord; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Test for {@link KafkaLogSerializationSchema} and {@link KafkaLogDeserializationSchema}. */ -public class KafkaLogSerializationTest { - - private static final String TOPIC = "my_topic"; - - @Test - public void testKeyed() throws Exception { - checkKeyed(LogChangelogMode.AUTO, 1, 3, 5); - checkKeyed(LogChangelogMode.UPSERT, 3, 6, 9); - checkKeyed(LogChangelogMode.ALL, 2, 5, 3); - } - - @Test - public void testNonKeyedUpsert() { - assertThatThrownBy(() -> checkNonKeyed(LogChangelogMode.UPSERT, 3, 6, 9)) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - public void testNonKeyed() throws Exception { - checkNonKeyed(LogChangelogMode.AUTO, 1, 3, 5); - checkNonKeyed(LogChangelogMode.ALL, 2, 5, 3); - } - - @Test - public void testUnawareBucket() throws Exception { - checkNonKeyed(LogChangelogMode.AUTO, -1, 3, 5); - checkNonKeyed(LogChangelogMode.ALL, -1, 5, 3); - } - - @Test - public void testNonKeyedWithInsertOnlyFormat() throws Exception { - check( - LogChangelogMode.AUTO, - false, - -1, - 3, - 5, - RowKind.INSERT, - Collections.singletonMap(LOG_FORMAT.key(), "json")); - check( - LogChangelogMode.AUTO, - false, - -1, - 3, - 5, - RowKind.UPDATE_AFTER, - Collections.singletonMap(LOG_FORMAT.key(), "json")); - } - - @Test - public void testKeyedWithInsertOnlyFormat() throws Exception { - Map dynamicOptions = new HashMap<>(); - dynamicOptions.put(LOG_FORMAT.key(), "json"); - - assertThatThrownBy( - () -> - check( - LogChangelogMode.AUTO, - true, - -1, - 3, - 5, - RowKind.INSERT, - dynamicOptions)) - .isInstanceOf(ValidationException.class) - .hasMessageContaining( - "A value format should deal with all records. But json has a changelog mode of [INSERT]"); - - dynamicOptions.put(LOG_IGNORE_DELETE.key(), "true"); - check(LogChangelogMode.AUTO, true, -1, 3, 5, RowKind.INSERT, dynamicOptions); - check(LogChangelogMode.AUTO, true, -1, 3, 5, RowKind.UPDATE_AFTER, dynamicOptions); - } - - private void checkKeyed(LogChangelogMode mode, int bucket, int key, int value) - throws Exception { - check(mode, true, bucket, key, value, RowKind.INSERT); - check(mode, true, bucket, key, value, RowKind.UPDATE_BEFORE); - check(mode, true, bucket, key, value, RowKind.UPDATE_AFTER); - check(mode, true, bucket, key, value, RowKind.DELETE); - } - - private void checkNonKeyed(LogChangelogMode mode, int bucket, int key, int value) - throws Exception { - check(mode, false, bucket, key, value, RowKind.INSERT); - check(mode, false, bucket, key, value, RowKind.UPDATE_BEFORE); - check(mode, false, bucket, key, value, RowKind.UPDATE_AFTER); - check(mode, false, bucket, key, value, RowKind.DELETE); - } - - private void check( - LogChangelogMode mode, boolean keyed, int bucket, int key, int value, RowKind rowKind) - throws Exception { - check(mode, keyed, bucket, key, value, rowKind, Collections.emptyMap()); - } - - private void check( - LogChangelogMode mode, - boolean keyed, - int bucket, - int key, - int value, - RowKind rowKind, - Map dynamicOptions) - throws Exception { - KafkaLogSerializationSchema serializer = - createTestSerializationSchema(testContext("", mode, keyed, dynamicOptions)); - serializer.open(null); - KafkaRecordDeserializationSchema deserializer = - createTestDeserializationSchema(testContext("", mode, keyed, dynamicOptions)); - deserializer.open(null); - - SinkRecord input = testRecord(keyed, bucket, key, value, rowKind); - ProducerRecord record = serializer.serialize(input, null); - - if (bucket >= 0) { - assertThat(record.partition().intValue()).isEqualTo(bucket); - } else { - assertThat(record.partition()).isNull(); - } - - AtomicReference rowReference = new AtomicReference<>(); - deserializer.deserialize( - toConsumerRecord(record), - new Collector() { - @Override - public void collect(RowData record) { - if (rowReference.get() != null) { - throw new RuntimeException(); - } - rowReference.set(record); - } - - @Override - public void close() {} - }); - RowData row = rowReference.get(); - - if (rowKind == RowKind.UPDATE_BEFORE) { - assertThat(row.getRowKind()).isEqualTo(org.apache.flink.types.RowKind.DELETE); - } else if (rowKind == RowKind.UPDATE_AFTER) { - assertThat(row.getRowKind()).isEqualTo(org.apache.flink.types.RowKind.INSERT); - } else { - assertThat(row.getRowKind()).isEqualTo(toFlinkRowKind(rowKind)); - } - assertThat(row.getInt(0)).isEqualTo(key); - if (row.getRowKind() == org.apache.flink.types.RowKind.INSERT - || mode == LogChangelogMode.ALL - || !keyed) { - assertThat(row.getInt(1)).isEqualTo(value); - } else { - assertThat(row.isNullAt(1)).isTrue(); - } - } - - private ConsumerRecord toConsumerRecord(ProducerRecord record) { - int partition = record.partition() == null ? -1 : record.partition(); - return new ConsumerRecord<>(TOPIC, partition, 0, record.key(), record.value()); - } - - private static KafkaLogSerializationSchema createTestSerializationSchema( - DynamicTableFactory.Context context) { - return discoverKafkaLogFactory() - .createSinkProvider(context, KafkaLogTestUtils.SINK_CONTEXT) - .createSerializationSchema(); - } - - private static KafkaRecordDeserializationSchema createTestDeserializationSchema( - DynamicTableFactory.Context context) { - return discoverKafkaLogFactory() - .createSourceProvider(context, KafkaLogTestUtils.SOURCE_CONTEXT, null) - .createDeserializationSchema(); - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactoryTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactoryTest.java deleted file mode 100644 index a3aa12fcc706..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactoryTest.java +++ /dev/null @@ -1,149 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.CoreOptions; -import org.apache.paimon.flink.AbstractFlinkTableFactory; -import org.apache.paimon.flink.FlinkConnectorOptions; -import org.apache.paimon.flink.log.LogStoreTableFactory; -import org.apache.paimon.flink.sink.FlinkTableSink; -import org.apache.paimon.flink.source.DataTableSource; -import org.apache.paimon.fs.Path; -import org.apache.paimon.fs.local.LocalFileIO; -import org.apache.paimon.options.Options; -import org.apache.paimon.schema.Schema; -import org.apache.paimon.schema.SchemaManager; -import org.apache.paimon.table.FileStoreTable; -import org.apache.paimon.table.FileStoreTableFactory; - -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.ObjectIdentifier; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.factories.DynamicTableFactory; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.RowType; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; - -import java.time.LocalDateTime; -import java.time.format.DateTimeFormatter; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.function.Consumer; - -import static org.apache.paimon.CoreOptions.SCAN_MODE; -import static org.apache.paimon.CoreOptions.SCAN_SNAPSHOT_ID; -import static org.apache.paimon.CoreOptions.SCAN_TIMESTAMP; -import static org.apache.paimon.CoreOptions.SCAN_TIMESTAMP_MILLIS; -import static org.assertj.core.api.Assertions.assertThat; - -/** UT about {@link KafkaLogStoreFactory}. */ -public class KafkaLogStoreFactoryTest { - - @ParameterizedTest - @EnumSource(CoreOptions.StartupMode.class) - public void testCreateKafkaLogStoreFactoryTimestamp(CoreOptions.StartupMode startupMode) { - String now = String.valueOf(System.currentTimeMillis()); - Consumer> setter = - (options) -> options.put(SCAN_TIMESTAMP_MILLIS.key(), now); - testCreateKafkaLogStoreFactory(startupMode, setter); - } - - @ParameterizedTest - @EnumSource(CoreOptions.StartupMode.class) - public void testCreateKafkaLogStoreFactoryTimestampStr(CoreOptions.StartupMode startupMode) { - DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); - String timestampString = LocalDateTime.now().format(formatter); - Consumer> setter = - (options) -> options.put(SCAN_TIMESTAMP.key(), timestampString); - testCreateKafkaLogStoreFactory(startupMode, setter); - } - - private static void testCreateKafkaLogStoreFactory( - CoreOptions.StartupMode startupMode, Consumer> optionsSetter) { - Map dynamicOptions = new HashMap<>(); - dynamicOptions.put(FlinkConnectorOptions.LOG_SYSTEM.key(), "kafka"); - dynamicOptions.put(SCAN_MODE.key(), startupMode.toString()); - if (startupMode == CoreOptions.StartupMode.FROM_SNAPSHOT - || startupMode == CoreOptions.StartupMode.FROM_SNAPSHOT_FULL) { - dynamicOptions.put(SCAN_SNAPSHOT_ID.key(), "1"); - } else if (startupMode == CoreOptions.StartupMode.FROM_TIMESTAMP) { - optionsSetter.accept(dynamicOptions); - } - dynamicOptions.put(SCAN_MODE.key(), startupMode.toString()); - DynamicTableFactory.Context context = - KafkaLogTestUtils.testContext( - "table", - "", - CoreOptions.LogChangelogMode.AUTO, - CoreOptions.LogConsistency.TRANSACTIONAL, - RowType.of(new IntType(), new IntType()), - new int[] {0}, - dynamicOptions); - - try { - Optional optional = - AbstractFlinkTableFactory.createOptionalLogStoreFactory(context); - assertThat(startupMode) - .isNotIn( - CoreOptions.StartupMode.FROM_SNAPSHOT, - CoreOptions.StartupMode.FROM_SNAPSHOT_FULL); - assertThat(optional.isPresent()).isTrue(); - assertThat(optional.get()).isInstanceOf(KafkaLogStoreFactory.class); - } catch (ValidationException e) { - assertThat(startupMode) - .isIn( - CoreOptions.StartupMode.FROM_SNAPSHOT, - CoreOptions.StartupMode.FROM_SNAPSHOT_FULL); - } - } - - @Test - public void testInputChangelogProducerWithKafkaLog(@TempDir java.nio.file.Path temp) - throws Exception { - Options options = new Options(); - options.set(CoreOptions.CHANGELOG_PRODUCER, CoreOptions.ChangelogProducer.INPUT); - - Path path = new Path(temp.toUri().toString()); - new SchemaManager(LocalFileIO.create(), path) - .createTable( - new Schema( - org.apache.paimon.types.RowType.of( - new org.apache.paimon.types.IntType(), - new org.apache.paimon.types.IntType()) - .getFields(), - Collections.emptyList(), - Collections.singletonList("f0"), - options.toMap(), - "")); - FileStoreTable table = FileStoreTableFactory.create(LocalFileIO.create(), path); - - ObjectIdentifier identifier = ObjectIdentifier.of("c", "d", "t"); - DataTableSource source = - new DataTableSource(identifier, table, true, null, new KafkaLogStoreFactory()); - assertThat(source.getChangelogMode()).isEqualTo(ChangelogMode.upsert()); - - FlinkTableSink sink = new FlinkTableSink(identifier, table, null, null); - assertThat(sink.getChangelogMode(ChangelogMode.all())).isEqualTo(ChangelogMode.all()); - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreRegisterITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreRegisterITCase.java deleted file mode 100644 index e92973f32d42..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreRegisterITCase.java +++ /dev/null @@ -1,192 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.flink.log.LogStoreTableFactory; -import org.apache.paimon.options.Options; - -import org.apache.kafka.clients.admin.AdminClient; -import org.apache.kafka.clients.admin.NewTopic; -import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.common.errors.TopicExistsException; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Test; - -import java.time.Duration; -import java.util.Collections; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.TimeUnit; - -import static org.apache.paimon.flink.FlinkCatalogOptions.REGISTER_TIMEOUT; -import static org.apache.paimon.flink.FlinkConnectorOptions.LOG_SYSTEM_PARTITIONS; -import static org.apache.paimon.flink.kafka.KafkaLogOptions.BOOTSTRAP_SERVERS; -import static org.apache.paimon.flink.kafka.KafkaLogOptions.TOPIC; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatCode; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Fail.fail; - -/** Tests for {@link KafkaLogStoreRegister}. */ -public class KafkaLogStoreRegisterITCase extends KafkaTableTestBase { - private static final String DATABASE = "mock_db"; - - private static final String TABLE = "mock_table"; - - @AfterEach - public void tearDown() { - // clean up all the topics - try (AdminClient admin = createAdminClient()) { - Set topics = admin.listTopics().names().get(); - admin.deleteTopics(topics).all().get(); - } catch (Exception ignored) { - // ignored - } - } - - @Test - public void testRegisterTopic() { - String topic = "register-topic"; - - Map result = - createKafkaLogStoreRegister(getBootstrapServers(), topic, 2).registerTopic(); - checkTopicExists(topic, 2, 1); - assertThat(result.get(TOPIC.key())).isEqualTo(topic); - } - - @Test - public void testRegisterTopicAuto() { - Map result = - createKafkaLogStoreRegister(getBootstrapServers()).registerTopic(); - - try (AdminClient admin = createAdminClient()) { - Set topics = admin.listTopics().names().get(5, TimeUnit.SECONDS); - assertThat(topics.size()).isEqualTo(1); - - String topicName = topics.stream().findFirst().get(); - assertThat(result.get(TOPIC.key())).isEqualTo(topicName); - - String preFix = String.format("%s_%s_", DATABASE, TABLE); - assertThat(topicName).startsWith(preFix); - - String uuid = topicName.substring(preFix.length()); - assertThat(uuid).matches("[0-9a-fA-F]{32}"); - - // assert use bucket count when log.system.partitions is missed. - assertThat(result.get(LOG_SYSTEM_PARTITIONS.key())).isEqualTo("1"); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - public void testRegisterTopicException() { - String topic = "register-topic"; - String invalidBootstrapServers = "invalid-bootstrap-servers:9092"; - - KafkaLogStoreRegister kafkaLogStoreRegister = - createKafkaLogStoreRegister(invalidBootstrapServers, topic); - assertThatThrownBy(kafkaLogStoreRegister::registerTopic) - .isInstanceOf(IllegalStateException.class) - .hasMessageContaining("Register topic for table mock_db.mock_table failed") - .hasRootCauseInstanceOf(ConfigException.class); - } - - @Test - public void testRegisterTopicExist() { - String topic = "topic-exist"; - createTopic(topic, 1, 1); - - assertThatThrownBy( - () -> - createKafkaLogStoreRegister(getBootstrapServers(), topic) - .registerTopic()) - .isInstanceOf(IllegalStateException.class) - .hasMessageContaining("Register topic for table mock_db.mock_table failed") - .hasRootCauseInstanceOf(TopicExistsException.class); - } - - @Test - public void testUnregisterTopic() { - String topic = "unregister-topic"; - createTopic(topic, 2, 1); - - createKafkaLogStoreRegister(getBootstrapServers(), topic, 2).unRegisterTopic(); - checkTopicNotExist(topic); - } - - @Test - public void testUnregisterTopicException() { - String topic = "not_exist_topic"; - - assertThatCode( - () -> - createKafkaLogStoreRegister(getBootstrapServers(), topic) - .unRegisterTopic()) - .doesNotThrowAnyException(); - } - - private KafkaLogStoreRegister createKafkaLogStoreRegister(String bootstrapServers) { - return createKafkaLogStoreRegister(bootstrapServers, null, null); - } - - private KafkaLogStoreRegister createKafkaLogStoreRegister( - String bootstrapServers, String topic) { - return createKafkaLogStoreRegister(bootstrapServers, topic, null); - } - - private KafkaLogStoreRegister createKafkaLogStoreRegister( - String bootstrapServers, String topic, Integer partition) { - Options tableOptions = new Options(); - tableOptions.set(BOOTSTRAP_SERVERS, bootstrapServers); - - if (topic != null) { - tableOptions.set(TOPIC, topic); - } - - if (partition != null) { - tableOptions.set(LOG_SYSTEM_PARTITIONS, partition); - } - tableOptions.set(REGISTER_TIMEOUT.key(), Duration.ofSeconds(20).toString()); - - return new KafkaLogStoreRegister( - new LogStoreTableFactory.RegisterContext() { - @Override - public Options getOptions() { - return tableOptions; - } - - @Override - public Identifier getIdentifier() { - return Identifier.create(DATABASE, TABLE); - } - }); - } - - private void createTopic(String topic, int partition, int replicationFactor) { - try (AdminClient admin = createAdminClient()) { - admin.createTopics( - Collections.singletonList( - new NewTopic(topic, partition, (short) replicationFactor))); - } catch (Exception e) { - fail(e.getMessage()); - } - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogTestUtils.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogTestUtils.java deleted file mode 100644 index ef57941c52d8..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogTestUtils.java +++ /dev/null @@ -1,275 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.CoreOptions.LogChangelogMode; -import org.apache.paimon.CoreOptions.LogConsistency; -import org.apache.paimon.data.GenericRow; -import org.apache.paimon.flink.log.LogStoreTableFactory; -import org.apache.paimon.table.sink.SinkRecord; -import org.apache.paimon.types.RowKind; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.Schema; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.catalog.ObjectIdentifier; -import org.apache.flink.table.catalog.ResolvedCatalogTable; -import org.apache.flink.table.catalog.ResolvedSchema; -import org.apache.flink.table.catalog.UniqueConstraint; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.factories.DynamicTableFactory; -import org.apache.flink.table.factories.FactoryUtil; -import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; -import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.TypeConversions; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.UUID; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -import static org.apache.paimon.CoreOptions.DYNAMIC_PARTITION_OVERWRITE; -import static org.apache.paimon.CoreOptions.LOG_CHANGELOG_MODE; -import static org.apache.paimon.CoreOptions.LOG_CONSISTENCY; -import static org.apache.paimon.data.BinaryRow.EMPTY_ROW; -import static org.apache.paimon.flink.FlinkConnectorOptions.LOG_SYSTEM; -import static org.apache.paimon.flink.kafka.KafkaLogOptions.BOOTSTRAP_SERVERS; -import static org.apache.paimon.flink.kafka.KafkaLogOptions.TOPIC; -import static org.apache.paimon.flink.kafka.KafkaTableTestBase.createTopicIfNotExists; -import static org.apache.paimon.flink.kafka.KafkaTableTestBase.getBootstrapServers; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.createTable; -import static org.apache.paimon.mergetree.compact.MergeTreeCompactManagerTest.row; - -/** Utils for the test of {@link KafkaLogStoreFactory}. */ -public class KafkaLogTestUtils { - - public static final DynamicTableSource.Context SOURCE_CONTEXT = - new DynamicTableSource.Context() { - @Override - public TypeInformation createTypeInformation(DataType producedDataType) { - return createTypeInformation( - TypeConversions.fromDataToLogicalType(producedDataType)); - } - - @Override - public TypeInformation createTypeInformation( - LogicalType producedLogicalType) { - return InternalTypeInfo.of(producedLogicalType); - } - - @Override - public DynamicTableSource.DataStructureConverter createDataStructureConverter( - DataType producedDataType) { - return ScanRuntimeProviderContext.INSTANCE.createDataStructureConverter( - producedDataType); - } - }; - - public static final DynamicTableSink.Context SINK_CONTEXT = - new DynamicTableSink.Context() { - - @Override - public boolean isBounded() { - return false; - } - - @Override - public TypeInformation createTypeInformation(DataType producedDataType) { - return createTypeInformation( - TypeConversions.fromDataToLogicalType(producedDataType)); - } - - @Override - public TypeInformation createTypeInformation( - LogicalType producedLogicalType) { - return InternalTypeInfo.of(producedLogicalType); - } - - @Override - public DynamicTableSink.DataStructureConverter createDataStructureConverter( - DataType producedDataType) { - return new SinkRuntimeProviderContext(isBounded()) - .createDataStructureConverter(producedDataType); - } - - @Override - public Optional getTargetColumns() { - return Optional.empty(); - } - }; - - public static KafkaLogStoreFactory discoverKafkaLogFactory() { - return (KafkaLogStoreFactory) - LogStoreTableFactory.discoverLogStoreFactory( - Thread.currentThread().getContextClassLoader(), - KafkaLogStoreFactory.IDENTIFIER); - } - - private static DynamicTableFactory.Context createContext( - String name, RowType rowType, int[] pk, Map options) { - return new FactoryUtil.DefaultDynamicTableContext( - ObjectIdentifier.of("catalog", "database", name), - KafkaLogTestUtils.createResolvedTable(options, rowType, pk), - Collections.emptyMap(), - new Configuration(), - Thread.currentThread().getContextClassLoader(), - false); - } - - static ResolvedCatalogTable createResolvedTable( - Map options, RowType rowType, int[] pk) { - List fieldNames = rowType.getFieldNames(); - List fieldDataTypes = - rowType.getChildren().stream() - .map(TypeConversions::fromLogicalToDataType) - .collect(Collectors.toList()); - CatalogTable origin = - CatalogTable.of( - Schema.newBuilder().fromFields(fieldNames, fieldDataTypes).build(), - null, - Collections.emptyList(), - options); - List resolvedColumns = - IntStream.range(0, fieldNames.size()) - .mapToObj(i -> Column.physical(fieldNames.get(i), fieldDataTypes.get(i))) - .collect(Collectors.toList()); - UniqueConstraint constraint = null; - if (pk.length > 0) { - List pkNames = - Arrays.stream(pk).mapToObj(fieldNames::get).collect(Collectors.toList()); - constraint = UniqueConstraint.primaryKey("pk", pkNames); - } - return new ResolvedCatalogTable( - origin, new ResolvedSchema(resolvedColumns, Collections.emptyList(), constraint)); - } - - public static DynamicTableFactory.Context testContext( - String servers, LogChangelogMode changelogMode, boolean keyed) { - return testContext(servers, changelogMode, keyed, Collections.emptyMap()); - } - - public static DynamicTableFactory.Context testContext( - String servers, - LogChangelogMode changelogMode, - boolean keyed, - Map dynamicOptions) { - return testContext( - "table", - servers, - changelogMode, - LogConsistency.TRANSACTIONAL, - keyed, - dynamicOptions); - } - - static DynamicTableFactory.Context testContext( - String name, - String servers, - LogChangelogMode changelogMode, - LogConsistency consistency, - boolean keyed, - Map dynamicOptions) { - return testContext( - name, - servers, - changelogMode, - consistency, - RowType.of(new IntType(), new IntType()), - keyed ? new int[] {0} : new int[0], - dynamicOptions); - } - - public static DynamicTableFactory.Context testContext( - String name, - String servers, - LogChangelogMode changelogMode, - LogConsistency consistency, - RowType type, - int[] keys, - Map dynamicOptions) { - Map options = new HashMap<>(); - options.put(LOG_CHANGELOG_MODE.key(), changelogMode.toString()); - options.put(LOG_CONSISTENCY.key(), consistency.toString()); - options.put(BOOTSTRAP_SERVERS.key(), servers); - options.put(TOPIC.key(), UUID.randomUUID().toString()); - options.putAll(dynamicOptions); - return createContext(name, type, keys, options); - } - - static SinkRecord testRecord(boolean hasPk, int bucket, int pk, int value, RowKind rowKind) { - return new SinkRecord( - EMPTY_ROW, - bucket, - hasPk ? row(pk) : EMPTY_ROW, - GenericRow.ofKind(rowKind, pk, value)); - } - - static String createTableWithKafkaLog( - List fieldsSpec, - List primaryKeys, - List partitionKeys, - boolean manuallyCreateLogTable) { - String topic = "topic_" + UUID.randomUUID(); - List bucketKeys = new ArrayList<>(); - if (primaryKeys.isEmpty()) { - for (String fieldSpec : fieldsSpec) { - String fieldName = fieldSpec.split(" ")[0]; - if (!partitionKeys.contains(fieldName) - && !"WATERMARK".equalsIgnoreCase(fieldName) - && !fieldSpec.contains(" AS ")) { - bucketKeys.add(fieldName); - } - } - } - String table = - createTable( - fieldsSpec, - primaryKeys, - bucketKeys, - partitionKeys, - new HashMap() { - { - put(LOG_SYSTEM.key(), "kafka"); - put(BOOTSTRAP_SERVERS.key(), getBootstrapServers()); - put(TOPIC.key(), topic); - put(DYNAMIC_PARTITION_OVERWRITE.key(), "false"); - } - }); - - if (manuallyCreateLogTable) { - createTopicIfNotExists(topic, 1); - } - - return table; - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaTableTestBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaTableTestBase.java deleted file mode 100644 index eddb8ba897dd..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaTableTestBase.java +++ /dev/null @@ -1,306 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.flink.util.AbstractTestBase; - -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.util.DockerImageVersions; -import org.apache.kafka.clients.admin.AdminClient; -import org.apache.kafka.clients.admin.DescribeTopicsResult; -import org.apache.kafka.clients.admin.NewTopic; -import org.apache.kafka.clients.admin.TopicDescription; -import org.apache.kafka.clients.admin.TopicListing; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.TopicExistsException; -import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.extension.AfterAllCallback; -import org.junit.jupiter.api.extension.BeforeAllCallback; -import org.junit.jupiter.api.extension.ExtensionContext; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.KafkaContainer; -import org.testcontainers.containers.Network; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.utility.DockerImageName; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Properties; -import java.util.Timer; -import java.util.TimerTask; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Fail.fail; - -/** Base class for Kafka Table IT Cases. */ -public abstract class KafkaTableTestBase extends AbstractTestBase { - - private static final Logger LOG = LoggerFactory.getLogger(KafkaTableTestBase.class); - - private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; - private static final Network NETWORK = Network.newNetwork(); - private static final int zkTimeoutMills = 30000; - - @RegisterExtension - public static final KafkaContainerExtension KAFKA_CONTAINER = - (KafkaContainerExtension) - new KafkaContainerExtension(DockerImageName.parse(DockerImageVersions.KAFKA)) { - @Override - protected void doStart() { - super.doStart(); - if (LOG.isInfoEnabled()) { - this.followOutput(new Slf4jLogConsumer(LOG)); - } - } - }.withEmbeddedZookeeper() - .withNetwork(NETWORK) - .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS) - .withEnv( - "KAFKA_TRANSACTION_MAX_TIMEOUT_MS", - String.valueOf(Duration.ofHours(2).toMillis())) - // Disable log deletion to prevent records from being deleted during - // test run - .withEnv("KAFKA_LOG_RETENTION_MS", "-1"); - - protected StreamExecutionEnvironment env; - protected StreamTableEnvironment tEnv; - - // Timer for scheduling logging task if the test hangs - private final Timer loggingTimer = new Timer("Debug Logging Timer"); - - @BeforeEach - public void setup() { - env = streamExecutionEnvironmentBuilder().streamingMode().build(); - tEnv = StreamTableEnvironment.create(env); - tEnv.getConfig() - .getConfiguration() - .set(ExecutionCheckpointingOptions.ENABLE_UNALIGNED, false); - - // Probe Kafka broker status per 30 seconds - scheduleTimeoutLogger( - Duration.ofSeconds(30), - () -> { - // List all non-internal topics - final Map topicDescriptions = - describeExternalTopics(); - LOG.info("Current existing topics: {}", topicDescriptions.keySet()); - - // Log status of topics - logTopicPartitionStatus(topicDescriptions); - }); - } - - @AfterEach - public void after() throws ExecutionException, InterruptedException { - // Cancel timer for debug logging - cancelTimeoutLogger(); - // Delete topics for avoid reusing topics of Kafka cluster - deleteTopics(); - } - - public static Properties getStandardProps() { - Properties standardProps = new Properties(); - standardProps.put("bootstrap.servers", KAFKA_CONTAINER.getBootstrapServers()); - standardProps.put("group.id", "flink-tests"); - standardProps.put("enable.auto.commit", false); - standardProps.put("auto.offset.reset", "earliest"); - standardProps.put("max.partition.fetch.bytes", 256); - standardProps.put("zookeeper.session.timeout.ms", zkTimeoutMills); - standardProps.put("zookeeper.connection.timeout.ms", zkTimeoutMills); - return standardProps; - } - - public static String getBootstrapServers() { - return KAFKA_CONTAINER.getBootstrapServers(); - } - - protected boolean topicExists(String topicName) { - return describeExternalTopics().containsKey(topicName); - } - - public static void createTopicIfNotExists(String topicName, int numBucket) { - try (final AdminClient adminClient = AdminClient.create(getStandardProps())) { - if (!adminClient.listTopics().names().get().contains(topicName)) { - adminClient - .createTopics( - Collections.singleton( - new NewTopic( - topicName, - Optional.of(numBucket), - Optional.empty()))) - .all() - .get(); - } - } catch (Exception e) { - if (!(e.getCause() instanceof TopicExistsException)) { - throw new RuntimeException( - String.format("Failed to create Kafka topic %s", topicName), e); - } - } - } - - protected void deleteTopicIfExists(String topicName) { - try (final AdminClient adminClient = AdminClient.create(getStandardProps())) { - if (adminClient.listTopics().names().get().contains(topicName)) { - adminClient.deleteTopics(Collections.singleton(topicName)).all().get(); - } - } catch (Exception e) { - if (!(e.getCause() instanceof UnknownTopicOrPartitionException)) { - throw new RuntimeException( - String.format("Failed to drop Kafka topic %s", topicName), e); - } - } - } - - private void deleteTopics() throws ExecutionException, InterruptedException { - final AdminClient adminClient = AdminClient.create(getStandardProps()); - adminClient.deleteTopics(adminClient.listTopics().names().get()).all().get(); - } - - // ------------------------ For Debug Logging Purpose ---------------------------------- - - private void scheduleTimeoutLogger(Duration period, Runnable loggingAction) { - TimerTask timeoutLoggerTask = - new TimerTask() { - @Override - public void run() { - try { - loggingAction.run(); - } catch (Exception e) { - throw new RuntimeException("Failed to execute logging action", e); - } - } - }; - loggingTimer.schedule(timeoutLoggerTask, 0L, period.toMillis()); - } - - private void cancelTimeoutLogger() { - loggingTimer.cancel(); - } - - private Map describeExternalTopics() { - try (final AdminClient adminClient = AdminClient.create(getStandardProps())) { - final List topics = - adminClient.listTopics().listings().get().stream() - .filter(listing -> !listing.isInternal()) - .map(TopicListing::name) - .collect(Collectors.toList()); - - return adminClient.describeTopics(topics).allTopicNames().get(); - } catch (Exception e) { - throw new RuntimeException("Failed to list Kafka topics", e); - } - } - - private void logTopicPartitionStatus(Map topicDescriptions) { - final Properties properties = getStandardProps(); - properties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "flink-tests-debugging"); - properties.setProperty( - ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, - StringDeserializer.class.getCanonicalName()); - properties.setProperty( - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, - StringDeserializer.class.getCanonicalName()); - final KafkaConsumer consumer = new KafkaConsumer(properties); - List partitions = new ArrayList<>(); - topicDescriptions.forEach( - (topic, description) -> - description - .partitions() - .forEach( - tpInfo -> - partitions.add( - new TopicPartition( - topic, tpInfo.partition())))); - final Map beginningOffsets = consumer.beginningOffsets(partitions); - final Map endOffsets = consumer.endOffsets(partitions); - partitions.forEach( - partition -> - LOG.info( - "TopicPartition \"{}\": starting offset: {}, stopping offset: {}", - partition, - beginningOffsets.get(partition), - endOffsets.get(partition))); - } - - /** Kafka container extension for junit5. */ - protected static class KafkaContainerExtension extends KafkaContainer - implements BeforeAllCallback, AfterAllCallback { - private KafkaContainerExtension(DockerImageName dockerImageName) { - super(dockerImageName); - } - - @Override - public void beforeAll(ExtensionContext extensionContext) throws Exception { - this.doStart(); - } - - @Override - public void afterAll(ExtensionContext extensionContext) throws Exception { - this.close(); - } - } - - // ------------------------ For Kafka Test Purpose ---------------------------------- - protected void checkTopicExists(String topic, int partition, int replicationFactor) { - try (AdminClient admin = createAdminClient()) { - DescribeTopicsResult topicDesc = admin.describeTopics(Collections.singleton(topic)); - TopicDescription description = - topicDesc.allTopicNames().get(10, TimeUnit.SECONDS).get(topic); - - assertThat(description.partitions().size()).isEqualTo(partition); - assertThat(description.partitions().get(0).replicas().size()) - .isEqualTo(replicationFactor); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - protected void checkTopicNotExist(String topic) { - try (AdminClient admin = createAdminClient()) { - assertThat(admin.describeTopics(Collections.emptyList()).allTopicNames().get()) - .doesNotContainKey(topic); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - protected AdminClient createAdminClient() { - Properties properties = new Properties(); - properties.put("bootstrap.servers", getBootstrapServers()); - - return AdminClient.create(properties); - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/LogSystemITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/LogSystemITCase.java deleted file mode 100644 index f484b6f7253f..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/LogSystemITCase.java +++ /dev/null @@ -1,441 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.utils.BlockingIterator; - -import org.apache.flink.table.api.TableResult; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.CatalogBaseTable; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; -import org.apache.flink.types.Row; -import org.apache.kafka.clients.admin.AdminClient; -import org.apache.kafka.clients.admin.ListOffsetsResult; -import org.apache.kafka.clients.admin.OffsetSpec; -import org.apache.kafka.common.TopicPartition; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; - -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** ITCase for table with log system. */ -public class LogSystemITCase extends KafkaTableTestBase { - - @BeforeEach - public void before() throws IOException { - tEnv.executeSql( - String.format( - "CREATE CATALOG PAIMON WITH (" + "'type'='paimon', 'warehouse'='%s')", - getTempDirPath())); - tEnv.useCatalog("PAIMON"); - } - - @Test - public void testAppendOnlyWithEventual() throws Exception { - createTopicIfNotExists("T", 1); - // disable checkpointing to test eventual - env.getCheckpointConfig().disableCheckpointing(); - env.setParallelism(1); - tEnv.executeSql( - String.format( - "CREATE TABLE T (i INT, j INT) WITH (" - + "'log.system'='kafka', " - + "'bucket'='1', " - + "'bucket-key'='i', " - + "'log.consistency'='eventual', " - + "'kafka.bootstrap.servers'='%s', " - + "'kafka.topic'='T')", - getBootstrapServers())); - tEnv.executeSql("CREATE TEMPORARY TABLE gen (i INT, j INT) WITH ('connector'='datagen')"); - TableResult write = tEnv.executeSql("INSERT INTO T SELECT * FROM gen"); - BlockingIterator read = - BlockingIterator.of(tEnv.executeSql("SELECT * FROM T").collect()); - List collect = read.collect(10); - assertThat(collect).hasSize(10); - write.getJobClient().get().cancel(); - read.close(); - } - - @Test - public void testReadFromFile() throws Exception { - createTopicIfNotExists("test-double-sink", 1); - env.getCheckpointConfig().setCheckpointInterval(3 * 1000); - env.setParallelism(1); - tEnv.executeSql( - String.format( - "CREATE TABLE kafka_file_double_sink (\n" - + " word STRING ,\n" - + " cnt BIGINT,\n" - + " PRIMARY KEY (word) NOT ENFORCED\n" - + ")\n" - + "WITH (\n" - + " 'merge-engine' = 'aggregation',\n" - + " 'changelog-producer' = 'full-compaction',\n" - + " 'log.system' = 'kafka',\n" - + " 'bucket'='1',\n" - + " 'streaming-read-mode'='file',\n" - + " 'fields.cnt.aggregate-function' = 'sum',\n" - + " 'kafka.bootstrap.servers' = '%s',\n" - + " 'kafka.topic' = 'test-double-sink',\n" - + " 'kafka.transaction.timeout.ms'='30000'\n" - + "\n" - + ");", - getBootstrapServers())); - TableResult write = - tEnv.executeSql( - "INSERT INTO kafka_file_double_sink values('a',1),('b',2),('c',3);"); - BlockingIterator read = - BlockingIterator.of( - tEnv.executeSql("SELECT * FROM kafka_file_double_sink").collect()); - assertThat(read.collect(3)) - .containsExactlyInAnyOrder(Row.of("a", 1L), Row.of("b", 2L), Row.of("c", 3L)); - write.getJobClient().get().cancel(); - read.close(); - } - - @Test - public void testReadFromLog() throws Exception { - createTopicIfNotExists("test-single-sink", 1); - // disable checkpointing to test eventual - env.getCheckpointConfig().disableCheckpointing(); - env.setParallelism(1); - // 'fields.cnt.aggregate-function' = 'sum' is miss will throw - // java.lang.UnsupportedOperationException: Aggregate function 'last_non_null_value' does - // not support retraction - // data will only be written to kafka - tEnv.executeSql( - String.format( - "CREATE TABLE kafka_file_single_sink (\n" - + " word STRING ,\n" - + " cnt BIGINT,\n" - + " PRIMARY KEY (word) NOT ENFORCED\n" - + ")\n" - + "WITH (\n" - + " 'merge-engine' = 'aggregation',\n" - + " 'changelog-producer' = 'full-compaction',\n" - + " 'log.consistency' = 'eventual',\n" - + " 'log.system' = 'kafka',\n" - + " 'bucket'='1',\n" - + " 'streaming-read-mode'='log',\n" - + " 'kafka.bootstrap.servers' = '%s',\n" - + " 'kafka.topic' = 'test-single-sink',\n" - + " 'kafka.transaction.timeout.ms'='30000'\n" - + "\n" - + ");", - getBootstrapServers())); - tEnv.executeSql( - "CREATE TEMPORARY TABLE word_table (\n" - + " word STRING\n" - + ") WITH (\n" - + " 'connector' = 'datagen',\n" - + " 'fields.word.length' = '1'\n" - + ");"); - TableResult write = - tEnv.executeSql( - "INSERT INTO kafka_file_single_sink SELECT word, COUNT(*) FROM word_table GROUP BY word;"); - BlockingIterator read = - BlockingIterator.of( - tEnv.executeSql("SELECT * FROM kafka_file_single_sink").collect()); - List collect = read.collect(10); - assertThat(collect).hasSize(10); - write.getJobClient().get().cancel(); - read.close(); - } - - @Test - public void testReadFromLogWithOutSteamingReadMode() throws Exception { - createTopicIfNotExists("test-single-sink", 1); - env.setParallelism(1); - - tEnv.executeSql( - "CREATE TABLE kafka_file_single_sink (\n" - + " word STRING ,\n" - + " cnt BIGINT,\n" - + " PRIMARY KEY (word) NOT ENFORCED\n" - + ")\n" - + "WITH (\n" - + " 'merge-engine' = 'aggregation',\n" - + " 'bucket'='1',\n" - + " 'changelog-producer' = 'full-compaction',\n" - + " 'streaming-read-mode'='log'\n" - + ");"); - tEnv.executeSql( - "CREATE TEMPORARY TABLE word_table (\n" - + " word STRING\n" - + ") WITH (\n" - + " 'connector' = 'datagen',\n" - + " 'fields.word.length' = '1'\n" - + ");"); - assertThatThrownBy( - () -> - tEnv.executeSql( - "INSERT INTO kafka_file_single_sink SELECT word, COUNT(*) FROM word_table GROUP BY word;")) - .rootCause() - .isInstanceOf(ValidationException.class) - .hasMessage( - "File store continuous reading does not support the log streaming read mode."); - } - - @Test - @Timeout(60) - public void testLogSystemAutoRegister() throws TableNotExistException { - // enable log system auto registration - tEnv.executeSql( - String.format( - "CREATE CATALOG PAIMON_REGISTER WITH (" - + "'type'='paimon', 'warehouse'='%s', 'log.system.auto-register'='true')", - getTempDirPath())); - tEnv.useCatalog("PAIMON_REGISTER"); - - env.getCheckpointConfig().disableCheckpointing(); - env.setParallelism(1); - - // check register table with specified bootstrap server and partition num. - tEnv.executeSql( - String.format( - "CREATE TABLE T (i INT, j INT) WITH (" - + "'log.system'='kafka', " - + "'bucket'='1', " - + "'bucket-key'='i', " - + "'log.system.partitions'='2', " - + "'kafka.bootstrap.servers'='%s', " - + "'kafka.topic'='Tt')", - getBootstrapServers())); - - checkTopicExists("Tt", 2, 1); - - tEnv.executeSql( - String.format( - "CREATE TABLE T2 (i INT, j INT) WITH (" - + "'log.system'='kafka', " - + "'bucket'='2', " - + "'bucket-key'='i', " - + "'kafka.bootstrap.servers'='%s', " - + "'kafka.topic'='T2')", - getBootstrapServers())); - - checkTopicExists("T2", 2, 1); - - // check register a random kafka topic - tEnv.executeSql( - String.format( - "CREATE TABLE T1 (i INT, j INT) WITH (" - + "'log.system'='kafka', " - + "'bucket'='1', " - + "'bucket-key'='i', " - + "'log.system.partitions'='2', " - + "'kafka.bootstrap.servers'='%s')", - getBootstrapServers())); - - CatalogBaseTable table = - tEnv.getCatalog("PAIMON_REGISTER") - .get() - .getTable(ObjectPath.fromString("default.T")); - checkTopicExists(table.getOptions().get("kafka.topic"), 2, 1); - - // check unregister topic when creating table fail - assertThatThrownBy( - () -> - tEnv.executeSql( - String.format( - "CREATE TABLE T (i INT, j INT) WITH (" - + "'log.system'='kafka', " - + "'bucket'='1', " - + "'bucket-key'='i', " - + "'log.system.partitions'='2', " - + "'kafka.bootstrap.servers'='%s', " - + "'kafka.topic'='T1')", - getBootstrapServers()))) - .isInstanceOf(org.apache.flink.table.api.ValidationException.class) - .hasMessage("Could not execute CreateTable in path `PAIMON_REGISTER`.`default`.`T`") - .cause() - .isInstanceOf( - org.apache.flink.table.catalog.exceptions.TableAlreadyExistException.class) - .hasMessage("Table (or view) default.T already exists in Catalog PAIMON_REGISTER."); - - checkTopicNotExist("T1"); - - // tEnv.useDatabase("NOT_EXIST"); - assertThatThrownBy( - () -> - tEnv.executeSql( - String.format( - "CREATE TABLE NOT_EXIST.T (i INT, j INT) WITH (" - + "'log.system'='kafka', " - + "'bucket'='1', " - + "'bucket-key'='i', " - + "'log.system.partitions'='2', " - + "'kafka.bootstrap.servers'='%s', " - + "'kafka.topic'='T1')", - getBootstrapServers()))) - .isInstanceOf(org.apache.flink.table.api.ValidationException.class) - .hasMessage( - "Could not execute CreateTable in path `PAIMON_REGISTER`.`NOT_EXIST`.`T`") - .cause() - .isInstanceOf( - org.apache.flink.table.catalog.exceptions.DatabaseNotExistException.class) - .hasMessage("Database NOT_EXIST does not exist in Catalog PAIMON_REGISTER."); - - checkTopicNotExist("T1"); - - // check unregister topic when drop table - tEnv.executeSql("DROP TABLE T"); - checkTopicNotExist("T"); - } - - @Test - @Timeout(60) - public void testLogSystemAutoRegisterWithDefaultOption() { - // enable log system auto registration - tEnv.executeSql( - String.format( - "CREATE CATALOG PAIMON_DEFAULT WITH (" - + "'type'='paimon', 'warehouse'='%s', " - + "'log.system.auto-register'='true', " - + "'table-default.kafka.bootstrap.servers'='%s'," - + "'table-default.log.system.partitions'='2')", - getTempDirPath(), getBootstrapServers())); - tEnv.useCatalog("PAIMON_DEFAULT"); - - env.getCheckpointConfig().disableCheckpointing(); - env.setParallelism(1); - - tEnv.executeSql( - "CREATE TABLE T (i INT, j INT) WITH (" - + "'log.system'='kafka', " - + "'kafka.topic'='T')"); - - checkTopicExists("T", 2, 1); - } - - @Test - public void testLogWriteRead() throws Exception { - String topic = UUID.randomUUID().toString(); - - try { - tEnv.executeSql( - String.format( - "CREATE TABLE T (a STRING, b STRING, c STRING) WITH (" - + "'log.system'='kafka', " - + "'bucket'='1', " - + "'bucket-key'='a', " - + "'kafka.bootstrap.servers'='%s'," - + "'kafka.topic'='%s'" - + ")", - getBootstrapServers(), topic)); - - tEnv.executeSql("INSERT INTO T VALUES ('1', '2', '3'), ('4', '5', '6')").await(); - BlockingIterator iterator = - BlockingIterator.of(tEnv.from("T").execute().collect()); - List result = iterator.collectAndClose(2); - assertThat(result) - .containsExactlyInAnyOrder(Row.of("1", "2", "3"), Row.of("4", "5", "6")); - } finally { - deleteTopicIfExists(topic); - } - } - - @Test - public void testLogWriteReadWithVirtual() throws Exception { - String topic = UUID.randomUUID().toString(); - createTopicIfNotExists(topic, 1); - - try { - tEnv.executeSql( - String.format( - "CREATE TABLE T (" - + "a STRING, " - + "b STRING, " - + "c STRING, " - + "d AS CAST(c as INT) + 1" - + ") WITH (" - + "'log.system'='kafka', " - + "'bucket'='1', " - + "'bucket-key'='a', " - + "'kafka.bootstrap.servers'='%s'," - + "'kafka.topic'='%s'" - + ")", - getBootstrapServers(), topic)); - - tEnv.executeSql("INSERT INTO T VALUES ('1', '2', '3'), ('4', '5', '6')").await(); - BlockingIterator iterator = - BlockingIterator.of(tEnv.from("T").execute().collect()); - List result = iterator.collectAndClose(2); - assertThat(result) - .containsExactlyInAnyOrder(Row.of("1", "2", "3", 4), Row.of("4", "5", "6", 7)); - } finally { - deleteTopicIfExists(topic); - } - } - - @Test - @Timeout(120) - public void testAppendOnlyWithUnawareBucket() throws Exception { - String topic = UUID.randomUUID().toString(); - createTopicIfNotExists(topic, 2); - - try { - // disable checkpointing to test eventual - env.getCheckpointConfig().disableCheckpointing(); - env.setParallelism(1); - tEnv.executeSql( - String.format( - "CREATE TABLE T (i INT, j INT) WITH (" - + "'log.system'='kafka', " - + "'log.consistency'='eventual', " - + "'bucket'='-1', " - + "'kafka.bootstrap.servers'='%s', " - + "'kafka.topic'='%s'," - + "'kafka.batch.size'='20')", - getBootstrapServers(), topic)); - tEnv.executeSql( - "CREATE TEMPORARY TABLE gen (i INT, j INT) WITH ('connector'='datagen', 'rows-per-second'='2')"); - TableResult write = tEnv.executeSql("INSERT INTO T SELECT * FROM gen"); - BlockingIterator read = - BlockingIterator.of(tEnv.executeSql("SELECT * FROM T").collect()); - List collect = read.collect(10); - assertThat(collect).hasSize(10); - write.getJobClient().get().cancel(); - read.close(); - - // check offsets - try (final AdminClient adminClient = AdminClient.create(getStandardProps())) { - Map topicPartitionOffsets = new HashMap<>(4); - for (int i = 0; i < 2; i++) { - topicPartitionOffsets.put(new TopicPartition(topic, i), OffsetSpec.latest()); - } - Map result = - adminClient.listOffsets(topicPartitionOffsets).all().get(); - assertThat(result.values()) - .allMatch(partitionOffsetInfo -> partitionOffsetInfo.offset() > 0); - } - } finally { - deleteTopicIfExists(topic); - } - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/StreamingReadWriteTableWithKafkaLogITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/StreamingReadWriteTableWithKafkaLogITCase.java deleted file mode 100644 index 437560990e59..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/StreamingReadWriteTableWithKafkaLogITCase.java +++ /dev/null @@ -1,831 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.CoreOptions; -import org.apache.paimon.utils.BlockingIterator; - -import org.apache.flink.types.Row; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.table.planner.factories.TestValuesTableFactory.changelogRow; -import static org.apache.paimon.CoreOptions.SCAN_MODE; -import static org.apache.paimon.CoreOptions.SCAN_TIMESTAMP_MILLIS; -import static org.apache.paimon.flink.kafka.KafkaLogTestUtils.createTableWithKafkaLog; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.SCAN_LATEST; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.assertNoMoreRecords; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.buildQuery; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.buildQueryWithTableOptions; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.buildSimpleQuery; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.checkFileStorePath; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.createTemporaryTable; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.init; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.insertInto; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.insertIntoFromTable; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.insertIntoPartition; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.insertOverwritePartition; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.testBatchRead; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.testStreamingRead; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.testStreamingReadWithReadFirst; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.validateStreamingReadResult; - -/** Streaming reading and writing with Kafka log IT cases. */ -public class StreamingReadWriteTableWithKafkaLogITCase extends KafkaTableTestBase { - - @BeforeEach - public void setUp() { - init(createAndRegisterTempFile("").toString()); - } - - // ---------------------------------------------------------------------------------------------------------------- - // Write First/Kafka log table auto created/scan.mode = latest-full (default setting) - // ---------------------------------------------------------------------------------------------------------------- - - @Test - public void testReadWriteWithPartitionedRecordsWithPk() throws Exception { - // test hybrid read - List initialRecords = - Arrays.asList( - // dt = 2022-01-01 - changelogRow("+I", "US Dollar", 102L, "2022-01-01"), - changelogRow("+I", "Euro", 114L, "2022-01-01"), - changelogRow("+I", "Yen", 1L, "2022-01-01"), - changelogRow("+U", "Euro", 116L, "2022-01-01"), - changelogRow("-D", "Yen", 1L, "2022-01-01"), - changelogRow("-D", "Euro", 116L, "2022-01-01"), - // dt = 2022-01-02 - changelogRow("+I", "Euro", 119L, "2022-01-02"), - changelogRow("+U", "Euro", 119L, "2022-01-02")); - - String temporaryTable = - createTemporaryTable( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), - Arrays.asList("currency", "dt"), - Collections.singletonList("dt"), - initialRecords, - "dt:2022-01-01;dt:2022-01-02", - false, - "I,UA,D"); - - String table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), - Arrays.asList("currency", "dt"), - Collections.singletonList("dt"), - false); - - insertIntoFromTable(temporaryTable, table); - - checkFileStorePath(table, Arrays.asList("dt=2022-01-01", "dt=2022-01-02")); - - BlockingIterator streamItr = - testStreamingRead( - buildQuery( - table, - "*", - "WHERE dt >= '2022-01-01' AND dt <= '2022-01-03' OR currency = 'HK Dollar'"), - Arrays.asList( - changelogRow("+I", "US Dollar", 102L, "2022-01-01"), - changelogRow("+I", "Euro", 119L, "2022-01-02"))); - - // test log store in hybrid mode accepts all filters - insertIntoPartition( - table, "PARTITION (dt = '2022-01-03')", "('HK Dollar', 100)", "('Yen', 20)"); - - insertIntoPartition(table, "PARTITION (dt = '2022-01-04')", "('Yen', 20)"); - - validateStreamingReadResult( - streamItr, - Arrays.asList( - changelogRow("+I", "HK Dollar", 100L, "2022-01-03"), - changelogRow("+I", "Yen", 20L, "2022-01-03"))); - - // overwrite partition 2022-01-02 - insertOverwritePartition( - table, "PARTITION (dt = '2022-01-02')", "('Euro', 100)", "('Yen', 1)"); - - // check no changelog generated for streaming read - assertNoMoreRecords(streamItr); - streamItr.close(); - - // batch read to check data refresh - testBatchRead( - buildSimpleQuery(table), - Arrays.asList( - changelogRow("+I", "US Dollar", 102L, "2022-01-01"), - changelogRow("+I", "Euro", 100L, "2022-01-02"), - changelogRow("+I", "Yen", 1L, "2022-01-02"), - changelogRow("+I", "HK Dollar", 100L, "2022-01-03"), - changelogRow("+I", "Yen", 20L, "2022-01-03"), - changelogRow("+I", "Yen", 20L, "2022-01-04"))); - - // filter on partition - testStreamingRead( - buildQuery(table, "*", "WHERE dt = '2022-01-01'"), - Collections.singletonList( - changelogRow("+I", "US Dollar", 102L, "2022-01-01"))) - .close(); - - // test field filter - testStreamingRead( - buildQuery(table, "*", "WHERE currency = 'US Dollar'"), - Collections.singletonList( - changelogRow("+I", "US Dollar", 102L, "2022-01-01"))) - .close(); - - // test partition and field filter - testStreamingRead( - buildQuery(table, "*", "WHERE dt = '2022-01-01' AND rate = 1"), - Collections.emptyList()) - .close(); - - // test projection and filter - testStreamingRead( - buildQuery( - table, - "rate, dt, currency", - "WHERE dt = '2022-01-02' AND currency = 'Euro'"), - Collections.singletonList(changelogRow("+I", 100L, "2022-01-02", "Euro"))) - .close(); - } - - @Test - public void testSReadWriteWithNonPartitionedRecordsWithPk() throws Exception { - // file store bounded read with merge - List initialRecords = - Arrays.asList( - changelogRow("+I", "US Dollar", 102L), - changelogRow("+I", "Euro", 114L), - changelogRow("+I", "Yen", 1L), - changelogRow("+U", "Euro", 116L), - changelogRow("-D", "Euro", 116L), - changelogRow("+I", "Euro", 119L), - changelogRow("+U", "Euro", 119L), - changelogRow("-D", "Yen", 1L)); - - String temporaryTable = - createTemporaryTable( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.singletonList("currency"), - Collections.emptyList(), - initialRecords, - null, - false, - "I, UA, D"); - - String table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.singletonList("currency"), - Collections.emptyList(), - false); - - insertIntoFromTable(temporaryTable, table); - - checkFileStorePath(table, Collections.emptyList()); - - testStreamingRead( - buildSimpleQuery(table), - Arrays.asList( - changelogRow("+I", "US Dollar", 102L), - changelogRow("+I", "Euro", 119L))) - .close(); - - // test field filter - testStreamingRead(buildQuery(table, "*", "WHERE currency = 'Yen'"), Collections.emptyList()) - .close(); - - // test projection - testStreamingRead( - buildQuery(table, "currency", ""), - Arrays.asList(changelogRow("+I", "US Dollar"), changelogRow("+I", "Euro"))) - .close(); - - // test projection and filter - testStreamingRead( - buildQuery(table, "currency", "WHERE rate = 102"), - Collections.singletonList(changelogRow("+I", "US Dollar"))) - .close(); - } - - // ---------------------------------------------------------------------------------------------------------------- - // Read First/Manually create Kafka log table/scan.mode = latest - // ---------------------------------------------------------------------------------------------------------------- - - @Test - public void testReadLatestChangelogOfPartitionedRecordsWithPk() throws Exception { - List initialRecords = - Arrays.asList( - // dt = 2022-01-01 - changelogRow("+I", "US Dollar", 102L, "2022-01-01"), - changelogRow("+I", "Euro", 114L, "2022-01-01"), - changelogRow("+I", "Yen", 1L, "2022-01-01"), - changelogRow("+U", "Euro", 116L, "2022-01-01"), - changelogRow("-D", "Yen", 1L, "2022-01-01"), - changelogRow("-D", "Euro", 116L, "2022-01-01"), - // dt = 2022-01-02 - changelogRow("+I", "Euro", 119L, "2022-01-02"), - changelogRow("+U", "Euro", 119L, "2022-01-02")); - - String temporaryTable = - createTemporaryTable( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), - Arrays.asList("currency", "dt"), - Collections.singletonList("dt"), - initialRecords, - "dt:2022-01-01;dt:2022-01-02", - false, - "I,UA,D"); - - String table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), - Arrays.asList("currency", "dt"), - Collections.singletonList("dt"), - true); - - BlockingIterator streamItr = - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions(table, "*", "", SCAN_LATEST), - Arrays.asList( - changelogRow("+I", "US Dollar", 102L, "2022-01-01"), - changelogRow("+I", "Euro", 114L, "2022-01-01"), - changelogRow("+I", "Yen", 1L, "2022-01-01"), - changelogRow("-U", "Euro", 114L, "2022-01-01"), - changelogRow("+U", "Euro", 116L, "2022-01-01"), - changelogRow("-D", "Yen", 1L, "2022-01-01"), - changelogRow("-D", "Euro", 116L, "2022-01-01"), - changelogRow("+I", "Euro", 119L, "2022-01-02"))); - - // test only read the latest log - insertInto(table, "('US Dollar', 104, '2022-01-01')", "('Euro', 100, '2022-01-02')"); - validateStreamingReadResult( - streamItr, - Arrays.asList( - changelogRow("-U", "US Dollar", 102L, "2022-01-01"), - changelogRow("+U", "US Dollar", 104L, "2022-01-01"), - changelogRow("-U", "Euro", 119L, "2022-01-02"), - changelogRow("+U", "Euro", 100L, "2022-01-02"))); - - assertNoMoreRecords(streamItr); - streamItr.close(); - - // test partition filter - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), - Arrays.asList("currency", "dt"), - Collections.singletonList("dt"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, "*", "WHERE dt = '2022-01-01'", SCAN_LATEST), - Arrays.asList( - changelogRow("+I", "US Dollar", 102L, "2022-01-01"), - changelogRow("+I", "Euro", 114L, "2022-01-01"), - changelogRow("+I", "Yen", 1L, "2022-01-01"), - changelogRow("-U", "Euro", 114L, "2022-01-01"), - changelogRow("+U", "Euro", 116L, "2022-01-01"), - changelogRow("-D", "Yen", 1L, "2022-01-01"), - changelogRow("-D", "Euro", 116L, "2022-01-01"))) - .close(); - - // test field filter - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), - Arrays.asList("currency", "dt"), - Collections.singletonList("dt"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, "*", "WHERE currency = 'Yen'", SCAN_LATEST), - Arrays.asList( - changelogRow("+I", "Yen", 1L, "2022-01-01"), - changelogRow("-D", "Yen", 1L, "2022-01-01"))) - .close(); - - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), - Arrays.asList("currency", "dt"), - Collections.singletonList("dt"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions(table, "*", "WHERE rate = 114", SCAN_LATEST), - Arrays.asList( - // part = 2022-01-01 - changelogRow("+I", "Euro", 114L, "2022-01-01"), - changelogRow("-U", "Euro", 114L, "2022-01-01"))) - .close(); - - // test partition and field filter - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), - Arrays.asList("currency", "dt"), - Collections.singletonList("dt"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, "*", "WHERE rate = 114 AND dt = '2022-01-02'", SCAN_LATEST), - Collections.emptyList()) - .close(); - - // test projection - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), - Arrays.asList("currency", "dt"), - Collections.singletonList("dt"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions(table, "rate", "", SCAN_LATEST), - Arrays.asList( - // part = 2022-01-01 - changelogRow("+I", 102L), // US Dollar - changelogRow("+I", 114L), // Euro - changelogRow("+I", 1L), // Yen - changelogRow("-U", 114L), // Euro - changelogRow("+U", 116L), // Euro - changelogRow("-D", 1L), // Yen - changelogRow("-D", 116L), // Euro - // part = 2022-01-02 - changelogRow("+I", 119L) // Euro - )) - .close(); - - // test projection and filter - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), - Arrays.asList("currency", "dt"), - Collections.singletonList("dt"), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, "rate", "WHERE dt = '2022-01-02'", SCAN_LATEST), - Collections.singletonList(changelogRow("+I", 119L)) // Euro - ) - .close(); - } - - @Test - public void testReadLatestChangelogOfNonPartitionedRecordsWithPk() throws Exception { - List initialRecords = - Arrays.asList( - changelogRow("+I", "US Dollar", 102L), - changelogRow("+I", "Euro", 114L), - changelogRow("+I", "Yen", 1L), - changelogRow("+U", "Euro", 116L), - changelogRow("-D", "Euro", 116L), - changelogRow("+I", "Euro", 119L), - changelogRow("+U", "Euro", 119L), - changelogRow("-D", "Yen", 1L)); - - String temporaryTable = - createTemporaryTable( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.singletonList("currency"), - Collections.emptyList(), - initialRecords, - null, - false, - "I,UA,D"); - - String table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.singletonList("currency"), - Collections.emptyList(), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions(table, "*", "", SCAN_LATEST), - Arrays.asList( - changelogRow("+I", "US Dollar", 102L), - changelogRow("+I", "Euro", 114L), - changelogRow("+I", "Yen", 1L), - changelogRow("-U", "Euro", 114L), - changelogRow("+U", "Euro", 116L), - changelogRow("-D", "Euro", 116L), - changelogRow("+I", "Euro", 119L), - changelogRow("-D", "Yen", 1L))) - .close(); - - // test field filter - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.singletonList("currency"), - Collections.emptyList(), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, "*", "WHERE currency = 'Euro'", SCAN_LATEST), - Arrays.asList( - changelogRow("+I", "Euro", 114L), - changelogRow("-U", "Euro", 114L), - changelogRow("+U", "Euro", 116L), - changelogRow("-D", "Euro", 116L), - changelogRow("+I", "Euro", 119L))) - .close(); - - // test projection - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.singletonList("currency"), - Collections.emptyList(), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions(table, "currency", "", SCAN_LATEST), - Arrays.asList( - changelogRow("+I", "US Dollar"), - changelogRow("+I", "Euro"), - changelogRow("+I", "Yen"), - changelogRow("-D", "Euro"), - changelogRow("+I", "Euro"), - changelogRow("-D", "Yen"))) - .close(); - - // test projection and filter - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.singletonList("currency"), - Collections.emptyList(), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, "rate", "WHERE currency = 'Euro'", SCAN_LATEST), - Arrays.asList( - changelogRow("+I", 114L), - changelogRow("-U", 114L), - changelogRow("+U", 116L), - changelogRow("-D", 116L), - changelogRow("+I", 119L))) - .close(); - } - - @Test - public void testReadLatestChangelogOfInsertOnlyRecords() throws Exception { - List initialRecords = - Arrays.asList( - changelogRow("+I", "US Dollar", 102L), - changelogRow("+I", "Euro", 114L), - changelogRow("+I", "Yen", 1L), - changelogRow("+I", "Euro", 114L), - changelogRow("+I", "Euro", 119L)); - - // without pk - String temporaryTable = - createTemporaryTable( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.emptyList(), - Collections.emptyList(), - initialRecords, - null, - true, - "I"); - - String table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.emptyList(), - Collections.emptyList(), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions(table, "*", "", SCAN_LATEST), - initialRecords) - .close(); - - // currency as pk in the next tests - temporaryTable = - createTemporaryTable( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.singletonList("currency"), - Collections.emptyList(), - initialRecords, - null, - true, - "I"); - - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.singletonList("currency"), - Collections.emptyList(), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions(table, "*", "", SCAN_LATEST), - Arrays.asList( - changelogRow("+I", "US Dollar", 102L), - changelogRow("+I", "Euro", 114L), - changelogRow("+I", "Yen", 1L), - changelogRow("-U", "Euro", 114L), - changelogRow("+U", "Euro", 119L))) - .close(); - - // test field filter - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.singletonList("currency"), - Collections.emptyList(), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions(table, "*", "WHERE rate = 114", SCAN_LATEST), - Arrays.asList( - changelogRow("+I", "Euro", 114L), changelogRow("-U", "Euro", 114L))) - .close(); - - // test projection - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.singletonList("currency"), - Collections.emptyList(), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions(table, "rate", "", SCAN_LATEST), - Arrays.asList( - changelogRow("+I", 102L), - changelogRow("+I", 114L), - changelogRow("+I", 1L), - changelogRow("-U", 114L), - changelogRow("+U", 119L))) - .close(); - - // test projection and filter - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.singletonList("currency"), - Collections.emptyList(), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, "currency", "WHERE rate = 114", SCAN_LATEST), - Arrays.asList(changelogRow("+I", "Euro"), changelogRow("-U", "Euro"))) - .close(); - } - - // ---------------------------------------------------------------------------------------------------------------- - // Write First/Kafka log table auto created/scan.mode = from-timestamp - // ---------------------------------------------------------------------------------------------------------------- - - @Test - public void testReadInsertOnlyChangelogFromTimestamp() throws Exception { - // test records 0 - List initialRecords0 = - Arrays.asList( - // dt = 2022-01-01 - changelogRow("+I", "US Dollar", 102L, "2022-01-01"), - changelogRow("+I", "Euro", 114L, "2022-01-01"), - changelogRow("+I", "Yen", 1L, "2022-01-01"), - changelogRow("+I", "Euro", 114L, "2022-01-01"), - changelogRow("+I", "US Dollar", 114L, "2022-01-01"), - // dt = 2022-01-02 - changelogRow("+I", "Euro", 119L, "2022-01-02")); - - // partitioned without pk, scan from timestamp 0 - String temporaryTable = - createTemporaryTable( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), - Collections.emptyList(), - Collections.singletonList("dt"), - initialRecords0, - "dt:2022-01-01;dt:2022-01-02", - true, - "I"); - - String table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), - Collections.emptyList(), - Collections.singletonList("dt"), - false); - - insertIntoFromTable(temporaryTable, table); - - testStreamingRead( - buildQueryWithTableOptions(table, "*", "", scanFromTimeStampMillis(0L)), - initialRecords0) - .close(); - - // partitioned with pk, scan from timestamp 0 - temporaryTable = - createTemporaryTable( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), - Arrays.asList("currency", "dt"), - Collections.singletonList("dt"), - initialRecords0, - "dt:2022-01-01;dt:2022-01-02", - true, - "I"); - - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), - Arrays.asList("currency", "dt"), - Collections.singletonList("dt"), - false); - - insertIntoFromTable(temporaryTable, table); - - testStreamingRead( - buildQueryWithTableOptions(table, "*", "", scanFromTimeStampMillis(0L)), - Arrays.asList( - changelogRow("+I", "US Dollar", 102L, "2022-01-01"), - changelogRow("+I", "Yen", 1L, "2022-01-01"), - changelogRow("+I", "Euro", 114L, "2022-01-01"), - changelogRow("-U", "US Dollar", 102L, "2022-01-01"), - changelogRow("+U", "US Dollar", 114L, "2022-01-01"), - changelogRow("+I", "Euro", 119L, "2022-01-02"))) - .close(); - - // test records 1 - List initialRecords1 = - Arrays.asList( - changelogRow("+I", "US Dollar", 102L), - changelogRow("+I", "Euro", 114L), - changelogRow("+I", "Yen", 1L), - changelogRow("+I", "Euro", 114L), - changelogRow("+I", "Euro", 119L)); - - // non-partitioned with pk, scan from timestamp 0 - temporaryTable = - createTemporaryTable( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.singletonList("currency"), - Collections.emptyList(), - initialRecords1, - null, - true, - "I"); - - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.singletonList("currency"), - Collections.emptyList(), - false); - - insertIntoFromTable(temporaryTable, table); - - testStreamingRead( - buildQueryWithTableOptions(table, "*", "", scanFromTimeStampMillis(0L)), - Arrays.asList( - changelogRow("+I", "US Dollar", 102L), - changelogRow("+I", "Euro", 114L), - changelogRow("+I", "Yen", 1L), - changelogRow("-U", "Euro", 114L), - changelogRow("+U", "Euro", 119L))) - .close(); - - // non-partitioned without pk, scan from timestamp 0 - temporaryTable = - createTemporaryTable( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.emptyList(), - Collections.emptyList(), - initialRecords1, - null, - true, - "I"); - - table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.emptyList(), - Collections.emptyList(), - false); - - insertIntoFromTable(temporaryTable, table); - - testStreamingRead( - buildQueryWithTableOptions(table, "*", "", scanFromTimeStampMillis(0L)), - initialRecords1) - .close(); - } - - @Test - public void testReadInsertOnlyChangelogFromEnormousTimestamp() throws Exception { - List initialRecords = - Arrays.asList( - changelogRow("+I", "US Dollar", 102L), - changelogRow("+I", "Euro", 114L), - changelogRow("+I", "Yen", 1L), - changelogRow("+I", "Euro", 114L), - changelogRow("+I", "Euro", 119L)); - - // non-partitioned without pk, scan from timestamp Long.MAX_VALUE - 1 - String temporaryTable = - createTemporaryTable( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.emptyList(), - Collections.emptyList(), - initialRecords, - null, - true, - "I"); - - String table = - createTableWithKafkaLog( - Arrays.asList("currency STRING", "rate BIGINT"), - Collections.emptyList(), - Collections.emptyList(), - false); - - insertIntoFromTable(temporaryTable, table); - - testStreamingRead( - buildQueryWithTableOptions( - table, "*", "", scanFromTimeStampMillis(Long.MAX_VALUE - 1)), - Collections.emptyList()) - .close(); - } - - // ---------------------------------------------------------------------------------------------------------------- - // Tools - // ---------------------------------------------------------------------------------------------------------------- - - private Map scanFromTimeStampMillis(Long timeStampMillis) { - return new HashMap() { - { - put(SCAN_MODE.key(), CoreOptions.StartupMode.FROM_TIMESTAMP.toString()); - put(SCAN_TIMESTAMP_MILLIS.key(), String.valueOf(timeStampMillis)); - } - }; - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/StreamingWarehouseITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/StreamingWarehouseITCase.java deleted file mode 100644 index 4d308f54cc1b..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/StreamingWarehouseITCase.java +++ /dev/null @@ -1,245 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.kafka; - -import org.apache.paimon.utils.BlockingIterator; - -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.types.Row; -import org.junit.jupiter.api.Test; - -import java.time.LocalDateTime; -import java.util.function.Function; - -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.bEnv; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.init; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.sEnv; -import static org.assertj.core.api.Assertions.assertThat; - -/** Paimon IT case to test concurrent batch overwrite and streaming insert into. */ -public class StreamingWarehouseITCase extends KafkaTableTestBase { - - @Test - public void testUserStory() throws Exception { - init(createAndRegisterTempFile("").toString(), 1); - // Step1: define trade order table schema - String orderSource = - "CREATE TEMPORARY TABLE IF NOT EXISTS trade_orders (\n" - + " order_id BIGINT NOT NULL,\n" - + " order_timestamp AS LOCALTIMESTAMP,\n" - + " buyer_id STRING,\n" - + " order_amount DOUBLE,\n" - + " loyalty_discount DOUBLE,\n" - + " shipping_fee DOUBLE,\n" - + " order_verified BOOLEAN,\n" - + " PRIMARY KEY (order_id) NOT ENFORCED\n" - + " )\n" - + "WITH (\n" - + " 'connector' = 'datagen',\n" - + " 'rows-per-second' = '10',\n" - + " 'fields.order_id.kind' = 'random',\n" - + " 'fields.order_id.min' = '1',\n" - + " 'fields.buyer_id.kind' = 'random',\n" - + " 'fields.buyer_id.length' = '3',\n" - + " 'fields.order_amount.min' = '10',\n" - + " 'fields.order_amount.max' = '1000',\n" - + " 'fields.loyalty_discount.min' = '0',\n" - + " 'fields.loyalty_discount.max' = '10',\n" - + " 'fields.shipping_fee.min' = '5',\n" - + " 'fields.shipping_fee.max' = '20'\n" - + " );"; - - String cleanedOrders = - String.format( - "CREATE TABLE IF NOT EXISTS cleaned_trade_order (\n" - + " order_id BIGINT NOT NULL,\n" - + " order_timestamp TIMESTAMP (3),\n" - + " buyer_id STRING,\n" - + " order_amount DOUBLE,\n" - + " loyalty_discount DOUBLE,\n" - + " shipping_fee DOUBLE,\n" - + " order_verified BOOLEAN,\n" - + " actual_gmv DOUBLE,\n" - + " dt STRING,\n" - + " PRIMARY KEY (dt, order_id) NOT ENFORCED\n" - + " )\n" - + "PARTITIONED BY (dt)\n" - + "WITH (\n" - + " 'bucket' = '1',\n" - + " 'log.system' = 'kafka',\n" - + " 'kafka.bootstrap.servers' = '%s',\n" - + " 'kafka.topic' = 'cleaned_trade_order');", - getBootstrapServers()); - sEnv.executeSql(orderSource); - bEnv.executeSql(orderSource); - sEnv.executeSql(cleanedOrders); - - // Step2: batch write some corrupted historical data - String corruptedHistoricalData = - "INSERT INTO cleaned_trade_order\n" - + "PARTITION (dt = '2022-04-14')\n" - + "SELECT order_id,\n" - + " TIMESTAMPADD (\n" - + " HOUR,\n" - + " RAND_INTEGER (24),\n" - + " TO_TIMESTAMP ('2022-04-14', 'yyyy-MM-dd')\n" - + " ) AS order_timestamp,\n" - + " IF (\n" - + " order_verified\n" - + " AND order_id % 2 = 1,\n" - + " '404NotFound',\n" - + " buyer_id\n" - + " ) AS buyer_id,\n" // corrupt data conditionally - + " order_amount,\n" - + " loyalty_discount,\n" - + " shipping_fee,\n" - + " order_verified,\n" - + " IF (\n" - + " order_verified\n" - + " AND order_id % 2 = 1,\n" - + " -1,\n" - + " order_amount + shipping_fee - loyalty_discount\n" - + " ) AS actual_gmv\n" // corrupt data conditionally - + "FROM\n" - + " trade_orders\n" - + " /*+ OPTIONS ('number-of-rows' = '50') */"; - bEnv.executeSql(corruptedHistoricalData).await(); - - // Step3: start downstream streaming task to read - String streamingRead = "SELECT * FROM cleaned_trade_order"; - BlockingIterator streamIter = - BlockingIterator.of(sEnv.executeSql(streamingRead).collect(), ORDER_CONVERTER); - // verify historical data is corrupted - streamIter.collect(50).stream() - .filter(order -> order.orderVerified && order.orderId % 2 == 1) - .forEach( - order -> { - assertThat(order.buyerId).isEqualTo("404NotFound"); - assertThat(order.actualGmv).isEqualTo(-1); - assertThat(order.dt).isEqualTo("2022-04-14"); - }); - - // Step4: prepare day-to-day streaming sync task - String streamingWrite = - "INSERT INTO cleaned_trade_order\n" - + "SELECT order_id,\n" - + " order_timestamp,\n" - + " buyer_id,\n" - + " order_amount,\n" - + " loyalty_discount,\n" - + " shipping_fee,\n" - + " order_verified,\n" - + " order_amount + shipping_fee - loyalty_discount AS actual_gmv,\n" - + " DATE_FORMAT (order_timestamp, 'yyyy-MM-dd') AS dt\n" - + "FROM\n" - + " trade_orders"; - JobClient dailyTaskHandler = sEnv.executeSql(streamingWrite).getJobClient().get(); - while (true) { - if (dailyTaskHandler.getJobStatus().get() == JobStatus.RUNNING) { - break; - } - } - - // Step5: prepare back-fill task to correct historical data - String backFillOverwrite = - "INSERT OVERWRITE cleaned_trade_order\n" - + "SELECT order_id,\n" - + " order_timestamp,\n" - + " IF (buyer_id = '404NotFound', '_ANONYMOUS_USER_', buyer_id) AS buyer_id,\n" - + " order_amount,\n" - + " loyalty_discount,\n" - + " shipping_fee,\n" - + " order_verified,\n" - + " IF (\n" - + " actual_gmv = -1,\n" - + " order_amount + shipping_fee - loyalty_discount,\n" - + " actual_gmv\n" - + " ) AS actual_gmv,\n" - + " dt\n" - + "FROM\n" - + " cleaned_trade_order\n" - + "WHERE\n" - + " dt = '2022-04-14';"; - - // wait for back-fill task to finish - bEnv.executeSql(backFillOverwrite).await(); - - // Step6: check streaming read does not achieve any changelog - int checkSize = 200; - while (checkSize > 0) { - Thread.sleep(1000L); - streamIter - .collect(10) // rows-per-second is 10 - .forEach(order -> assertThat(order.dt).isGreaterThan("2022-04-14")); - checkSize -= 10; - } - - // verify corrupted historical data is corrected - BlockingIterator batchIter = - BlockingIterator.of( - bEnv.executeSql("SELECT * FROM cleaned_trade_order WHERE dt ='2022-04-14'") - .collect(), - ORDER_CONVERTER); - batchIter.collect(50).stream() - .filter(order -> order.orderVerified && order.orderId % 2 == 1) - .forEach( - order -> { - assertThat(order.buyerId).isEqualTo("_ANONYMOUS_USER_"); - assertThat(order.actualGmv) - .isEqualTo( - order.orderAmount - + order.shippingFee - - order.loyaltyDiscount); - assertThat(order.dt).isEqualTo("2022-04-14"); - }); - - streamIter.close(); - dailyTaskHandler.cancel().get(); - } - - private static final Function ORDER_CONVERTER = - (row) -> { - assert row != null && row.getArity() == 9; - CleanedTradeOrder order = new CleanedTradeOrder(); - order.orderId = (Long) row.getField(0); - order.orderTimestamp = (LocalDateTime) row.getField(1); - order.buyerId = (String) row.getField(2); - order.orderAmount = (Double) row.getField(3); - order.loyaltyDiscount = (Double) row.getField(4); - order.shippingFee = (Double) row.getField(5); - order.orderVerified = (Boolean) row.getField(6); - order.actualGmv = (Double) row.getField(7); - order.dt = (String) row.getField(8); - return order; - }; - - /** A test POJO. */ - private static class CleanedTradeOrder { - protected Long orderId; - protected LocalDateTime orderTimestamp; - protected String buyerId; - protected Double orderAmount; - protected Double loyaltyDiscount; - protected Double shippingFee; - protected Boolean orderVerified; - protected Double actualGmv; - protected String dt; - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/source/LogHybridSourceFactoryTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/source/LogHybridSourceFactoryTest.java deleted file mode 100644 index 5751495d4481..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/source/LogHybridSourceFactoryTest.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.source; - -import org.apache.paimon.CoreOptions; -import org.apache.paimon.flink.kafka.KafkaLogSourceProvider; -import org.apache.paimon.flink.kafka.KafkaLogTestUtils; - -import org.junit.jupiter.api.Test; - -import static org.apache.paimon.flink.kafka.KafkaLogTestUtils.discoverKafkaLogFactory; -import static org.apache.paimon.flink.kafka.KafkaLogTestUtils.testContext; - -/** Test for {@link LogHybridSourceFactory}. */ -public class LogHybridSourceFactoryTest { - - @Test - public void testRestoreLogSource() { - KafkaLogSourceProvider sourceProvider = - discoverKafkaLogFactory() - .createSourceProvider( - testContext("", CoreOptions.LogChangelogMode.UPSERT, true), - KafkaLogTestUtils.SOURCE_CONTEXT, - null); - LogHybridSourceFactory sourceFactory = new LogHybridSourceFactory(sourceProvider); - sourceFactory.create(() -> null); - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java index d623b7e933d6..2f9c2b3cda31 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java @@ -18,14 +18,10 @@ package org.apache.paimon.flink; -import org.apache.paimon.CoreOptions.LogChangelogMode; -import org.apache.paimon.CoreOptions.LogConsistency; -import org.apache.paimon.CoreOptions.StreamingReadMode; import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.flink.log.LogStoreTableFactory; import org.apache.paimon.flink.sink.FlinkFormatTableSink; import org.apache.paimon.flink.sink.FlinkTableSink; import org.apache.paimon.flink.source.DataTableSource; @@ -46,7 +42,6 @@ import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableConfig; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource; @@ -65,23 +60,13 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.Set; import java.util.regex.Pattern; import static java.lang.Boolean.parseBoolean; -import static org.apache.paimon.CoreOptions.LOG_CHANGELOG_MODE; -import static org.apache.paimon.CoreOptions.LOG_CONSISTENCY; -import static org.apache.paimon.CoreOptions.SCAN_MODE; -import static org.apache.paimon.CoreOptions.STREAMING_READ_MODE; -import static org.apache.paimon.CoreOptions.StartupMode.FROM_SNAPSHOT; -import static org.apache.paimon.CoreOptions.StartupMode.FROM_SNAPSHOT_FULL; import static org.apache.paimon.flink.FlinkConnectorOptions.FILESYSTEM_JOB_LEVEL_SETTINGS_ENABLED; -import static org.apache.paimon.flink.FlinkConnectorOptions.LOG_SYSTEM; -import static org.apache.paimon.flink.FlinkConnectorOptions.NONE; import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_BOUNDED; import static org.apache.paimon.flink.LogicalTypeConversion.toLogicalType; -import static org.apache.paimon.flink.log.LogStoreTableFactory.discoverLogStoreFactory; /** Abstract paimon factory to create table source and table sink. */ public abstract class AbstractFlinkTableFactory @@ -113,12 +98,7 @@ && parseBoolean(options.get(SCAN_BOUNDED.key()))) { if (origin instanceof SystemCatalogTable) { return new SystemTableSource(table, unbounded, context.getObjectIdentifier()); } else { - return new DataTableSource( - context.getObjectIdentifier(), - table, - unbounded, - context, - createOptionalLogStoreFactory(context).orElse(null)); + return new DataTableSource(context.getObjectIdentifier(), table, unbounded, context); } } @@ -129,11 +109,7 @@ public DynamicTableSink createDynamicTableSink(Context context) { return new FlinkFormatTableSink( context.getObjectIdentifier(), (FormatTable) table, context); } else { - return new FlinkTableSink( - context.getObjectIdentifier(), - table, - context, - createOptionalLogStoreFactory(context).orElse(null)); + return new FlinkTableSink(context.getObjectIdentifier(), table, context); } } @@ -149,50 +125,6 @@ public Set> optionalOptions() { // ~ Tools ------------------------------------------------------------------ - public static Optional createOptionalLogStoreFactory( - DynamicTableFactory.Context context) { - return createOptionalLogStoreFactory( - context.getClassLoader(), context.getCatalogTable().getOptions()); - } - - static Optional createOptionalLogStoreFactory( - ClassLoader classLoader, Map options) { - Options configOptions = new Options(); - options.forEach(configOptions::setString); - - if (configOptions.get(LOG_SYSTEM).equalsIgnoreCase(NONE)) { - // Use file store continuous reading - validateFileStoreContinuous(configOptions); - return Optional.empty(); - } else if (configOptions.get(SCAN_MODE) == FROM_SNAPSHOT - || configOptions.get(SCAN_MODE) == FROM_SNAPSHOT_FULL) { - throw new ValidationException( - String.format( - "Log system does not support %s and %s scan mode", - FROM_SNAPSHOT, FROM_SNAPSHOT_FULL)); - } - - return Optional.of(discoverLogStoreFactory(classLoader, configOptions.get(LOG_SYSTEM))); - } - - private static void validateFileStoreContinuous(Options options) { - LogChangelogMode changelogMode = options.get(LOG_CHANGELOG_MODE); - StreamingReadMode streamingReadMode = options.get(STREAMING_READ_MODE); - if (changelogMode == LogChangelogMode.UPSERT) { - throw new ValidationException( - "File store continuous reading does not support upsert changelog mode."); - } - LogConsistency consistency = options.get(LOG_CONSISTENCY); - if (consistency == LogConsistency.EVENTUAL) { - throw new ValidationException( - "File store continuous reading does not support eventual consistency mode."); - } - if (streamingReadMode == StreamingReadMode.LOG) { - throw new ValidationException( - "File store continuous reading does not support the log streaming read mode."); - } - } - static CatalogContext createCatalogContext(DynamicTableFactory.Context context) { return CatalogContext.create( Options.fromMap(context.getCatalogTable().getOptions()), new FlinkFileIOLoader()); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java index cd259903f21a..ff86a67162ce 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java @@ -22,7 +22,6 @@ import org.apache.paimon.TableType; import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.catalog.Catalog; -import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.catalog.Database; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.catalog.PropertyChange; @@ -117,7 +116,6 @@ import javax.annotation.Nullable; -import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -149,13 +147,9 @@ import static org.apache.paimon.catalog.Catalog.NUM_ROWS_PROP; import static org.apache.paimon.catalog.Catalog.TOTAL_SIZE_PROP; import static org.apache.paimon.flink.FlinkCatalogOptions.DISABLE_CREATE_TABLE_IN_DEFAULT_DB; -import static org.apache.paimon.flink.FlinkCatalogOptions.LOG_SYSTEM_AUTO_REGISTER; -import static org.apache.paimon.flink.FlinkCatalogOptions.REGISTER_TIMEOUT; import static org.apache.paimon.flink.LogicalTypeConversion.toBlobType; import static org.apache.paimon.flink.LogicalTypeConversion.toDataType; import static org.apache.paimon.flink.LogicalTypeConversion.toLogicalType; -import static org.apache.paimon.flink.log.LogStoreRegister.registerLogSystem; -import static org.apache.paimon.flink.log.LogStoreRegister.unRegisterLogSystem; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.SCHEMA; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.compoundKey; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn; @@ -184,9 +178,6 @@ public class FlinkCatalog extends AbstractCatalog { private final ClassLoader classLoader; private final Catalog catalog; private final String name; - private final boolean logStoreAutoRegister; - - private final Duration logStoreAutoRegisterTimeout; private final boolean disableCreateTableInDefaultDatabase; @@ -201,8 +192,6 @@ public FlinkCatalog( this.catalog = catalog; this.name = name; this.classLoader = classLoader; - this.logStoreAutoRegister = options.get(LOG_SYSTEM_AUTO_REGISTER); - this.logStoreAutoRegisterTimeout = options.get(REGISTER_TIMEOUT); this.disableCreateTableInDefaultDatabase = options.get(DISABLE_CREATE_TABLE_IN_DEFAULT_DB); if (!disableCreateTableInDefaultDatabase) { try { @@ -400,16 +389,7 @@ public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) try { Table table = null; - if (logStoreAutoRegister) { - try { - table = catalog.getTable(identifier); - } catch (Catalog.TableNotExistException ignored) { - } - } catalog.dropTable(toIdentifier(tablePath), ignoreIfNotExists); - if (logStoreAutoRegister && table != null) { - unRegisterLogSystem(identifier, table.options(), classLoader); - } } catch (Catalog.TableNotExistException e) { throw new TableNotExistException(getName(), tablePath); } @@ -429,28 +409,20 @@ public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ig return; } - Identifier identifier = toIdentifier(tablePath); // the returned value of "table.getOptions" may be unmodifiable (for example from // TableDescriptor) Map options = new HashMap<>(table.getOptions()); if (table instanceof CatalogMaterializedTable) { fillOptionsForMaterializedTable((CatalogMaterializedTable) table, options); } - Schema paimonSchema = buildPaimonSchema(identifier, table, options); + Schema paimonSchema = buildPaimonSchema(table, options); - boolean unRegisterLogSystem = false; try { - catalog.createTable(identifier, paimonSchema, ignoreIfExists); + catalog.createTable(toIdentifier(tablePath), paimonSchema, ignoreIfExists); } catch (Catalog.TableAlreadyExistException e) { - unRegisterLogSystem = true; throw new TableAlreadyExistException(getName(), tablePath); } catch (Catalog.DatabaseNotExistException e) { - unRegisterLogSystem = true; throw new DatabaseNotExistException(getName(), e.database()); - } finally { - if (logStoreAutoRegister && unRegisterLogSystem) { - unRegisterLogSystem(identifier, options, classLoader); - } } } @@ -519,8 +491,7 @@ private static void fillOptionsForMaterializedTable( options.putAll(mtOptions.toMap()); } - protected Schema buildPaimonSchema( - Identifier identifier, CatalogBaseTable catalogTable, Map options) { + protected Schema buildPaimonSchema(CatalogBaseTable catalogTable, Map options) { String connector = options.get(CONNECTOR.key()); options.remove(CONNECTOR.key()); if (!StringUtils.isNullOrWhitespaceOnly(connector) @@ -533,15 +504,6 @@ protected Schema buildPaimonSchema( + " You can create TEMPORARY table instead if you want to create the table of other connector."); } - if (logStoreAutoRegister) { - // Although catalog.createTable will copy the default options, but we need this info - // here before create table, such as table-default.kafka.bootstrap.servers defined in - // catalog options. Temporarily, we copy the default options here. - CatalogUtils.tableDefaultOptions(catalog.options()).forEach(options::putIfAbsent); - options.put(REGISTER_TIMEOUT.key(), logStoreAutoRegisterTimeout.toString()); - registerLogSystem(catalog, identifier, options, classLoader); - } - if (catalogTable instanceof CatalogTable) { return fromCatalogTable(((CatalogTable) catalogTable).copy(options)); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalogOptions.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalogOptions.java index 63ad4ddbe0ae..8f6214ee08d1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalogOptions.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalogOptions.java @@ -18,13 +18,10 @@ package org.apache.paimon.flink; -import org.apache.paimon.annotation.Documentation.ExcludeFromDocumentation; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.options.ConfigOption; import org.apache.paimon.options.ConfigOptions; -import java.time.Duration; - /** Options for flink catalog. */ public class FlinkCatalogOptions { @@ -33,24 +30,6 @@ public class FlinkCatalogOptions { .stringType() .defaultValue(Catalog.DEFAULT_DATABASE); - @ExcludeFromDocumentation("Confused without log system") - public static final ConfigOption LOG_SYSTEM_AUTO_REGISTER = - ConfigOptions.key("log.system.auto-register") - .booleanType() - .defaultValue(false) - .withDescription( - "If true, the register will automatically create and delete a topic in log system for Paimon table. Default kafka log store register " - + "is supported, users can implement customized register for log system, for example, create a new class which extends " - + "KafkaLogStoreFactory and return a customized LogStoreRegister for their kafka cluster to create/delete topics."); - - @ExcludeFromDocumentation("Confused without log system") - public static final ConfigOption REGISTER_TIMEOUT = - ConfigOptions.key("log.system.auto-register-timeout") - .durationType() - .defaultValue(Duration.ofMinutes(1)) - .withDescription( - "The timeout for register to create or delete topic in log system."); - public static final ConfigOption DISABLE_CREATE_TABLE_IN_DEFAULT_DB = ConfigOptions.key("disable-create-table-in-default-db") .booleanType() 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 a1089d991ae2..700ace3b48dc 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 @@ -20,22 +20,18 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.CoreOptions.ChangelogProducer; -import org.apache.paimon.CoreOptions.StreamingReadMode; import org.apache.paimon.annotation.Documentation.ExcludeFromDocumentation; import org.apache.paimon.options.ConfigOption; import org.apache.paimon.options.ConfigOptions; import org.apache.paimon.options.MemorySize; import org.apache.paimon.options.description.DescribedEnum; -import org.apache.paimon.options.description.Description; import org.apache.paimon.options.description.InlineElement; -import org.apache.paimon.options.description.TextElement; import java.lang.reflect.Field; import java.time.Duration; import java.util.ArrayList; import java.util.List; -import static org.apache.paimon.CoreOptions.STREAMING_READ_MODE; import static org.apache.paimon.options.ConfigOptions.key; import static org.apache.paimon.options.description.TextElement.text; @@ -48,48 +44,6 @@ public class FlinkConnectorOptions { public static final int MIN_CLUSTERING_SAMPLE_FACTOR = 20; - @ExcludeFromDocumentation("Confused without log system") - public static final ConfigOption LOG_SYSTEM = - ConfigOptions.key("log.system") - .stringType() - .defaultValue(NONE) - .withDescription( - Description.builder() - .text("The log system used to keep changes of the table.") - .linebreak() - .linebreak() - .text("Possible values:") - .linebreak() - .list( - TextElement.text( - "\"none\": No log system, the data is written only to file store," - + " and the streaming read will be directly read from the file store.")) - .list( - TextElement.text( - "\"kafka\": Kafka log system, the data is double written to file" - + " store and kafka, and the streaming read will be read from kafka. If streaming read from file, configures " - + STREAMING_READ_MODE.key() - + " to " - + StreamingReadMode.FILE.getValue() - + ".")) - .build()); - - @ExcludeFromDocumentation("Confused without log system") - public static final ConfigOption LOG_SYSTEM_PARTITIONS = - ConfigOptions.key("log.system.partitions") - .intType() - .defaultValue(1) - .withDescription( - "The number of partitions of the log system. If log system is kafka, this is kafka partitions."); - - @ExcludeFromDocumentation("Confused without log system") - public static final ConfigOption LOG_SYSTEM_REPLICATION = - ConfigOptions.key("log.system.replication") - .intType() - .defaultValue(1) - .withDescription( - "The number of replication of the log system. If log system is kafka, this is kafka replicationFactor."); - public static final ConfigOption SINK_PARALLELISM = ConfigOptions.key("sink.parallelism") .intType() diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java index 59fa394b4995..b807af5c86c2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java @@ -351,9 +351,9 @@ protected boolean buildForPostponeBucketCompaction( realTable.rowType(), blobAsDescriptor, table.catalogEnvironment().catalogContext()), - new RowDataChannelComputer(realTable.schema(), false), + new RowDataChannelComputer(realTable.schema()), null); - FixedBucketSink sink = new FixedBucketSink(realTable, null, null); + FixedBucketSink sink = new FixedBucketSink(realTable, null); DataStream written = sink.doWrite(partitioned, commitUser, partitioned.getParallelism()) .forward() @@ -365,7 +365,7 @@ protected boolean buildForPostponeBucketCompaction( dataStreams.add(sourcePair.getRight()); } - FixedBucketSink sink = new FixedBucketSink(fileStoreTable, null, null); + FixedBucketSink sink = new FixedBucketSink(fileStoreTable, null); DataStream dataStream = dataStreams.get(0); for (int i = 1; i < dataStreams.size(); i++) { dataStream = dataStream.union(dataStreams.get(i)); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RemoveUnexistingFilesAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RemoveUnexistingFilesAction.java index 996182e0b33c..c67ddd7ea372 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RemoveUnexistingFilesAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RemoveUnexistingFilesAction.java @@ -31,7 +31,6 @@ import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.CommitMessageImpl; import org.apache.paimon.table.sink.TableCommitImpl; -import org.apache.paimon.utils.Preconditions; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; @@ -180,9 +179,7 @@ public void processElement(StreamRecord record) throws Exception { new ArrayList<>(entry.getValue().values()), Collections.emptyList()), CompactIncrement.emptyIncrement()); - output.collect( - new StreamRecord<>( - new Committable(Long.MAX_VALUE, Committable.Kind.FILE, message))); + output.collect(new StreamRecord<>(new Committable(Long.MAX_VALUE, message))); for (String path : entry.getValue().keySet()) { output.collect(RESULT_SIDE_OUTPUT, new StreamRecord<>(path)); } @@ -213,16 +210,13 @@ public void open() throws Exception { } @Override - public void processElement(StreamRecord record) throws Exception { + public void processElement(StreamRecord record) { Committable committable = record.getValue(); - Preconditions.checkArgument( - committable.kind() == Committable.Kind.FILE, - "Committable has kind " + committable.kind() + ". This is unexpected!"); - commitMessages.add((CommitMessage) committable.wrappedCommittable()); + commitMessages.add(committable.commitMessage()); } @Override - public void endInput() throws Exception { + public void endInput() { try { commit.commit(Long.MAX_VALUE, commitMessages); } catch (Exception e) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/RemoveClusterBeforeFilesOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/RemoveClusterBeforeFilesOperator.java index 83b41940bc4e..985433ffef01 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/RemoveClusterBeforeFilesOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/RemoveClusterBeforeFilesOperator.java @@ -57,9 +57,7 @@ public void processElement(StreamRecord element) throws Exception { dataSplit.dataFiles(), Collections.emptyList(), Collections.emptyList())); - output.collect( - new StreamRecord<>( - new Committable(Long.MAX_VALUE, Committable.Kind.FILE, message))); + output.collect(new StreamRecord<>(new Committable(Long.MAX_VALUE, message))); } @Override @@ -69,9 +67,7 @@ public void endInput() throws Exception { private void emitDvIndexCommitMessages(long checkpointId) { if (dvCommitMessage != null) { - output.collect( - new StreamRecord<>( - new Committable(checkpointId, Committable.Kind.FILE, dvCommitMessage))); + output.collect(new StreamRecord<>(new Committable(checkpointId, dvCommitMessage))); } } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/RewriteIncrementalClusterCommittableOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/RewriteIncrementalClusterCommittableOperator.java index 01c2deba6466..e0eb2866791f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/RewriteIncrementalClusterCommittableOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/RewriteIncrementalClusterCommittableOperator.java @@ -67,11 +67,7 @@ public void open() throws Exception { @Override public void processElement(StreamRecord element) throws Exception { Committable committable = element.getValue(); - if (committable.kind() != Committable.Kind.FILE) { - output.collect(element); - } - - CommitMessageImpl message = (CommitMessageImpl) committable.wrappedCommittable(); + CommitMessageImpl message = (CommitMessageImpl) committable.commitMessage(); checkArgument(message.bucket() == 0); BinaryRow partition = message.partition(); partitionFiles @@ -107,9 +103,7 @@ protected void emitAll(long checkpointId) { table.coreOptions().bucket(), DataIncrement.emptyIncrement(), compactIncrement); - output.collect( - new StreamRecord<>( - new Committable(checkpointId, Committable.Kind.FILE, clusterMessage))); + output.collect(new StreamRecord<>(new Committable(checkpointId, clusterMessage))); } partitionFiles.clear(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperator.java index 827d79ca5aba..9f79b9fea862 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperator.java @@ -82,12 +82,7 @@ public void open() throws Exception { public void processElement(StreamRecord record) throws Exception { Committable committable = record.getValue(); checkpointId = Math.max(checkpointId, committable.checkpointId()); - if (committable.kind() != Committable.Kind.FILE) { - output.collect(new StreamRecord<>(Either.Left(committable))); - return; - } - - CommitMessageImpl message = (CommitMessageImpl) committable.wrappedCommittable(); + CommitMessageImpl message = (CommitMessageImpl) committable.commitMessage(); if (message.newFilesIncrement().newFiles().isEmpty()) { output.collect(new StreamRecord<>(Either.Left(committable))); return; @@ -133,8 +128,7 @@ public void processElement(StreamRecord record) throws Exception { message.newFilesIncrement().deletedIndexFiles()), message.compactIncrement()); if (!newMessage.isEmpty()) { - Committable newCommittable = - new Committable(committable.checkpointId(), Committable.Kind.FILE, newMessage); + Committable newCommittable = new Committable(committable.checkpointId(), newMessage); output.collect(new StreamRecord<>(Either.Left(newCommittable))); } } @@ -171,10 +165,7 @@ private void emitAll() { Collections.emptyList()), CompactIncrement.emptyIncrement()); output.collect( - new StreamRecord<>( - Either.Left( - new Committable( - checkpointId, Committable.Kind.FILE, message)))); + new StreamRecord<>(Either.Left(new Committable(checkpointId, message)))); } } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactWorkerOperator.java index 0e71f9186546..c50abcc3728d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactWorkerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactWorkerOperator.java @@ -87,9 +87,7 @@ public void processElement( } else { long checkpointId = record.getValue().right().f0; CommitMessage message = doCompact(record.getValue().right().f1); - output.collect( - new StreamRecord<>( - new Committable(checkpointId, Committable.Kind.FILE, message))); + output.collect(new StreamRecord<>(new Committable(checkpointId, message))); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendTableCompactor.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendTableCompactor.java index 0a455146698a..1e9e8ee2873a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendTableCompactor.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendTableCompactor.java @@ -199,7 +199,7 @@ public List prepareCommit(boolean waitCompaction, long checkpointId tempList.add(future.get()); } return tempList.stream() - .map(s -> new Committable(checkpointId, Committable.Kind.FILE, s)) + .map(s -> new Committable(checkpointId, s)) .collect(Collectors.toList()); } catch (InterruptedException e) { throw new RuntimeException("Interrupted while waiting tasks done.", e); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/IncrementalClusterCompact.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/IncrementalClusterCompact.java index c7fef7026689..18eb84a68d4d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/IncrementalClusterCompact.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/IncrementalClusterCompact.java @@ -150,7 +150,7 @@ protected List> buildCompactOperator( // 2.3 write and then reorganize the committable // set parallelism to null, and it'll forward parallelism when doWrite() - RowAppendTableSink sink = new RowAppendTableSink(table, null, null, null); + RowAppendTableSink sink = new RowAppendTableSink(table, null, null); boolean blobAsDescriptor = table.coreOptions().blobAsDescriptor(); DataStream written = sink.doWrite( @@ -184,7 +184,7 @@ protected List> buildCompactOperator( } protected void buildCommitOperator(List> dataStreams) { - RowAppendTableSink sink = new RowAppendTableSink(table, null, null, null); + RowAppendTableSink sink = new RowAppendTableSink(table, null, null); DataStream dataStream = dataStreams.get(0); for (int i = 1; i < dataStreams.size(); i++) { dataStream = dataStream.union(dataStreams.get(i)); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactCoordinateOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactCoordinateOperator.java index d33671703164..d5294742c4ff 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactCoordinateOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactCoordinateOperator.java @@ -74,12 +74,7 @@ public void open() throws Exception { public void processElement(StreamRecord record) { Committable committable = record.getValue(); checkpointId = Math.max(checkpointId, committable.checkpointId()); - if (committable.kind() != Committable.Kind.FILE) { - output.collect(new StreamRecord<>(Either.Left(record.getValue()))); - return; - } - - CommitMessageImpl message = (CommitMessageImpl) committable.wrappedCommittable(); + CommitMessageImpl message = (CommitMessageImpl) committable.commitMessage(); if (message.newFilesIncrement().changelogFiles().isEmpty() && message.compactIncrement().changelogFiles().isEmpty()) { output.collect(new StreamRecord<>(Either.Left(record.getValue()))); @@ -147,8 +142,7 @@ public void processElement(StreamRecord record) { skippedCompactChangelogs, message.compactIncrement().newIndexFiles(), message.compactIncrement().deletedIndexFiles())); - Committable newCommittable = - new Committable(committable.checkpointId(), Committable.Kind.FILE, newMessage); + Committable newCommittable = new Committable(committable.checkpointId(), newMessage); output.collect(new StreamRecord<>(Either.Left(newCommittable))); } @@ -203,8 +197,7 @@ private void emitPartitionChangelogCompactTask(BinaryRow partition) { Collections.emptyList(), entry.getValue())); } - Committable newCommittable = - new Committable(checkpointId, Committable.Kind.FILE, message); + Committable newCommittable = new Committable(checkpointId, message); output.collect(new StreamRecord<>(Either.Left(newCommittable))); } else { output.collect( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactSortOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactSortOperator.java index e6183ccb8a51..7317a26b1697 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactSortOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactSortOperator.java @@ -59,12 +59,7 @@ public void open() { @Override public void processElement(StreamRecord record) throws Exception { Committable committable = record.getValue(); - if (committable.kind() != Committable.Kind.FILE) { - output.collect(record); - return; - } - - CommitMessageImpl message = (CommitMessageImpl) committable.wrappedCommittable(); + CommitMessageImpl message = (CommitMessageImpl) committable.commitMessage(); if (message.newFilesIncrement().changelogFiles().isEmpty() && message.compactIncrement().changelogFiles().isEmpty()) { output.collect(record); @@ -104,8 +99,7 @@ public void processElement(StreamRecord record) throws Exception { message.compactIncrement().newIndexFiles(), message.compactIncrement().deletedIndexFiles())); if (!newMessage.isEmpty()) { - Committable newCommittable = - new Committable(committable.checkpointId(), Committable.Kind.FILE, newMessage); + Committable newCommittable = new Committable(committable.checkpointId(), newMessage); output.collect(new StreamRecord<>(newCommittable)); } } @@ -142,8 +136,7 @@ private void emitAll(long checkpointId) { Collections.emptyList(), sortedChangelogs( compactChangelogFiles, partition, bucket))); - Committable newCommittable = - new Committable(checkpointId, Committable.Kind.FILE, newMessage); + Committable newCommittable = new Committable(checkpointId, newMessage); output.collect(new StreamRecord<>(newCommittable)); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTask.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTask.java index 4f79b467e028..eb1b5b322e94 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTask.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTask.java @@ -280,7 +280,7 @@ private List produceNewCommittables( Collections.emptyList(), Collections.emptyList(), compactChangelog)); - newCommittables.add(new Committable(checkpointId, Committable.Kind.FILE, newMessage)); + newCommittables.add(new Committable(checkpointId, newMessage)); } return newCommittables; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/factories/FlinkFactoryUtil.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/factories/FlinkFactoryUtil.java deleted file mode 100644 index fa56becbdce0..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/factories/FlinkFactoryUtil.java +++ /dev/null @@ -1,498 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.factories; - -import org.apache.paimon.flink.log.LogStoreTableFactory; - -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.DelegatingConfiguration; -import org.apache.flink.configuration.FallbackKey; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.TableException; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.format.EncodingFormat; -import org.apache.flink.table.factories.DecodingFormatFactory; -import org.apache.flink.table.factories.DynamicTableFactory; -import org.apache.flink.table.factories.DynamicTableFactory.Context; -import org.apache.flink.table.factories.EncodingFormatFactory; -import org.apache.flink.table.factories.Factory; -import org.apache.flink.table.factories.FormatFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.ServiceLoader; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import java.util.stream.StreamSupport; - -import static org.apache.flink.configuration.ConfigurationUtils.canBePrefixMap; -import static org.apache.flink.configuration.ConfigurationUtils.filterPrefixMapKey; - -/** Utility for working with {@link Factory}s. */ -public final class FlinkFactoryUtil { - - private static final Logger LOG = LoggerFactory.getLogger(FlinkFactoryUtil.class); - - public static final ConfigOption FORMAT = - ConfigOptions.key("format") - .stringType() - .noDefaultValue() - .withDescription( - "Defines the format identifier for encoding data. " - + "The identifier is used to discover a suitable format factory."); - - /** - * Suffix for keys of {@link ConfigOption} in case a connector requires multiple formats (e.g. - * for both key and value). - * - *

See {@link #createFlinkTableFactoryHelper(LogStoreTableFactory, Context)} Context)} for - * more information. - */ - public static final String FORMAT_SUFFIX = ".format"; - - /** - * Creates a utility that helps in discovering formats, merging options with {@link - * DynamicTableFactory.Context#getEnrichmentOptions()} and validating them all for a {@link - * LogStoreTableFactory}. - * - *

The following example sketches the usage: - * - *

{@code
-     * // in createDynamicTableSource()
-     * helper = FlinkFactoryUtil.createFlinkTableFactoryHelper(this, context);
-     *
-     * keyFormat = helper.discoverDecodingFormat(DeserializationFormatFactory.class, KEY_FORMAT);
-     * valueFormat = helper.discoverDecodingFormat(DeserializationFormatFactory.class, VALUE_FORMAT);
-     *
-     * helper.validate();
-     *
-     * ... // construct connector with discovered formats
-     * }
- */ - public static FlinkTableFactoryHelper createFlinkTableFactoryHelper( - LogStoreTableFactory factory, DynamicTableFactory.Context context) { - return new FlinkTableFactoryHelper(factory, context); - } - - /** Discovers a flink Factory using the given factory base class and identifier. */ - @SuppressWarnings("unchecked") - public static T discoverFlinkFactory( - ClassLoader classLoader, Class factoryClass, String factoryIdentifier) { - final List factories = discoverFlinkFactories(classLoader); - - final List foundFactories = - factories.stream() - .filter(f -> factoryClass.isAssignableFrom(f.getClass())) - .collect(Collectors.toList()); - - if (foundFactories.isEmpty()) { - throw new ValidationException( - String.format( - "Could not find any factories that implement '%s' in the classpath.", - factoryClass.getName())); - } - - final List matchingFactories = - foundFactories.stream() - .filter(f -> f.factoryIdentifier().equals(factoryIdentifier)) - .collect(Collectors.toList()); - - if (matchingFactories.isEmpty()) { - throw new ValidationException( - String.format( - "Could not find any factory for identifier '%s' that implements '%s' in the classpath.\n\n" - + "Available factory identifiers are:\n\n" - + "%s", - factoryIdentifier, - factoryClass.getName(), - foundFactories.stream() - .map(Factory::factoryIdentifier) - .distinct() - .sorted() - .collect(Collectors.joining("\n")))); - } - if (matchingFactories.size() > 1) { - throw new ValidationException( - String.format( - "Multiple factories for identifier '%s' that implement '%s' found in the classpath.\n\n" - + "Ambiguous factory classes are:\n\n" - + "%s", - factoryIdentifier, - factoryClass.getName(), - matchingFactories.stream() - .map(f -> f.getClass().getName()) - .sorted() - .collect(Collectors.joining("\n")))); - } - - return (T) matchingFactories.get(0); - } - - /** Returns the required option prefix for options of the given format. */ - public static String getFormatPrefix( - ConfigOption formatOption, String formatIdentifier) { - final String formatOptionKey = formatOption.key(); - if (formatOptionKey.equals(FORMAT.key())) { - return formatIdentifier + "."; - } else if (formatOptionKey.endsWith(FORMAT_SUFFIX)) { - // extract the key prefix, e.g. extract 'key' from 'key.format' - String keyPrefix = - formatOptionKey.substring(0, formatOptionKey.length() - FORMAT_SUFFIX.length()); - return keyPrefix + "." + formatIdentifier + "."; - } else { - throw new ValidationException( - "Format identifier key should be 'format' or suffix with '.format', " - + "don't support format identifier key '" - + formatOptionKey - + "'."); - } - } - - // -------------------------------------------------------------------------------------------- - // Helper methods - // -------------------------------------------------------------------------------------------- - - static List discoverFlinkFactories(ClassLoader classLoader) { - final Iterator serviceLoaderIterator = - ServiceLoader.load(Factory.class, classLoader).iterator(); - - final List loadResults = new ArrayList<>(); - while (true) { - try { - // error handling should also be applied to the hasNext() call because service - // loading might cause problems here as well - if (!serviceLoaderIterator.hasNext()) { - break; - } - - loadResults.add(serviceLoaderIterator.next()); - } catch (Throwable t) { - if (t instanceof NoClassDefFoundError) { - LOG.debug( - "NoClassDefFoundError when loading a " - + LogStoreTableFactory.class.getCanonicalName() - + ". This is expected when trying to load a format dependency but no flink-connector-files is loaded.", - t); - } else { - throw new TableException( - "Unexpected error when trying to load service provider.", t); - } - } - } - - return loadResults; - } - - private static Set allKeysExpanded(ConfigOption option, Set actualKeys) { - return allKeysExpanded("", option, actualKeys); - } - - private static Set allKeysExpanded( - String prefix, ConfigOption option, Set actualKeys) { - final Set staticKeys = - allKeys(option).map(k -> prefix + k).collect(Collectors.toSet()); - if (!canBePrefixMap(option)) { - return staticKeys; - } - // include all prefix keys of a map option by considering the actually provided keys - return Stream.concat( - staticKeys.stream(), - staticKeys.stream() - .flatMap( - k -> - actualKeys.stream() - .filter(c -> filterPrefixMapKey(k, c)))) - .collect(Collectors.toSet()); - } - - private static Stream allKeys(ConfigOption option) { - return Stream.concat(Stream.of(option.key()), fallbackKeys(option)); - } - - private static Stream fallbackKeys(ConfigOption option) { - return StreamSupport.stream(option.fallbackKeys().spliterator(), false) - .map(FallbackKey::getKey); - } - - private static Stream deprecatedKeys(ConfigOption option) { - return StreamSupport.stream(option.fallbackKeys().spliterator(), false) - .filter(FallbackKey::isDeprecated) - .map(FallbackKey::getKey); - } - - /** Base flink helper utility for validating all options for a {@link LogStoreTableFactory}. */ - public static class FlinkFactoryHelper { - - protected final F factory; - - protected final Configuration allOptions; - - protected final Set consumedOptionKeys; - - protected final Set deprecatedOptionKeys; - - public FlinkFactoryHelper( - F factory, Map configuration, ConfigOption... implicitOptions) { - this.factory = factory; - this.allOptions = Configuration.fromMap(configuration); - - final List> consumedOptions = new ArrayList<>(); - consumedOptions.addAll(Arrays.asList(implicitOptions)); - - consumedOptionKeys = - consumedOptions.stream() - .flatMap( - option -> allKeysExpanded(option, allOptions.keySet()).stream()) - .collect(Collectors.toSet()); - - deprecatedOptionKeys = - consumedOptions.stream() - .flatMap(FlinkFactoryUtil::deprecatedKeys) - .collect(Collectors.toSet()); - } - - /** Returns all options currently being consumed by the factory. */ - public ReadableConfig getOptions() { - return allOptions; - } - } - - /** - * Helper utility for discovering formats and validating all options for a {@link - * DynamicTableFactory}. - * - * @see #createFlinkTableFactoryHelper(LogStoreTableFactory, Context) - */ - public static class FlinkTableFactoryHelper extends FlinkFactoryHelper { - - private final Context context; - - private final Configuration enrichingOptions; - - private FlinkTableFactoryHelper(LogStoreTableFactory tableFactory, Context context) { - super(tableFactory, context.getCatalogTable().getOptions()); - this.context = context; - this.enrichingOptions = Configuration.fromMap(context.getEnrichmentOptions()); - } - - /** - * Returns all options currently being consumed by the factory. This method returns the - * options already merged with {@link Context#getEnrichmentOptions()}, using {@link - * DynamicTableFactory#forwardOptions()} as reference of mergeable options. - */ - @Override - public ReadableConfig getOptions() { - return super.getOptions(); - } - - /** - * Discovers a {@link DecodingFormat} of the given type using the given option as factory - * identifier. - */ - public > DecodingFormat discoverDecodingFormat( - Class formatFactoryClass, ConfigOption formatOption) { - return discoverOptionalDecodingFormat(formatFactoryClass, formatOption) - .orElseThrow( - () -> - new ValidationException( - String.format( - "Could not find required scan format '%s'.", - formatOption.key()))); - } - - /** - * Discovers a {@link DecodingFormat} of the given type using the given option (if present) - * as factory identifier. - */ - public > - Optional> discoverOptionalDecodingFormat( - Class formatFactoryClass, ConfigOption formatOption) { - return discoverOptionalFormatFactory(formatFactoryClass, formatOption) - .map( - formatFactory -> { - String formatPrefix = - formatFlinkPrefix(formatFactory, formatOption); - try { - return formatFactory.createDecodingFormat( - context, - createFormatOptions(formatPrefix, formatFactory)); - } catch (Throwable t) { - throw new ValidationException( - String.format( - "Error creating scan format '%s' in option space '%s'.", - formatFactory.factoryIdentifier(), - formatPrefix), - t); - } - }); - } - - /** - * Discovers a {@link EncodingFormat} of the given type using the given option as factory - * identifier. - */ - public > EncodingFormat discoverEncodingFormat( - Class formatFactoryClass, ConfigOption formatOption) { - return discoverOptionalEncodingFormat(formatFactoryClass, formatOption) - .orElseThrow( - () -> - new ValidationException( - String.format( - "Could not find required sink format '%s'.", - formatOption.key()))); - } - - /** - * Discovers a {@link EncodingFormat} of the given type using the given option (if present) - * as factory identifier. - */ - public > - Optional> discoverOptionalEncodingFormat( - Class formatFactoryClass, ConfigOption formatOption) { - return discoverOptionalFormatFactory(formatFactoryClass, formatOption) - .map( - formatFactory -> { - String formatPrefix = - formatFlinkPrefix(formatFactory, formatOption); - try { - return formatFactory.createEncodingFormat( - context, - createFormatOptions(formatPrefix, formatFactory)); - } catch (Throwable t) { - throw new ValidationException( - String.format( - "Error creating sink format '%s' in option space '%s'.", - formatFactory.factoryIdentifier(), - formatPrefix), - t); - } - }); - } - - // ---------------------------------------------------------------------------------------- - - private Optional discoverOptionalFormatFactory( - Class formatFactoryClass, ConfigOption formatOption) { - final String identifier = allOptions.get(formatOption); - checkFormatIdentifierMatchesWithEnrichingOptions(formatOption, identifier); - if (identifier == null) { - return Optional.empty(); - } - final F factory = - discoverFlinkFactory(context.getClassLoader(), formatFactoryClass, identifier); - String formatPrefix = formatFlinkPrefix(factory, formatOption); - - // log all used options of other factories - final List> consumedOptions = new ArrayList<>(); - consumedOptions.addAll(factory.requiredOptions()); - consumedOptions.addAll(factory.optionalOptions()); - - consumedOptions.stream() - .flatMap( - option -> - allKeysExpanded(formatPrefix, option, allOptions.keySet()) - .stream()) - .forEach(consumedOptionKeys::add); - - consumedOptions.stream() - .flatMap(FlinkFactoryUtil::deprecatedKeys) - .map(k -> formatPrefix + k) - .forEach(deprecatedOptionKeys::add); - - return Optional.of(factory); - } - - private String formatFlinkPrefix(Factory formatFactory, ConfigOption formatOption) { - String identifier = formatFactory.factoryIdentifier(); - return getFormatPrefix(formatOption, identifier); - } - - @SuppressWarnings({"unchecked"}) - private ReadableConfig createFormatOptions( - String formatPrefix, FormatFactory formatFactory) { - Set> forwardableConfigOptions = formatFactory.forwardOptions(); - Configuration formatConf = new DelegatingConfiguration(allOptions, formatPrefix); - if (forwardableConfigOptions.isEmpty()) { - return formatConf; - } - - Configuration formatConfFromEnrichingOptions = - new DelegatingConfiguration(enrichingOptions, formatPrefix); - - for (ConfigOption option : forwardableConfigOptions) { - formatConfFromEnrichingOptions - .getOptional(option) - .ifPresent(o -> formatConf.set((ConfigOption) option, o)); - } - - return formatConf; - } - - /** - * This function assumes that the format config is used only and only if the original - * configuration contains the format config option. It will fail if there is a mismatch of - * the identifier between the format in the plan table map and the one in enriching table - * map. - */ - private void checkFormatIdentifierMatchesWithEnrichingOptions( - ConfigOption formatOption, String identifierFromPlan) { - Optional identifierFromEnrichingOptions = - enrichingOptions.getOptional(formatOption); - - if (!identifierFromEnrichingOptions.isPresent()) { - return; - } - - if (identifierFromPlan == null) { - throw new ValidationException( - String.format( - "The persisted plan has no format option '%s' specified, while the catalog table has it with value '%s'. " - + "This is invalid, as either only the persisted plan table defines the format, " - + "or both the persisted plan table and the catalog table defines the same format.", - formatOption, identifierFromEnrichingOptions.get())); - } - - if (!Objects.equals(identifierFromPlan, identifierFromEnrichingOptions.get())) { - throw new ValidationException( - String.format( - "Both persisted plan table and catalog table define the format option '%s', " - + "but they mismatch: '%s' != '%s'.", - formatOption, - identifierFromPlan, - identifierFromEnrichingOptions.get())); - } - } - } - // -------------------------------------------------------------------------------------------- - - private FlinkFactoryUtil() { - // no instantiation - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogSinkProvider.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogSinkProvider.java deleted file mode 100644 index 0869d6976abf..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogSinkProvider.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.log; - -import org.apache.paimon.flink.sink.LogSinkFunction; - -import java.io.Serializable; - -/** A {@link Serializable} sink provider for log store. */ -public interface LogSinkProvider extends Serializable { - - /** Creates a {@link LogSinkFunction} instance. */ - LogSinkFunction createSink(); -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogSourceProvider.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogSourceProvider.java deleted file mode 100644 index 5a780fb638b7..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogSourceProvider.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.log; - -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.connector.base.source.hybrid.HybridSource; -import org.apache.flink.table.data.RowData; - -import javax.annotation.Nullable; - -import java.io.Serializable; -import java.util.Map; - -/** - * A {@link Serializable} source provider for log store. - * - *

This class is serializable, it can be wrapped as the {@link HybridSource.SourceFactory}. - */ -public interface LogSourceProvider extends Serializable { - - /** - * Creates a {@link Source} instance. - * - * @param bucketOffsets optional, configure if you need to specify the startup offset. - */ - Source createSource(@Nullable Map bucketOffsets); - - /** - * Do pre-operations before log {@link Source} creation if you need, like system properties - * setting before job submitting, for the log {@link Source} inside of {@link HybridSource} will - * be created during job running. - */ - void preCreateSource(); -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogStoreRegister.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogStoreRegister.java deleted file mode 100644 index ad501e204ce6..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogStoreRegister.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.log; - -import org.apache.paimon.catalog.Catalog; -import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.factories.FactoryUtil; -import org.apache.paimon.options.Options; - -import java.util.Map; - -import static org.apache.paimon.flink.FlinkConnectorOptions.LOG_SYSTEM; -import static org.apache.paimon.flink.FlinkConnectorOptions.NONE; - -/** - * {@link LogStoreRegister} will register and unregister topic for a Paimon table, you can implement - * it for customized log system management. - */ -public interface LogStoreRegister { - /** Register topic in log system for the table. */ - Map registerTopic(); - - /** Unregister topic in log system for the table. */ - void unRegisterTopic(); - - static void registerLogSystem( - Catalog catalog, - Identifier identifier, - Map options, - ClassLoader classLoader) { - Options tableOptions = Options.fromMap(options); - String logStore = tableOptions.get(LOG_SYSTEM); - if (!tableOptions.get(LOG_SYSTEM).equalsIgnoreCase(NONE)) { - try { - catalog.getTable(identifier); - } catch (Catalog.TableNotExistException e) { - LogStoreRegister logStoreRegister = - getLogStoreRegister(identifier, classLoader, tableOptions, logStore); - options.putAll(logStoreRegister.registerTopic()); - } - } - } - - static void unRegisterLogSystem( - Identifier identifier, Map options, ClassLoader classLoader) { - Options tableOptions = Options.fromMap(options); - String logStore = tableOptions.get(LOG_SYSTEM); - if (!tableOptions.get(LOG_SYSTEM).equalsIgnoreCase(NONE)) { - LogStoreRegister logStoreRegister = - getLogStoreRegister(identifier, classLoader, tableOptions, logStore); - logStoreRegister.unRegisterTopic(); - } - } - - static LogStoreRegister getLogStoreRegister( - Identifier identifier, ClassLoader classLoader, Options tableOptions, String logStore) { - LogStoreTableFactory registerFactory = - FactoryUtil.discoverFactory(classLoader, LogStoreTableFactory.class, logStore); - return registerFactory.createRegister( - new LogStoreTableFactory.RegisterContext() { - @Override - public Options getOptions() { - return tableOptions; - } - - @Override - public Identifier getIdentifier() { - return identifier; - } - }); - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogStoreTableFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogStoreTableFactory.java deleted file mode 100644 index 4b15dc8369fd..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogStoreTableFactory.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.log; - -import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.factories.Factory; -import org.apache.paimon.factories.FactoryUtil; -import org.apache.paimon.flink.factories.FlinkFactoryUtil.FlinkTableFactoryHelper; -import org.apache.paimon.options.Options; - -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.format.EncodingFormat; -import org.apache.flink.table.connector.format.Format; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.DeserializationFormatFactory; -import org.apache.flink.table.factories.DynamicTableFactory.Context; -import org.apache.flink.table.factories.SerializationFormatFactory; -import org.apache.flink.types.RowKind; - -import javax.annotation.Nullable; - -import static org.apache.paimon.CoreOptions.LOG_FORMAT; -import static org.apache.paimon.CoreOptions.LOG_IGNORE_DELETE; -import static org.apache.paimon.CoreOptions.LOG_KEY_FORMAT; - -/** - * Base interface for configuring a default log table connector. The log table is used by managed - * table factory. - * - *

Log tables are for processing only unbounded data. Support streaming reading and streaming - * writing. - */ -public interface LogStoreTableFactory extends Factory { - - /** - * Creates a {@link LogSourceProvider} instance from a {@link CatalogTable} and additional - * context information. - */ - LogSourceProvider createSourceProvider( - Context context, - DynamicTableSource.Context sourceContext, - @Nullable int[][] projectFields); - - /** - * Creates a {@link LogSinkProvider} instance from a {@link CatalogTable} and additional context - * information. - */ - LogSinkProvider createSinkProvider(Context context, DynamicTableSink.Context sinkContext); - - /** - * Creates a {@link LogStoreRegister} instance for table ddl, it will register table to log - * store when a table is created or dropped. - */ - LogStoreRegister createRegister(RegisterContext context); - - /** Context to create log store register. */ - interface RegisterContext { - /** Options for the table. */ - Options getOptions(); - - /** Identifier for the table. */ - Identifier getIdentifier(); - } - - // -------------------------------------------------------------------------------------------- - - static ConfigOption logKeyFormat() { - return ConfigOptions.key(LOG_KEY_FORMAT.key()) - .stringType() - .defaultValue(LOG_KEY_FORMAT.defaultValue()); - } - - static ConfigOption logFormat() { - return ConfigOptions.key(LOG_FORMAT.key()) - .stringType() - .defaultValue(LOG_FORMAT.defaultValue()); - } - - static ConfigOption logIgnoreDelete() { - return ConfigOptions.key(LOG_IGNORE_DELETE.key()) - .booleanType() - .defaultValue(LOG_IGNORE_DELETE.defaultValue()); - } - - static LogStoreTableFactory discoverLogStoreFactory(ClassLoader cl, String identifier) { - return FactoryUtil.discoverFactory(cl, LogStoreTableFactory.class, identifier); - } - - static DecodingFormat> getKeyDecodingFormat( - FlinkTableFactoryHelper helper) { - DecodingFormat> format = - helper.discoverDecodingFormat(DeserializationFormatFactory.class, logKeyFormat()); - validateKeyFormat(format, helper.getOptions().get(logKeyFormat())); - return format; - } - - static EncodingFormat> getKeyEncodingFormat( - FlinkTableFactoryHelper helper) { - EncodingFormat> format = - helper.discoverEncodingFormat(SerializationFormatFactory.class, logKeyFormat()); - validateKeyFormat(format, helper.getOptions().get(logKeyFormat())); - return format; - } - - static DecodingFormat> getValueDecodingFormat( - FlinkTableFactoryHelper helper, boolean hasPrimaryKey) { - DecodingFormat> format = - helper.discoverDecodingFormat(DeserializationFormatFactory.class, logFormat()); - boolean insertOnly = !hasPrimaryKey || helper.getOptions().get(logIgnoreDelete()); - validateValueFormat(format, helper.getOptions().get(logFormat()), insertOnly); - return format; - } - - static EncodingFormat> getValueEncodingFormat( - FlinkTableFactoryHelper helper, boolean hasPrimaryKey) { - EncodingFormat> format = - helper.discoverEncodingFormat(SerializationFormatFactory.class, logFormat()); - boolean insertOnly = !hasPrimaryKey || helper.getOptions().get(logIgnoreDelete()); - validateValueFormat(format, helper.getOptions().get(logFormat()), insertOnly); - return format; - } - - static void validateKeyFormat(Format format, String name) { - if (!format.getChangelogMode().containsOnly(RowKind.INSERT)) { - throw new ValidationException( - String.format( - "A key format should only deal with INSERT-only records. " - + "But %s has a changelog mode of %s.", - name, format.getChangelogMode())); - } - } - - static void validateValueFormat(Format format, String name, boolean insertOnly) { - if (!insertOnly && !format.getChangelogMode().equals(ChangelogMode.all())) { - throw new ValidationException( - String.format( - "A value format should deal with all records. " - + "But %s has a changelog mode of %s.", - name, format.getChangelogMode())); - } - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogWriteCallback.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogWriteCallback.java deleted file mode 100644 index 067c82bbac5d..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogWriteCallback.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.log; - -import org.apache.paimon.flink.sink.LogSinkFunction.WriteCallback; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.LongAccumulator; - -/** A {@link WriteCallback} implementation. */ -public class LogWriteCallback implements WriteCallback { - - private final ConcurrentHashMap offsetMap = new ConcurrentHashMap<>(); - - @Override - public void onCompletion(int bucket, long offset) { - LongAccumulator acc = offsetMap.get(bucket); - if (acc == null) { - // computeIfAbsent will lock on the key - acc = offsetMap.computeIfAbsent(bucket, k -> new LongAccumulator(Long::max, 0)); - } // else lock free - - // Save the next offset, what we need to provide to the hybrid reading is the starting - // offset of the next transaction - acc.accumulate(offset + 1); - } - - public Map offsets() { - Map offsets = new HashMap<>(); - offsetMap.forEach((k, v) -> offsets.put(k, v.longValue())); - return offsets; - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/PostponeBucketCommittableRewriter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/PostponeBucketCommittableRewriter.java index 50eb5a079600..d26a637bc4ea 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/PostponeBucketCommittableRewriter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/PostponeBucketCommittableRewriter.java @@ -76,7 +76,6 @@ public List emitAll(long checkpointId) { Committable committable = new Committable( checkpointId, - Committable.Kind.FILE, bucketFiles.makeMessage( partitionEntry.getKey(), bucketEntry.getKey())); result.add(committable); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/RemovePostponeBucketFilesOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/RemovePostponeBucketFilesOperator.java index 09d3f829bf84..668a98879f49 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/RemovePostponeBucketFilesOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/RemovePostponeBucketFilesOperator.java @@ -51,9 +51,7 @@ public void processElement(StreamRecord element) throws Exception { dataSplit.dataFiles(), Collections.emptyList(), Collections.emptyList())); - output.collect( - new StreamRecord<>( - new Committable(Long.MAX_VALUE, Committable.Kind.FILE, message))); + output.collect(new StreamRecord<>(new Committable(Long.MAX_VALUE, message))); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/RewritePostponeBucketCommittableOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/RewritePostponeBucketCommittableOperator.java index 43b4cc124441..7eeba2dd0872 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/RewritePostponeBucketCommittableOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/RewritePostponeBucketCommittableOperator.java @@ -46,11 +46,7 @@ public void open() throws Exception { @Override public void processElement(StreamRecord element) throws Exception { Committable committable = element.getValue(); - if (committable.kind() != Committable.Kind.FILE) { - output.collect(element); - } - - rewriter.add((CommitMessageImpl) committable.wrappedCommittable()); + rewriter.add((CommitMessageImpl) committable.commitMessage()); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java index 3450d80c30c4..194bd77d7e65 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java @@ -101,8 +101,7 @@ protected List prepareCommit(boolean waitCompaction, long tableId.getDatabaseName(), tableId.getObjectName(), committable.checkpointId(), - committable.kind(), - committable.wrappedCommittable())); + committable.commitMessage())); } } 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 ccad6b35ca1d..6f33a5e45f02 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 @@ -55,18 +55,15 @@ public abstract class AppendTableSink extends FlinkWriteSink { private static final long serialVersionUID = 1L; protected final FileStoreTable table; - protected final LogSinkFunction logSinkFunction; @Nullable protected final Integer parallelism; public AppendTableSink( FileStoreTable table, @Nullable Map overwritePartitions, - LogSinkFunction logSinkFunction, @Nullable Integer parallelism) { super(table, overwritePartitions); this.table = table; - this.logSinkFunction = logSinkFunction; this.parallelism = parallelism; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/Committable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/Committable.java index 27a00717e2d8..73cf99bc6971 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/Committable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/Committable.java @@ -18,31 +18,26 @@ package org.apache.paimon.flink.sink; +import org.apache.paimon.table.sink.CommitMessage; + /** Committable produced by {@link PrepareCommitOperator}. */ public class Committable { private final long checkpointId; - private final Kind kind; - - private final Object wrappedCommittable; + private final CommitMessage commitMessage; - public Committable(long checkpointId, Kind kind, Object wrappedCommittable) { + public Committable(long checkpointId, CommitMessage commitMessage) { this.checkpointId = checkpointId; - this.kind = kind; - this.wrappedCommittable = wrappedCommittable; + this.commitMessage = commitMessage; } public long checkpointId() { return checkpointId; } - public Kind kind() { - return kind; - } - - public Object wrappedCommittable() { - return wrappedCommittable; + public CommitMessage commitMessage() { + return commitMessage; } @Override @@ -50,39 +45,8 @@ public String toString() { return "Committable{" + "checkpointId=" + checkpointId - + ", kind=" - + kind - + ", wrappedCommittable=" - + wrappedCommittable + + ", commitMessage=" + + commitMessage + '}'; } - - /** Kind of the produced Committable. */ - public enum Kind { - FILE((byte) 0), - - LOG_OFFSET((byte) 1); - - private final byte value; - - Kind(byte value) { - this.value = value; - } - - public byte toByteValue() { - return value; - } - - public static Kind fromByteValue(byte value) { - switch (value) { - case 0: - return FILE; - case 1: - return LOG_OFFSET; - default: - throw new UnsupportedOperationException( - "Unsupported byte value '" + value + "' for value kind."); - } - } - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableSerializer.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableSerializer.java index 3f7304ededc0..7092684bfe8b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableSerializer.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableSerializer.java @@ -37,31 +37,15 @@ public CommittableSerializer(CommitMessageSerializer commitMessageSerializer) { @Override public int getVersion() { - return 2; + return 3; } @Override public byte[] serialize(Committable committable) throws IOException { - byte[] wrapped; - int version; - switch (committable.kind()) { - case FILE: - version = commitMessageSerializer.getVersion(); - wrapped = - commitMessageSerializer.serialize( - (CommitMessage) committable.wrappedCommittable()); - break; - case LOG_OFFSET: - version = 1; - wrapped = ((LogOffsetCommittable) committable.wrappedCommittable()).toBytes(); - break; - default: - throw new UnsupportedOperationException("Unsupported kind: " + committable.kind()); - } - - return ByteBuffer.allocate(8 + 1 + wrapped.length + 4) + int version = commitMessageSerializer.getVersion(); + byte[] wrapped = commitMessageSerializer.serialize(committable.commitMessage()); + return ByteBuffer.allocate(8 + wrapped.length + 4) .putLong(committable.checkpointId()) - .put(committable.kind().toByteValue()) .put(wrapped) .putInt(version) .array(); @@ -75,22 +59,11 @@ public Committable deserialize(int committableVersion, byte[] bytes) throws IOEx ByteBuffer buffer = ByteBuffer.wrap(bytes); long checkpointId = buffer.getLong(); - Committable.Kind kind = Committable.Kind.fromByteValue(buffer.get()); - byte[] wrapped = new byte[bytes.length - 13]; + byte[] wrapped = new byte[bytes.length - 12]; buffer.get(wrapped); int version = buffer.getInt(); - Object wrappedCommittable; - switch (kind) { - case FILE: - wrappedCommittable = commitMessageSerializer.deserialize(version, wrapped); - break; - case LOG_OFFSET: - wrappedCommittable = LogOffsetCommittable.fromBytes(wrapped); - break; - default: - throw new UnsupportedOperationException("Unsupported kind: " + kind); - } - return new Committable(checkpointId, kind, wrappedCommittable); + CommitMessage commitMessage = commitMessageSerializer.deserialize(version, wrapped); + return new Committable(checkpointId, commitMessage); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DataEvolutionCompactionWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DataEvolutionCompactionWorkerOperator.java index e5803d9aff3f..3ef414942d60 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DataEvolutionCompactionWorkerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DataEvolutionCompactionWorkerOperator.java @@ -57,10 +57,7 @@ private DataEvolutionCompactionWorkerOperator( public void processElement(StreamRecord element) throws Exception { DataEvolutionCompactTask task = element.getValue(); committables.add( - new Committable( - Long.MAX_VALUE, - Committable.Kind.FILE, - task.doCompact(fileStoreTable, commitUser))); + new Committable(Long.MAX_VALUE, task.doCompact(fileStoreTable, commitUser))); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketRowWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketRowWriteOperator.java index a0eaa13b084f..07191741c599 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketRowWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketRowWriteOperator.java @@ -46,11 +46,6 @@ private DynamicBucketRowWriteOperator( super(parameters, table, storeSinkWriteProvider, initialCommitUser); } - @Override - protected boolean containLogSystem() { - return false; - } - @Override public void processElement(StreamRecord> element) throws Exception { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java index c43c2ebb1ef5..ce2debe9d371 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java @@ -35,14 +35,8 @@ public class FixedBucketSink extends FlinkWriteSink { private static final long serialVersionUID = 1L; - @Nullable private final LogSinkFunction logSinkFunction; - - public FixedBucketSink( - FileStoreTable table, - @Nullable Map overwritePartition, - @Nullable LogSinkFunction logSinkFunction) { + public FixedBucketSink(FileStoreTable table, @Nullable Map overwritePartition) { super(table, overwritePartition); - this.logSinkFunction = logSinkFunction; } @Override @@ -51,9 +45,7 @@ protected OneInputStreamOperatorFactory createWriteOpe Options options = table.coreOptions().toConfiguration(); boolean coordinatorEnabled = options.get(SINK_WRITER_COORDINATOR_ENABLED); return coordinatorEnabled - ? new RowDataStoreWriteOperator.CoordinatedFactory( - table, logSinkFunction, writeProvider, commitUser) - : new RowDataStoreWriteOperator.Factory( - table, logSinkFunction, writeProvider, commitUser); + ? new RowDataStoreWriteOperator.CoordinatedFactory(table, writeProvider, commitUser) + : new RowDataStoreWriteOperator.Factory(table, writeProvider, commitUser); } } 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 01a034e78ad5..46c1f040eccb 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 @@ -87,7 +87,6 @@ public class FlinkSinkBuilder { // ============== for extension ============== protected boolean compactSink = false; - @Nullable protected LogSinkFunction logSinkFunction; public FlinkSinkBuilder(Table table) { if (!(table instanceof FileStoreTable)) { @@ -262,7 +261,6 @@ public static DataStream mapToInternalRow( protected DataStreamSink buildDynamicBucketSink( DataStream input, boolean globalIndex) { - checkArgument(logSinkFunction == null, "Dynamic bucket mode can not work with log system."); return compactSink && !globalIndex // todo support global index sort compact ? new DynamicBucketCompactSink(table, overwritePartition).build(input, parallelism) @@ -285,11 +283,8 @@ protected DataStreamSink buildForFixedBucket(DataStream input) { parallelism = bucketNums; } DataStream partitioned = - partition( - input, - new RowDataChannelComputer(table.schema(), logSinkFunction != null), - parallelism); - FixedBucketSink sink = new FixedBucketSink(table, overwritePartition, logSinkFunction); + partition(input, new RowDataChannelComputer(table.schema()), parallelism); + FixedBucketSink sink = new FixedBucketSink(table, overwritePartition); return sink.sinkFrom(partitioned); } @@ -335,8 +330,7 @@ private DataStreamSink buildUnawareBucketSink(DataStream input) parallelism); } - return new RowAppendTableSink(table, overwritePartition, logSinkFunction, parallelism) - .sinkFrom(input); + return new RowAppendTableSink(table, overwritePartition, parallelism).sinkFrom(input); } private DataStream trySortInput(DataStream input) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java index 7344418430aa..2a50f558fcf2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java @@ -18,7 +18,6 @@ package org.apache.paimon.flink.sink; -import org.apache.paimon.flink.log.LogStoreTableFactory; import org.apache.paimon.table.Table; import org.apache.paimon.table.sink.BatchTableCommit; @@ -26,18 +25,13 @@ import org.apache.flink.table.connector.sink.abilities.SupportsTruncate; import org.apache.flink.table.factories.DynamicTableFactory; -import javax.annotation.Nullable; - /** Table sink to create sink. */ public class FlinkTableSink extends SupportsRowLevelOperationFlinkTableSink implements SupportsTruncate { public FlinkTableSink( - ObjectIdentifier tableIdentifier, - Table table, - DynamicTableFactory.Context context, - @Nullable LogStoreTableFactory logStoreTableFactory) { - super(tableIdentifier, table, context, logStoreTableFactory); + ObjectIdentifier tableIdentifier, Table table, DynamicTableFactory.Context context) { + super(tableIdentifier, table, context); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSinkBase.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSinkBase.java index 79e449407a9f..b7c37d525eff 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSinkBase.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSinkBase.java @@ -20,11 +20,8 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.CoreOptions.ChangelogProducer; -import org.apache.paimon.CoreOptions.LogChangelogMode; import org.apache.paimon.CoreOptions.MergeEngine; import org.apache.paimon.flink.PaimonDataStreamSinkProvider; -import org.apache.paimon.flink.log.LogSinkProvider; -import org.apache.paimon.flink.log.LogStoreTableFactory; import org.apache.paimon.options.Options; import org.apache.paimon.table.FormatTable; import org.apache.paimon.table.Table; @@ -38,8 +35,6 @@ import org.apache.flink.table.factories.DynamicTableFactory; import org.apache.flink.types.RowKind; -import javax.annotation.Nullable; - import java.util.HashMap; import java.util.Map; @@ -47,7 +42,6 @@ import static org.apache.paimon.CoreOptions.CLUSTERING_COLUMNS; import static org.apache.paimon.CoreOptions.CLUSTERING_INCREMENTAL; import static org.apache.paimon.CoreOptions.CLUSTERING_STRATEGY; -import static org.apache.paimon.CoreOptions.LOG_CHANGELOG_MODE; import static org.apache.paimon.CoreOptions.MERGE_ENGINE; import static org.apache.paimon.flink.FlinkConnectorOptions.CLUSTERING_SAMPLE_FACTOR; import static org.apache.paimon.flink.FlinkConnectorOptions.CLUSTERING_SORT_IN_CLUSTER; @@ -59,7 +53,6 @@ public abstract class FlinkTableSinkBase protected final ObjectIdentifier tableIdentifier; protected final DynamicTableFactory.Context context; - @Nullable protected final LogStoreTableFactory logStoreTableFactory; protected final Table table; @@ -67,14 +60,10 @@ public abstract class FlinkTableSinkBase protected boolean overwrite = false; public FlinkTableSinkBase( - ObjectIdentifier tableIdentifier, - Table table, - DynamicTableFactory.Context context, - @Nullable LogStoreTableFactory logStoreTableFactory) { + ObjectIdentifier tableIdentifier, Table table, DynamicTableFactory.Context context) { this.tableIdentifier = tableIdentifier; this.table = table; this.context = context; - this.logStoreTableFactory = logStoreTableFactory; } @Override @@ -98,10 +87,6 @@ && new CoreOptions(options).definedAggFunc()) { return requestedMode; } - if (options.get(LOG_CHANGELOG_MODE) == LogChangelogMode.ALL) { - return requestedMode; - } - // with primary key, default sink upsert ChangelogMode.Builder builder = ChangelogMode.newBuilder(); for (RowKind kind : requestedMode.getContainedKinds()) { @@ -127,23 +112,16 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { formatTable, overwrite, staticPartitions) .sinkFrom(dataStream)); } - LogSinkProvider logSinkProvider = null; - if (logStoreTableFactory != null) { - logSinkProvider = logStoreTableFactory.createSinkProvider(this.context, context); - } Options conf = Options.fromMap(table.options()); // Do not sink to log store when overwrite mode - final LogSinkFunction logSinkFunction = - overwrite ? null : (logSinkProvider == null ? null : logSinkProvider.createSink()); return new PaimonDataStreamSinkProvider( (dataStream) -> { - LogFlinkSinkBuilder builder = createSinkBuilder(); - builder.logSinkFunction(logSinkFunction) - .forRowData( - new DataStream<>( - dataStream.getExecutionEnvironment(), - dataStream.getTransformation())); + FlinkSinkBuilder builder = createSinkBuilder(); + builder.forRowData( + new DataStream<>( + dataStream.getExecutionEnvironment(), + dataStream.getTransformation())); if (!conf.get(CLUSTERING_INCREMENTAL)) { builder.clusteringIfPossible( conf.get(CLUSTERING_COLUMNS), @@ -159,14 +137,13 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { }); } - protected LogFlinkSinkBuilder createSinkBuilder() { - return new LogFlinkSinkBuilder(table); + protected FlinkSinkBuilder createSinkBuilder() { + return new FlinkSinkBuilder(table); } @Override public DynamicTableSink copy() { - FlinkTableSink copied = - new FlinkTableSink(tableIdentifier, table, context, logStoreTableFactory); + FlinkTableSink copied = new FlinkTableSink(tableIdentifier, table, context); copied.staticPartitions = new HashMap<>(staticPartitions); copied.overwrite = overwrite; return copied; 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 d6946046276a..7b7cb18bb819 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 @@ -72,16 +72,14 @@ protected CommittableStateManager createCommittableStateMan protected static OneInputStreamOperatorFactory createNoStateRowWriteOperatorFactory( FileStoreTable table, - LogSinkFunction logSinkFunction, StoreSinkWrite.Provider writeProvider, String commitUser) { - return new RowDataStoreWriteOperator.Factory( - table, logSinkFunction, writeProvider, commitUser) { + return new RowDataStoreWriteOperator.Factory(table, writeProvider, commitUser) { @Override @SuppressWarnings("unchecked, rawtypes") public StreamOperator createStreamOperator(StreamOperatorParameters parameters) { return new StatelessRowDataStoreWriteOperator( - parameters, table, logSinkFunction, writeProvider, commitUser); + parameters, table, writeProvider, commitUser); } }; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LogFlinkSinkBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LogFlinkSinkBuilder.java deleted file mode 100644 index aa64b3e35f47..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LogFlinkSinkBuilder.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.sink; - -import org.apache.paimon.table.Table; - -import javax.annotation.Nullable; - -/** A special version {@link FlinkSinkBuilder} with log sink. */ -public class LogFlinkSinkBuilder extends FlinkSinkBuilder { - - public LogFlinkSinkBuilder(Table table) { - super(table); - } - - FlinkSinkBuilder logSinkFunction(@Nullable LogSinkFunction logSinkFunction) { - this.logSinkFunction = logSinkFunction; - return this; - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LogOffsetCommittable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LogOffsetCommittable.java deleted file mode 100644 index 287c2e491675..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LogOffsetCommittable.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.sink; - -import java.nio.ByteBuffer; -import java.util.Objects; - -/** Log offset committable for a bucket. */ -public class LogOffsetCommittable { - - private final int bucket; - - private final long offset; - - public LogOffsetCommittable(int bucket, long offset) { - this.bucket = bucket; - this.offset = offset; - } - - public int bucket() { - return bucket; - } - - public long offset() { - return offset; - } - - public byte[] toBytes() { - return ByteBuffer.allocate(12).putInt(bucket).putLong(offset).array(); - } - - public static LogOffsetCommittable fromBytes(byte[] bytes) { - ByteBuffer buffer = ByteBuffer.wrap(bytes); - return new LogOffsetCommittable(buffer.getInt(), buffer.getLong()); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - LogOffsetCommittable that = (LogOffsetCommittable) o; - return bucket == that.bucket && offset == that.offset; - } - - @Override - public int hashCode() { - return Objects.hash(bucket, offset); - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LogSinkFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LogSinkFunction.java deleted file mode 100644 index 95efeba276dd..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LogSinkFunction.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.sink; - -import org.apache.paimon.table.sink.SinkRecord; - -import org.apache.flink.streaming.api.functions.sink.legacy.SinkFunction; - -/** Log {@link SinkFunction} with {@link WriteCallback}. */ -public interface LogSinkFunction extends SinkFunction { - - void setWriteCallback(WriteCallback writeCallback); - - /** Flush pending records. */ - void flush() throws Exception; - - /** - * A callback interface that the user can implement to know the offset of the bucket when the - * request is complete. - */ - interface WriteCallback { - - /** - * A callback method the user can implement to provide asynchronous handling of request - * completion. This method will be called when the record sent to the server has been - * acknowledged. - */ - void onCompletion(int bucket, long offset); - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittable.java index 42d9af6b55e0..bbaaf401c0c1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittable.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.table.sink.CommitMessage; /** * MultiTableCommittable produced by {@link PrepareCommitOperator}. This type of Committable will @@ -30,20 +31,12 @@ public class MultiTableCommittable { private final String database; private final String table; private final long checkpointId; - - private final Committable.Kind kind; - - private final Object wrappedCommittable; + private final CommitMessage commitMessage; public MultiTableCommittable( - String database, - String table, - long checkpointId, - Committable.Kind kind, - Object wrappedCommittable) { + String database, String table, long checkpointId, CommitMessage commitMessage) { this.checkpointId = checkpointId; - this.kind = kind; - this.wrappedCommittable = wrappedCommittable; + this.commitMessage = commitMessage; this.database = database; this.table = table; } @@ -53,8 +46,7 @@ public static MultiTableCommittable fromCommittable(Identifier id, Committable c id.getDatabaseName(), id.getObjectName(), committable.checkpointId(), - committable.kind(), - committable.wrappedCommittable()); + committable.commitMessage()); } public String getDatabase() { @@ -69,12 +61,8 @@ public long checkpointId() { return checkpointId; } - public Committable.Kind kind() { - return kind; - } - - public Object wrappedCommittable() { - return wrappedCommittable; + public CommitMessage commitMessage() { + return commitMessage; } @Override @@ -82,39 +70,8 @@ public String toString() { return "MultiTableCommittable{" + "checkpointId=" + checkpointId - + ", kind=" - + kind - + ", wrappedCommittable=" - + wrappedCommittable + + ", commitMessage=" + + commitMessage + '}'; } - - /** Kind of the produced Committable. */ - public enum Kind { - FILE((byte) 0), - - LOG_OFFSET((byte) 1); - - private final byte value; - - Kind(byte value) { - this.value = value; - } - - public byte toByteValue() { - return value; - } - - public static Kind fromByteValue(byte value) { - switch (value) { - case 0: - return FILE; - case 1: - return LOG_OFFSET; - default: - throw new UnsupportedOperationException( - "Unsupported byte value '" + value + "' for value kind."); - } - } - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableSerializer.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableSerializer.java index 2b427db4aa2f..3576286527ff 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableSerializer.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableSerializer.java @@ -47,7 +47,7 @@ public MultiTableCommittableSerializer(CommitMessageSerializer commitMessageSeri @Override public int getVersion() { - return 2; + return 3; } @Override @@ -101,26 +101,11 @@ public MultiTableCommittable deserialize(int committableVersion, byte[] bytes) } public byte[] serializeCommittable(MultiTableCommittable committable) throws IOException { - byte[] wrapped; - int version; - switch (committable.kind()) { - case FILE: - version = commitMessageSerializer.getVersion(); - wrapped = - commitMessageSerializer.serialize( - (CommitMessage) committable.wrappedCommittable()); - break; - case LOG_OFFSET: - version = 1; - wrapped = ((LogOffsetCommittable) committable.wrappedCommittable()).toBytes(); - break; - default: - throw new UnsupportedOperationException("Unsupported kind: " + committable.kind()); - } + int version = commitMessageSerializer.getVersion(); + byte[] wrapped = commitMessageSerializer.serialize(committable.commitMessage()); - return ByteBuffer.allocate(8 + 1 + wrapped.length + 4) + return ByteBuffer.allocate(8 + wrapped.length + 4) .putLong(committable.checkpointId()) - .put(committable.kind().toByteValue()) .put(wrapped) .putInt(version) .array(); @@ -134,22 +119,10 @@ public Committable deserializeCommittable(int committableVersion, byte[] bytes) ByteBuffer buffer = ByteBuffer.wrap(bytes); long checkpointId = buffer.getLong(); - Committable.Kind kind = Committable.Kind.fromByteValue(buffer.get()); - byte[] wrapped = new byte[bytes.length - 13]; + byte[] wrapped = new byte[bytes.length - 12]; buffer.get(wrapped); int version = buffer.getInt(); - - Object wrappedCommittable; - switch (kind) { - case FILE: - wrappedCommittable = commitMessageSerializer.deserialize(version, wrapped); - break; - case LOG_OFFSET: - wrappedCommittable = LogOffsetCommittable.fromBytes(wrapped); - break; - default: - throw new UnsupportedOperationException("Unsupported kind: " + kind); - } - return new Committable(checkpointId, kind, wrappedCommittable); + CommitMessage commitMessage = commitMessageSerializer.deserialize(version, wrapped); + return new Committable(checkpointId, commitMessage); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PostponeBatchWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PostponeBatchWriteOperator.java index a47925accfc2..9d166d0c257c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PostponeBatchWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PostponeBatchWriteOperator.java @@ -61,7 +61,7 @@ public PostponeBatchWriteOperator( StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser, Map knownNumBuckets) { - super(parameters, table, null, storeSinkWriteProvider, initialCommitUser); + super(parameters, table, storeSinkWriteProvider, initialCommitUser); this.knownNumBuckets = new HashMap<>(knownNumBuckets); this.bucketFunction = BucketFunction.create( @@ -96,21 +96,16 @@ protected List prepareCommit(boolean waitCompaction, long checkpoin throws IOException { List committables = new ArrayList<>(); for (Committable committable : super.prepareCommit(waitCompaction, checkpointId)) { - if (committable.kind() == Committable.Kind.FILE) { - CommitMessageImpl message = (CommitMessageImpl) committable.wrappedCommittable(); - committables.add( - new Committable( - committable.checkpointId(), - committable.kind(), - new CommitMessageImpl( - message.partition(), - message.bucket(), - checkNotNull(knownNumBuckets.get(message.partition())), - message.newFilesIncrement(), - message.compactIncrement()))); - } else { - committables.add(committable); - } + CommitMessageImpl message = (CommitMessageImpl) committable.commitMessage(); + committables.add( + new Committable( + committable.checkpointId(), + new CommitMessageImpl( + message.partition(), + message.bucket(), + checkNotNull(knownNumBuckets.get(message.partition())), + message.newFilesIncrement(), + message.compactIncrement()))); } return committables; 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 index 1ab6f398c5c6..8fece7ce36a9 100644 --- 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 @@ -41,7 +41,7 @@ public PostponeBucketSink( @Override protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return createNoStateRowWriteOperatorFactory(table, null, writeProvider, commitUser); + return createNoStateRowWriteOperatorFactory(table, writeProvider, commitUser); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PostponeFixedBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PostponeFixedBucketSink.java index f81ec9986681..130804f73ca6 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PostponeFixedBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PostponeFixedBucketSink.java @@ -50,7 +50,7 @@ public PostponeFixedBucketSink( @Override protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new RowDataStoreWriteOperator.Factory(table, null, writeProvider, commitUser) { + return new RowDataStoreWriteOperator.Factory(table, writeProvider, commitUser) { @Override @SuppressWarnings("unchecked, rawtypes") public StreamOperator createStreamOperator(StreamOperatorParameters parameters) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java index abf9a3fc7bb0..ad655698d134 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java @@ -35,7 +35,6 @@ import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -118,12 +117,11 @@ public void processElement(StreamRecord element) throws Exception } @Override - protected List prepareCommit(boolean waitCompaction, long checkpointId) - throws IOException { + protected List prepareCommit(boolean waitCompaction, long checkpointId) { ArrayList temp = new ArrayList<>(messages); messages.clear(); return temp.stream() - .map(s -> new Committable(checkpointId, Committable.Kind.FILE, s)) + .map(s -> new Committable(checkpointId, s)) .collect(Collectors.toList()); } } 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 a58839a84147..6e3272f9e0e9 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 @@ -32,18 +32,14 @@ public class RowAppendTableSink extends AppendTableSink { private static final long serialVersionUID = 1L; public RowAppendTableSink( - FileStoreTable table, - Map overwritePartitions, - LogSinkFunction logSinkFunction, - Integer parallelism) { - super(table, overwritePartitions, logSinkFunction, parallelism); + FileStoreTable table, Map overwritePartitions, Integer parallelism) { + super(table, overwritePartitions, parallelism); } @Override protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return createNoStateRowWriteOperatorFactory( - table, logSinkFunction, writeProvider, commitUser); + return createNoStateRowWriteOperatorFactory(table, writeProvider, commitUser); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataChannelComputer.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataChannelComputer.java index 77020f1ada31..1df93c82bcb1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataChannelComputer.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataChannelComputer.java @@ -31,14 +31,12 @@ public class RowDataChannelComputer implements ChannelComputer { private static final long serialVersionUID = 1L; private final TableSchema schema; - private final boolean hasLogSink; private transient int numChannels; private transient KeyAndBucketExtractor extractor; - public RowDataChannelComputer(TableSchema schema, boolean hasLogSink) { + public RowDataChannelComputer(TableSchema schema) { this.schema = schema; - this.hasLogSink = hasLogSink; } @Override @@ -54,11 +52,7 @@ public int channel(InternalRow record) { } public int channel(BinaryRow partition, int bucket) { - // log sinks like Kafka only consider bucket and don't care about partition - // so same bucket, even from different partition, must go to the same channel - return hasLogSink - ? ChannelComputer.select(bucket, numChannels) - : ChannelComputer.select(partition, bucket, numChannels); + return ChannelComputer.select(partition, bucket, numChannels); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataKeyAndBucketExtractor.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataKeyAndBucketExtractor.java index ae4d6e9eb662..8082df005fe8 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataKeyAndBucketExtractor.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataKeyAndBucketExtractor.java @@ -54,9 +54,4 @@ public int bucket() { public BinaryRow trimmedPrimaryKey() { return wrapped.trimmedPrimaryKey(); } - - @Override - public BinaryRow logPrimaryKey() { - return wrapped.logPrimaryKey(); - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java index 4a9b64f7b1b2..7bc78ca22e63 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java @@ -19,133 +19,40 @@ package org.apache.paimon.flink.sink; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.flink.log.LogWriteCallback; -import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.SinkRecord; -import org.apache.flink.api.common.functions.Function; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichFunction; -import org.apache.flink.api.common.functions.util.FunctionUtils; -import org.apache.flink.api.common.state.CheckpointListener; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateSnapshotContext; -import org.apache.flink.streaming.api.functions.sink.legacy.SinkFunction; -import org.apache.flink.streaming.api.operators.InternalTimerService; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperatorParameters; -import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; -import org.apache.flink.streaming.util.functions.StreamingFunctionUtils; import javax.annotation.Nullable; import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; import java.util.List; -import java.util.Objects; - -import static org.apache.paimon.CoreOptions.LOG_IGNORE_DELETE; /** A {@link PrepareCommitOperator} to write {@link InternalRow}. Record schema is fixed. */ public class RowDataStoreWriteOperator extends TableWriteOperator { private static final long serialVersionUID = 3L; - @Nullable private final LogSinkFunction logSinkFunction; - private transient SimpleContext sinkContext; - @Nullable private transient LogWriteCallback logCallback; - private transient boolean logIgnoreDelete; - - /** We listen to this ourselves because we don't have an {@link InternalTimerService}. */ - private long currentWatermark = Long.MIN_VALUE; - protected RowDataStoreWriteOperator( StreamOperatorParameters parameters, FileStoreTable table, - @Nullable LogSinkFunction logSinkFunction, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { super(parameters, table, storeSinkWriteProvider, initialCommitUser); - this.logSinkFunction = logSinkFunction; - if (logSinkFunction != null) { - FunctionUtils.setFunctionRuntimeContext(logSinkFunction, getRuntimeContext()); - } - } - - @Override - public void initializeState(StateInitializationContext context) throws Exception { - super.initializeState(context); - - if (logSinkFunction != null) { - StreamingFunctionUtils.restoreFunctionState(context, logSinkFunction); - } - } - - @Override - protected boolean containLogSystem() { - return logSinkFunction != null; } @Override public void open() throws Exception { super.open(); - - this.sinkContext = new SimpleContext(getProcessingTimeService()); - if (logSinkFunction != null) { - openFunction(logSinkFunction); - logCallback = new LogWriteCallback(); - logSinkFunction.setWriteCallback(logCallback); - logIgnoreDelete = Options.fromMap(table.options()).get(LOG_IGNORE_DELETE); - } - } - - private static void openFunction(Function function) throws Exception { - if (function instanceof RichFunction) { - RichFunction richFunction = (RichFunction) function; - - try { - Method method = RichFunction.class.getDeclaredMethod("open", OpenContext.class); - method.invoke(richFunction, new OpenContext() {}); - return; - } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { - // to stay compatible with Flink 1.18- - } - - Method method = RichFunction.class.getDeclaredMethod("open", Configuration.class); - method.invoke(richFunction, new Configuration()); - } - } - - @Override - public void processWatermark(Watermark mark) throws Exception { - super.processWatermark(mark); - - this.currentWatermark = mark.getTimestamp(); - if (logSinkFunction != null) { - logSinkFunction.writeWatermark( - new org.apache.flink.api.common.eventtime.Watermark(mark.getTimestamp())); - } } @Override public void processElement(StreamRecord element) throws Exception { - sinkContext.timestamp = element.hasTimestamp() ? element.getTimestamp() : null; - - SinkRecord record = write(element.getValue()); - - if (record != null - && logSinkFunction != null - && (!logIgnoreDelete || record.row().getRowKind().isAdd())) { - // write to log store, need to preserve original pk (which includes partition fields) - SinkRecord logRecord = write.toLogRecord(record); - logSinkFunction.invoke(logRecord, sinkContext); - } + write(element.getValue()); } @Nullable @@ -157,115 +64,22 @@ protected SinkRecord write(InternalRow row) throws Exception { } } - @Override - public void snapshotState(StateSnapshotContext context) throws Exception { - super.snapshotState(context); - - if (logSinkFunction != null) { - StreamingFunctionUtils.snapshotFunctionState( - context, getOperatorStateBackend(), logSinkFunction); - } - } - - @Override - public void finish() throws Exception { - super.finish(); - - if (logSinkFunction != null) { - logSinkFunction.finish(); - } - } - - @Override - public void close() throws Exception { - super.close(); - - if (logSinkFunction != null) { - FunctionUtils.closeFunction(logSinkFunction); - } - } - - @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception { - super.notifyCheckpointComplete(checkpointId); - - if (logSinkFunction instanceof CheckpointListener) { - ((CheckpointListener) logSinkFunction).notifyCheckpointComplete(checkpointId); - } - } - - @Override - public void notifyCheckpointAborted(long checkpointId) throws Exception { - super.notifyCheckpointAborted(checkpointId); - - if (logSinkFunction instanceof CheckpointListener) { - ((CheckpointListener) logSinkFunction).notifyCheckpointAborted(checkpointId); - } - } - @Override protected List prepareCommit(boolean waitCompaction, long checkpointId) throws IOException { List committables = super.prepareCommit(waitCompaction, checkpointId); - - if (logCallback != null) { - try { - Objects.requireNonNull(logSinkFunction).flush(); - } catch (Exception e) { - throw new IOException(e); - } - logCallback - .offsets() - .forEach( - (k, v) -> - committables.add( - new Committable( - checkpointId, - Committable.Kind.LOG_OFFSET, - new LogOffsetCommittable(k, v)))); - } tryRefreshWrite(); return committables; } - private class SimpleContext implements SinkFunction.Context { - - @Nullable private Long timestamp; - - private final ProcessingTimeService processingTimeService; - - public SimpleContext(ProcessingTimeService processingTimeService) { - this.processingTimeService = processingTimeService; - } - - @Override - public long currentProcessingTime() { - return processingTimeService.getCurrentProcessingTime(); - } - - @Override - public long currentWatermark() { - return currentWatermark; - } - - @Override - public Long timestamp() { - return timestamp; - } - } - /** {@link StreamOperatorFactory} of {@link RowDataStoreWriteOperator}. */ public static class Factory extends TableWriteOperator.Factory { - @Nullable private final LogSinkFunction logSinkFunction; - public Factory( FileStoreTable table, - @Nullable LogSinkFunction logSinkFunction, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { super(table, storeSinkWriteProvider, initialCommitUser); - this.logSinkFunction = logSinkFunction; } @Override @@ -274,11 +88,7 @@ public > T createStreamOperator( StreamOperatorParameters parameters) { return (T) new RowDataStoreWriteOperator( - parameters, - table, - logSinkFunction, - storeSinkWriteProvider, - initialCommitUser); + parameters, table, storeSinkWriteProvider, initialCommitUser); } @Override @@ -292,15 +102,11 @@ public Class getStreamOperatorClass(ClassLoader classL public static class CoordinatedFactory extends TableWriteOperator.CoordinatedFactory { - @Nullable private final LogSinkFunction logSinkFunction; - public CoordinatedFactory( FileStoreTable table, - @Nullable LogSinkFunction logSinkFunction, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { super(table, storeSinkWriteProvider, initialCommitUser); - this.logSinkFunction = logSinkFunction; } @Override @@ -309,11 +115,7 @@ public > T createStreamOperatorImpl( StreamOperatorParameters parameters) { return (T) new RowDataStoreWriteOperator( - parameters, - table, - logSinkFunction, - storeSinkWriteProvider, - initialCommitUser); + parameters, table, storeSinkWriteProvider, initialCommitUser); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StatelessRowDataStoreWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StatelessRowDataStoreWriteOperator.java index 49705ea805b8..eac883b56d9b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StatelessRowDataStoreWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StatelessRowDataStoreWriteOperator.java @@ -23,8 +23,6 @@ import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.streaming.api.operators.StreamOperatorParameters; -import javax.annotation.Nullable; - /** Stateless writer used for unaware append and postpone bucket table. */ public class StatelessRowDataStoreWriteOperator extends RowDataStoreWriteOperator { @@ -33,10 +31,9 @@ public class StatelessRowDataStoreWriteOperator extends RowDataStoreWriteOperato public StatelessRowDataStoreWriteOperator( StreamOperatorParameters parameters, FileStoreTable table, - @Nullable LogSinkFunction logSinkFunction, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { - super(parameters, table, logSinkFunction, storeSinkWriteProvider, initialCommitUser); + super(parameters, table, storeSinkWriteProvider, initialCommitUser); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCommitter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCommitter.java index 4314d1477c5d..ceba772140c1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCommitter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCommitter.java @@ -89,18 +89,8 @@ public ManifestCommittable combine( ManifestCommittable manifestCommittable, List committables) { for (Committable committable : committables) { - switch (committable.kind()) { - case FILE: - CommitMessage file = (CommitMessage) committable.wrappedCommittable(); - manifestCommittable.addFileCommittable(file); - break; - case LOG_OFFSET: - LogOffsetCommittable offset = - (LogOffsetCommittable) committable.wrappedCommittable(); - manifestCommittable.addLogOffset( - offset.bucket(), offset.offset(), allowLogOffsetDuplicate); - break; - } + CommitMessage file = committable.commitMessage(); + manifestCommittable.addFileCommittable(file); } return manifestCommittable; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java index 77aafe0baf22..d8dde60b31c2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java @@ -129,20 +129,8 @@ public WrappedManifestCommittable combine( ManifestCommittable manifestCommittable = wrappedManifestCommittable.computeCommittableIfAbsent( identifier, checkpointId, watermark); - - switch (committable.kind()) { - case FILE: - CommitMessage file = (CommitMessage) committable.wrappedCommittable(); - manifestCommittable.addFileCommittable(file); - break; - case LOG_OFFSET: - LogOffsetCommittable offset = - (LogOffsetCommittable) committable.wrappedCommittable(); - StoreCommitter committer = tableCommitters.get(identifier); - manifestCommittable.addLogOffset( - offset.bucket(), offset.offset(), committer.allowLogOffsetDuplicate()); - break; - } + CommitMessage file = committable.commitMessage(); + manifestCommittable.addFileCommittable(file); } return wrappedManifestCommittable; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWrite.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWrite.java index a7b1434740de..06c0353f184b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWrite.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWrite.java @@ -56,8 +56,6 @@ public interface StoreSinkWrite { @Nullable SinkRecord write(InternalRow rowData, int bucket) throws Exception; - SinkRecord toLogRecord(SinkRecord record); - void compact(BinaryRow partition, int bucket, boolean fullCompaction) throws Exception; void notifyNewFiles(long snapshotId, BinaryRow partition, int bucket, List files); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteImpl.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteImpl.java index 73c49be1e00f..289f465a1733 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteImpl.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteImpl.java @@ -115,11 +115,6 @@ public SinkRecord write(InternalRow rowData, int bucket) throws Exception { return write.writeAndReturn(rowData, bucket); } - @Override - public SinkRecord toLogRecord(SinkRecord record) { - return write.toLogRecord(record); - } - @Override public void compact(BinaryRow partition, int bucket, boolean fullCompaction) throws Exception { write.compact(partition, bucket, fullCompaction); @@ -147,8 +142,7 @@ public List prepareCommit(boolean waitCompaction, long checkpointId try { for (CommitMessage committable : write.prepareCommit(this.waitCompaction || waitCompaction, checkpointId)) { - committables.add( - new Committable(checkpointId, Committable.Kind.FILE, committable)); + committables.add(new Committable(checkpointId, committable)); } } catch (Exception e) { throw new IOException(e); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/SupportsRowLevelOperationFlinkTableSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/SupportsRowLevelOperationFlinkTableSink.java index fc4357907d87..7ed8e9f1768b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/SupportsRowLevelOperationFlinkTableSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/SupportsRowLevelOperationFlinkTableSink.java @@ -22,7 +22,6 @@ import org.apache.paimon.CoreOptions.MergeEngine; import org.apache.paimon.flink.LogicalTypeConversion; import org.apache.paimon.flink.PredicateConverter; -import org.apache.paimon.flink.log.LogStoreTableFactory; import org.apache.paimon.options.Options; import org.apache.paimon.predicate.OnlyPartitionKeyEqualVisitor; import org.apache.paimon.predicate.Predicate; @@ -66,17 +65,13 @@ public abstract class SupportsRowLevelOperationFlinkTableSink extends FlinkTable @Nullable protected Predicate deletePredicate; public SupportsRowLevelOperationFlinkTableSink( - ObjectIdentifier tableIdentifier, - Table table, - DynamicTableFactory.Context context, - @Nullable LogStoreTableFactory logStoreTableFactory) { - super(tableIdentifier, table, context, logStoreTableFactory); + ObjectIdentifier tableIdentifier, Table table, DynamicTableFactory.Context context) { + super(tableIdentifier, table, context); } @Override public DynamicTableSink copy() { - FlinkTableSink copied = - new FlinkTableSink(tableIdentifier, table, context, logStoreTableFactory); + FlinkTableSink copied = new FlinkTableSink(tableIdentifier, table, context); copied.staticPartitions = new HashMap<>(staticPartitions); copied.overwrite = overwrite; copied.deletePredicate = deletePredicate; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java index 70a6af8a25dc..f93bdfb560dd 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java @@ -75,17 +75,11 @@ public TableWriteOperator( public void initializeState(StateInitializationContext context) throws Exception { super.initializeState(context); - boolean containLogSystem = containLogSystem(); int numTasks = RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()); int subtaskId = RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()); StateValueFilter stateFilter = - (tableName, partition, bucket) -> { - int task = - containLogSystem - ? ChannelComputer.select(bucket, numTasks) - : ChannelComputer.select(partition, bucket, numTasks); - return task == subtaskId; - }; + (tableName, partition, bucket) -> + subtaskId == ChannelComputer.select(partition, bucket, numTasks); state = createState(subtaskId, context, stateFilter); write = @@ -127,8 +121,6 @@ protected String getCommitUser(StateInitializationContext context) throws Except return commitUser; } - protected abstract boolean containLogSystem(); - @Override public void snapshotState(StateSnapshotContext context) throws Exception { super.snapshotState(context); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BaseDataTableSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BaseDataTableSource.java index 73ccbf7fb108..606b383ab6cb 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BaseDataTableSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BaseDataTableSource.java @@ -19,13 +19,8 @@ package org.apache.paimon.flink.source; import org.apache.paimon.CoreOptions; -import org.apache.paimon.CoreOptions.ChangelogProducer; -import org.apache.paimon.CoreOptions.LogChangelogMode; -import org.apache.paimon.CoreOptions.LogConsistency; import org.apache.paimon.flink.FlinkConnectorOptions.WatermarkEmitStrategy; import org.apache.paimon.flink.PaimonDataStreamScanProvider; -import org.apache.paimon.flink.log.LogSourceProvider; -import org.apache.paimon.flink.log.LogStoreTableFactory; import org.apache.paimon.flink.lookup.FileStoreLookupFunction; import org.apache.paimon.flink.lookup.LookupRuntimeProviderFactory; import org.apache.paimon.flink.lookup.partitioner.BucketIdExtractor; @@ -75,9 +70,6 @@ import java.util.stream.IntStream; import static org.apache.paimon.CoreOptions.CHANGELOG_PRODUCER; -import static org.apache.paimon.CoreOptions.LOG_CHANGELOG_MODE; -import static org.apache.paimon.CoreOptions.LOG_CONSISTENCY; -import static org.apache.paimon.CoreOptions.LOG_IGNORE_DELETE; import static org.apache.paimon.CoreOptions.MergeEngine.FIRST_ROW; import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_ASYNC; import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_ASYNC_THREAD_NUMBER; @@ -113,7 +105,6 @@ public abstract class BaseDataTableSource extends FlinkTableSource protected final ObjectIdentifier tableIdentifier; protected final boolean unbounded; protected final DynamicTableFactory.Context context; - @Nullable protected final LogStoreTableFactory logStoreTableFactory; @Nullable private BucketShufflePartitioner bucketShufflePartitioner; @Nullable protected WatermarkStrategy watermarkStrategy; @Nullable protected Long countPushed; @@ -123,7 +114,6 @@ public BaseDataTableSource( Table table, boolean unbounded, DynamicTableFactory.Context context, - @Nullable LogStoreTableFactory logStoreTableFactory, @Nullable Predicate predicate, @Nullable int[][] projectFields, @Nullable Long limit, @@ -134,7 +124,6 @@ public BaseDataTableSource( this.tableIdentifier = tableIdentifier; this.unbounded = unbounded; this.context = context; - this.logStoreTableFactory = logStoreTableFactory; this.watermarkStrategy = watermarkStrategy; this.countPushed = countPushed; @@ -149,33 +138,23 @@ public ChangelogMode getChangelogMode() { if (table.primaryKeys().isEmpty()) { return ChangelogMode.insertOnly(); - } else { - Options options = Options.fromMap(table.options()); - - if (new CoreOptions(options).mergeEngine() == FIRST_ROW) { - return ChangelogMode.insertOnly(); - } + } - if (options.get(SCAN_REMOVE_NORMALIZE)) { - return ChangelogMode.all(); - } + Options options = Options.fromMap(table.options()); - if (logStoreTableFactory == null - && options.get(CHANGELOG_PRODUCER) != ChangelogProducer.NONE) { - return ChangelogMode.all(); - } + if (new CoreOptions(options).mergeEngine() == FIRST_ROW) { + return ChangelogMode.insertOnly(); + } - if (logStoreTableFactory != null && options.get(LOG_IGNORE_DELETE)) { - return ChangelogMode.insertOnly(); - } + if (options.get(SCAN_REMOVE_NORMALIZE)) { + return ChangelogMode.all(); + } - // optimization: transaction consistency and all changelog mode avoid the generation of - // normalized nodes. See FlinkTableSink.getChangelogMode validation. - return options.get(LOG_CONSISTENCY) == LogConsistency.TRANSACTIONAL - && options.get(LOG_CHANGELOG_MODE) == LogChangelogMode.ALL - ? ChangelogMode.all() - : ChangelogMode.upsert(); + if (options.get(CHANGELOG_PRODUCER) != CoreOptions.ChangelogProducer.NONE) { + return ChangelogMode.all(); } + + return ChangelogMode.upsert(); } @Override @@ -184,12 +163,6 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { return createCountStarScan(); } - LogSourceProvider logSourceProvider = null; - if (logStoreTableFactory != null) { - logSourceProvider = - logStoreTableFactory.createSourceProvider(context, scanContext, projectFields); - } - WatermarkStrategy watermarkStrategy = this.watermarkStrategy; Options options = Options.fromMap(table.options()); if (watermarkStrategy != null) { @@ -216,7 +189,6 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { new FlinkSourceBuilder(table) .sourceName(tableIdentifier.asSummaryString()) .sourceBounded(!unbounded) - .logSourceProvider(logSourceProvider) .projection(projectFields) .predicate(predicate) .partitionPredicate(partitionPredicate) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/DataTableSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/DataTableSource.java index eab0432c79b2..a19e04d6e5c5 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/DataTableSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/DataTableSource.java @@ -18,7 +18,6 @@ package org.apache.paimon.flink.source; -import org.apache.paimon.flink.log.LogStoreTableFactory; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.stats.ColStats; import org.apache.paimon.stats.Statistics; @@ -57,20 +56,8 @@ public DataTableSource( ObjectIdentifier tableIdentifier, Table table, boolean unbounded, - DynamicTableFactory.Context context, - @Nullable LogStoreTableFactory logStoreTableFactory) { - this( - tableIdentifier, - table, - unbounded, - context, - logStoreTableFactory, - null, - null, - null, - null, - null, - null); + DynamicTableFactory.Context context) { + this(tableIdentifier, table, unbounded, context, null, null, null, null, null, null); } public DataTableSource( @@ -78,7 +65,6 @@ public DataTableSource( Table table, boolean unbounded, DynamicTableFactory.Context context, - @Nullable LogStoreTableFactory logStoreTableFactory, @Nullable Predicate predicate, @Nullable int[][] projectFields, @Nullable Long limit, @@ -90,7 +76,6 @@ public DataTableSource( table, unbounded, context, - logStoreTableFactory, predicate, projectFields, limit, @@ -106,7 +91,6 @@ public DataTableSource copy() { table, unbounded, context, - logStoreTableFactory, predicate, projectFields, limit, 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 1fe2e218a0fb..3e96dec1ea50 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 @@ -19,13 +19,10 @@ package org.apache.paimon.flink.source; import org.apache.paimon.CoreOptions; -import org.apache.paimon.CoreOptions.StartupMode; -import org.apache.paimon.CoreOptions.StreamingReadMode; import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.flink.NestedProjectedRowData; import org.apache.paimon.flink.Projection; -import org.apache.paimon.flink.log.LogSourceProvider; import org.apache.paimon.flink.sink.FlinkSink; import org.apache.paimon.flink.source.align.AlignedContinuousFileStoreSource; import org.apache.paimon.flink.source.operator.MonitorSource; @@ -42,9 +39,7 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; -import org.apache.flink.connector.base.source.hybrid.HybridSource; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; @@ -66,7 +61,6 @@ import java.util.Optional; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; -import static org.apache.paimon.CoreOptions.StreamingReadMode.FILE; import static org.apache.paimon.flink.FlinkConnectorOptions.SOURCE_OPERATOR_UID_SUFFIX; import static org.apache.paimon.flink.FlinkConnectorOptions.generateCustomUid; import static org.apache.paimon.flink.LogicalTypeConversion.toLogicalType; @@ -92,7 +86,6 @@ public class FlinkSourceBuilder { @Nullable private int[][] projectedFields; @Nullable private Predicate predicate; @Nullable private PartitionPredicate partitionPredicate; - @Nullable private LogSourceProvider logSourceProvider; @Nullable private Integer parallelism; @Nullable private Long limit; @Nullable private WatermarkStrategy watermarkStrategy; @@ -193,12 +186,6 @@ public FlinkSourceBuilder dynamicPartitionFilteringFields( return this; } - @Deprecated - FlinkSourceBuilder logSourceProvider(LogSourceProvider logSourceProvider) { - this.logSourceProvider = logSourceProvider; - return this; - } - private ReadBuilder createReadBuilder(@Nullable org.apache.paimon.types.RowType readType) { ReadBuilder readBuilder = table.newReadBuilder(); if (readType != null) { @@ -338,38 +325,14 @@ public DataStream build() { } TableScanUtils.streamingReadingValidate(table); - // TODO visit all options through CoreOptions - StartupMode startupMode = CoreOptions.startupMode(conf); - StreamingReadMode streamingReadMode = CoreOptions.streamReadType(conf); - - if (logSourceProvider != null && streamingReadMode != FILE) { - logSourceProvider.preCreateSource(); - if (startupMode != StartupMode.LATEST_FULL) { - return toDataStream(logSourceProvider.createSource(null)); - } else { - return toDataStream( - HybridSource.builder( - LogHybridSourceFactory.buildHybridFirstSource( - table, - projectedRowType(), - predicate, - partitionPredicate, - outerProject())) - .addSource( - new LogHybridSourceFactory(logSourceProvider), - Boundedness.CONTINUOUS_UNBOUNDED) - .build()); - } + if (conf.get(FlinkConnectorOptions.SOURCE_CHECKPOINT_ALIGN_ENABLED)) { + return buildAlignedContinuousFileSource(); + } else if (conf.contains(CoreOptions.CONSUMER_ID) + && conf.get(CoreOptions.CONSUMER_CONSISTENCY_MODE) + == CoreOptions.ConsumerMode.EXACTLY_ONCE) { + return buildDedicatedSplitGenSource(false); } else { - if (conf.get(FlinkConnectorOptions.SOURCE_CHECKPOINT_ALIGN_ENABLED)) { - return buildAlignedContinuousFileSource(); - } else if (conf.contains(CoreOptions.CONSUMER_ID) - && conf.get(CoreOptions.CONSUMER_CONSISTENCY_MODE) - == CoreOptions.ConsumerMode.EXACTLY_ONCE) { - return buildDedicatedSplitGenSource(false); - } else { - return buildContinuousFileSource(); - } + return buildContinuousFileSource(); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/LogHybridSourceFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/LogHybridSourceFactory.java deleted file mode 100644 index 18241d3fa25a..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/LogHybridSourceFactory.java +++ /dev/null @@ -1,169 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.source; - -import org.apache.paimon.Snapshot; -import org.apache.paimon.flink.FlinkConnectorOptions; -import org.apache.paimon.flink.NestedProjectedRowData; -import org.apache.paimon.flink.log.LogSourceProvider; -import org.apache.paimon.flink.metrics.FlinkMetricRegistry; -import org.apache.paimon.options.Options; -import org.apache.paimon.partition.PartitionPredicate; -import org.apache.paimon.predicate.Predicate; -import org.apache.paimon.table.DataTable; -import org.apache.paimon.table.Table; -import org.apache.paimon.table.source.ReadBuilder; -import org.apache.paimon.table.source.StreamDataTableScan; -import org.apache.paimon.table.source.StreamTableScan; -import org.apache.paimon.types.RowType; -import org.apache.paimon.utils.SnapshotManager; - -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.api.connector.source.SplitEnumerator; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.connector.base.source.hybrid.HybridSource; -import org.apache.flink.connector.base.source.hybrid.HybridSource.SourceFactory; -import org.apache.flink.table.data.RowData; - -import javax.annotation.Nullable; - -import java.util.Collection; -import java.util.Map; - -/** Log {@link SourceFactory} from {@link StaticFileStoreSplitEnumerator}. */ -public class LogHybridSourceFactory - implements SourceFactory, StaticFileStoreSplitEnumerator> { - - private final LogSourceProvider provider; - - public LogHybridSourceFactory(LogSourceProvider provider) { - this.provider = provider; - } - - @Override - public Source create( - HybridSource.SourceSwitchContext context) { - StaticFileStoreSplitEnumerator enumerator = context.getPreviousEnumerator(); - Map logOffsets = null; - if (enumerator != null) { - Snapshot snapshot = enumerator.snapshot(); - if (snapshot != null) { - logOffsets = snapshot.logOffsets(); - } - } - return provider.createSource(logOffsets); - } - - public static FlinkSource buildHybridFirstSource( - Table table, - @Nullable RowType readType, - @Nullable Predicate predicate, - @Nullable PartitionPredicate partitionPredicate, - @Nullable NestedProjectedRowData rowData) { - if (!(table instanceof DataTable)) { - throw new UnsupportedOperationException( - String.format( - "FlinkHybridFirstSource only accepts DataTable. Unsupported table type: '%s'.", - table.getClass().getSimpleName())); - } - - DataTable dataTable = (DataTable) table; - - ReadBuilder readBuilder = table.newReadBuilder(); - if (readType != null) { - readBuilder.withReadType(readType); - } - if (predicate != null) { - readBuilder.withFilter(predicate); - } - if (partitionPredicate != null) { - readBuilder.withPartitionFilter(partitionPredicate); - } - - return new FlinkHybridFirstSource( - readBuilder, - dataTable.snapshotManager(), - dataTable.coreOptions().toConfiguration(), - rowData, - dataTable.coreOptions().blobAsDescriptor()); - } - - /** The first source of a log {@link HybridSource}. */ - private static class FlinkHybridFirstSource extends FlinkSource { - - private static final long serialVersionUID = 3L; - - private final SnapshotManager snapshotManager; - private final Options options; - - public FlinkHybridFirstSource( - ReadBuilder readBuilder, - SnapshotManager snapshotManager, - Options options, - @Nullable NestedProjectedRowData rowData, - boolean blobAsDescriptor) { - super(readBuilder, null, rowData, blobAsDescriptor); - this.snapshotManager = snapshotManager; - this.options = options; - } - - @Override - public Boundedness getBoundedness() { - return Boundedness.BOUNDED; - } - - @Override - public SplitEnumerator restoreEnumerator( - SplitEnumeratorContext context, - PendingSplitsCheckpoint checkpoint) { - Long snapshotId = null; - Collection splits; - if (checkpoint == null) { - FileStoreSourceSplitGenerator splitGenerator = new FileStoreSourceSplitGenerator(); - // get snapshot id and splits from scan - StreamTableScan scan = readBuilder.newStreamScan(); - // register scan metrics - if (context.metricGroup() != null) { - ((StreamDataTableScan) scan) - .withMetricRegistry(new FlinkMetricRegistry(context.metricGroup())); - } - splits = splitGenerator.createSplits(scan.plan()); - Long nextSnapshotId = scan.checkpoint(); - if (nextSnapshotId != null) { - snapshotId = nextSnapshotId - 1; - } - } else { - // restore from checkpoint - snapshotId = checkpoint.currentSnapshotId(); - splits = checkpoint.splits(); - } - - Snapshot snapshot = snapshotId == null ? null : snapshotManager.snapshot(snapshotId); - return new StaticFileStoreSplitEnumerator( - context, - snapshot, - StaticFileStoreSource.createSplitAssigner( - context, - options.get(FlinkConnectorOptions.SCAN_SPLIT_ENUMERATOR_BATCH_SIZE), - options.get(FlinkConnectorOptions.SCAN_SPLIT_ENUMERATOR_ASSIGN_MODE), - splits)); - } - } -} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ChangelogModeTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ChangelogModeTest.java index 7a24efa0fa41..b968df474690 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ChangelogModeTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ChangelogModeTest.java @@ -18,7 +18,6 @@ package org.apache.paimon.flink; -import org.apache.paimon.CoreOptions; import org.apache.paimon.flink.sink.FlinkTableSink; import org.apache.paimon.flink.source.DataTableSource; import org.apache.paimon.fs.Path; @@ -40,6 +39,9 @@ import java.util.Collections; +import static org.apache.paimon.CoreOptions.CHANGELOG_PRODUCER; +import static org.apache.paimon.CoreOptions.ChangelogProducer.INPUT; +import static org.apache.paimon.CoreOptions.ChangelogProducer.LOOKUP; import static org.assertj.core.api.Assertions.assertThat; /** Test for changelog mode with flink source and sink. */ @@ -68,10 +70,10 @@ private void test(Options options, ChangelogMode expectSource, ChangelogMode exp "")); FileStoreTable table = FileStoreTableFactory.create(LocalFileIO.create(), path); - DataTableSource source = new DataTableSource(identifier, table, true, null, null); + DataTableSource source = new DataTableSource(identifier, table, true, null); assertThat(source.getChangelogMode()).isEqualTo(expectSource); - FlinkTableSink sink = new FlinkTableSink(identifier, table, null, null); + FlinkTableSink sink = new FlinkTableSink(identifier, table, null); assertThat(sink.getChangelogMode(ChangelogMode.all())).isEqualTo(expectSink); } @@ -90,14 +92,22 @@ public void testDefault() throws Exception { @Test public void testInputChangelogProducer() throws Exception { Options options = new Options(); - options.set(CoreOptions.CHANGELOG_PRODUCER, CoreOptions.ChangelogProducer.INPUT); + options.set(CHANGELOG_PRODUCER, INPUT); test(options, ChangelogMode.all(), ChangelogMode.all()); } @Test - public void testChangelogModeAll() throws Exception { + public void testLookupChangelogProducer() throws Exception { Options options = new Options(); - options.set(CoreOptions.LOG_CHANGELOG_MODE, CoreOptions.LogChangelogMode.ALL); - test(options, ChangelogMode.all(), ChangelogMode.all()); + options.set(CHANGELOG_PRODUCER, LOOKUP); + + // Compatible to Flink 2.0 (keyOnlyDeletes) + ChangelogMode.Builder upsertBuilder = ChangelogMode.newBuilder(); + for (RowKind kind : ChangelogMode.all().getContainedKinds()) { + if (kind != RowKind.UPDATE_BEFORE) { + upsertBuilder.addContainedKind(kind); + } + } + test(options, ChangelogMode.all(), upsertBuilder.build()); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/ComputedColumnAndWatermarkTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ComputedColumnAndWatermarkTableITCase.java similarity index 62% rename from paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/ComputedColumnAndWatermarkTableITCase.java rename to paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ComputedColumnAndWatermarkTableITCase.java index 352f748f1320..95497f60db07 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/ComputedColumnAndWatermarkTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ComputedColumnAndWatermarkTableITCase.java @@ -16,7 +16,9 @@ * limitations under the License. */ -package org.apache.paimon.flink.kafka; +package org.apache.paimon.flink; + +import org.apache.paimon.flink.util.AbstractTestBase; import org.apache.flink.types.Row; import org.junit.jupiter.api.BeforeEach; @@ -29,20 +31,16 @@ import java.util.stream.Collectors; import static org.apache.flink.table.planner.factories.TestValuesTableFactory.changelogRow; -import static org.apache.paimon.flink.kafka.KafkaLogTestUtils.createTableWithKafkaLog; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.SCAN_LATEST; import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.buildQuery; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.buildQueryWithTableOptions; import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.buildSimpleQuery; import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.createTable; import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.createTemporaryTable; import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.init; import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.insertIntoFromTable; import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.testBatchRead; -import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.testStreamingReadWithReadFirst; /** Paimon IT case when the table has computed column and watermark spec. */ -public class ComputedColumnAndWatermarkTableITCase extends KafkaTableTestBase { +public class ComputedColumnAndWatermarkTableITCase extends AbstractTestBase { @BeforeEach public void setUp() { @@ -221,116 +219,4 @@ public void testBatchSelectWithWatermark() throws Exception { testBatchRead(buildSimpleQuery(table), initialRecords); } - - @Test - public void testStreamingSelectWithWatermark() throws Exception { - // physical column as watermark - List initialRecords = - Arrays.asList( - changelogRow( - "+I", - "US Dollar", - 102L, - LocalDateTime.parse("1990-04-07T10:00:11.120")), - changelogRow( - "+I", "Euro", 119L, LocalDateTime.parse("2020-04-07T10:10:11.120")), - changelogRow( - "+I", "Yen", 1L, LocalDateTime.parse("2022-04-07T09:54:11.120"))); - - String temporaryTable = - createTemporaryTable( - Arrays.asList("currency STRING", "rate BIGINT", "ts TIMESTAMP(3)"), - Collections.emptyList(), - Collections.emptyList(), - initialRecords, - null, - true, - "I"); - - String table = - createTableWithKafkaLog( - Arrays.asList( - "currency STRING", - "rate BIGINT", - "ts TIMESTAMP(3)", - "WATERMARK FOR ts AS ts - INTERVAL '3' YEAR"), - Collections.emptyList(), - Collections.emptyList(), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, - "*", - "WHERE CURRENT_WATERMARK(ts) IS NULL OR ts > CURRENT_WATERMARK(ts)", - SCAN_LATEST), - Collections.singletonList( - changelogRow( - "+I", - "US Dollar", - 102L, - LocalDateTime.parse("1990-04-07T10:00:11.120")))) - .close(); - - // computed column as watermark - table = - createTableWithKafkaLog( - Arrays.asList( - "currency STRING", - "rate BIGINT", - "ts TIMESTAMP(3)", - "ts1 AS ts", - "WATERMARK FOR ts1 AS ts1 - INTERVAL '3' YEAR"), - Collections.emptyList(), - Collections.emptyList(), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, - "currency, rate, ts1", - "WHERE CURRENT_WATERMARK(ts1) IS NULL OR ts1 > CURRENT_WATERMARK(ts1)", - SCAN_LATEST), - Collections.singletonList( - changelogRow( - "+I", - "US Dollar", - 102L, - LocalDateTime.parse("1990-04-07T10:00:11.120")))) - .close(); - - // query both event time and processing time - table = - createTableWithKafkaLog( - Arrays.asList( - "currency STRING", - "rate BIGINT", - "ts TIMESTAMP(3)", - "ptime AS PROCTIME()", - "WATERMARK FOR ts AS ts - INTERVAL '3' YEAR"), - Collections.emptyList(), - Collections.emptyList(), - true); - - testStreamingReadWithReadFirst( - temporaryTable, - table, - buildQueryWithTableOptions( - table, - "currency, rate, ts, CHAR_LENGTH(DATE_FORMAT(ptime, 'yyyy-MM-dd HH:mm'))", - "WHERE CURRENT_WATERMARK(ts) IS NULL OR ts > CURRENT_WATERMARK(ts)", - SCAN_LATEST), - Collections.singletonList( - changelogRow( - "+I", - "US Dollar", - 102L, - LocalDateTime.parse("1990-04-07T10:00:11.120"), - 16))) - .close(); - } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java index 7494f91205eb..5322d81eb4ad 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java @@ -21,12 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.TableType; import org.apache.paimon.catalog.CatalogContext; -import org.apache.paimon.catalog.Identifier; import org.apache.paimon.catalog.PropertyChange; -import org.apache.paimon.flink.log.LogSinkProvider; -import org.apache.paimon.flink.log.LogSourceProvider; -import org.apache.paimon.flink.log.LogStoreRegister; -import org.apache.paimon.flink.log.LogStoreTableFactory; import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; @@ -59,8 +54,6 @@ import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; import org.apache.flink.table.catalog.exceptions.TableNotExistException; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.expressions.ResolvedExpression; import org.apache.flink.table.expressions.utils.ResolvedExpressionMock; import org.apache.flink.table.factories.DynamicTableFactory; @@ -72,8 +65,6 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; -import javax.annotation.Nullable; - import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -92,8 +83,6 @@ import static org.apache.paimon.CoreOptions.PATH; import static org.apache.paimon.CoreOptions.SCAN_FILE_CREATION_TIME_MILLIS; import static org.apache.paimon.flink.FlinkCatalogOptions.DISABLE_CREATE_TABLE_IN_DEFAULT_DB; -import static org.apache.paimon.flink.FlinkCatalogOptions.LOG_SYSTEM_AUTO_REGISTER; -import static org.apache.paimon.flink.FlinkConnectorOptions.LOG_SYSTEM; import static org.apache.paimon.flink.FlinkTestBase.createResolvedTable; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatCode; @@ -110,8 +99,6 @@ /** Test for {@link FlinkCatalog}. */ public class FlinkCatalogTest extends FlinkCatalogTestBase { - private static final String TESTING_LOG_STORE = "testing"; - private final ObjectPath path1 = new ObjectPath("db1", "t1"); private final ObjectPath path3 = new ObjectPath("db1", "t2"); @@ -121,17 +108,16 @@ public class FlinkCatalogTest extends FlinkCatalogTestBase { private final ObjectPath nonExistDbPath = ObjectPath.fromString("non.exist"); private final ObjectPath nonExistObjectPath = ObjectPath.fromString("db1.nonexist"); - private String warehouse; private Catalog catalog; @TempDir public static java.nio.file.Path temporaryFolder; @BeforeEach public void beforeEach() throws IOException { - warehouse = new File(temporaryFolder.toFile(), UUID.randomUUID().toString()).toString(); + String warehouse = + new File(temporaryFolder.toFile(), UUID.randomUUID().toString()).toString(); Options conf = new Options(); conf.setString("warehouse", warehouse); - conf.set(LOG_SYSTEM_AUTO_REGISTER, true); catalog = FlinkCatalogFactory.createCatalog( "test-catalog", @@ -277,14 +263,7 @@ public void testAlterMaterializedTable(Map options) throws Excep catalog.alterTable(tablePath, expectedMaterializedTable, tableChanges, false); CatalogBaseTable updatedTable = catalog.getTable(tablePath); - checkEquals( - tablePath, - expectedMaterializedTable, - updatedTable, - Collections.singletonMap( - FlinkCatalogOptions.REGISTER_TIMEOUT.key(), - FlinkCatalogOptions.REGISTER_TIMEOUT.defaultValue().toString()), - Collections.emptySet()); + checkEquals(tablePath, expectedMaterializedTable, updatedTable, Collections.emptySet()); } @ParameterizedTest @@ -699,74 +678,14 @@ public void testCreateTableWithColumnOptions() throws Exception { Map expected = got.getOptions(); expected.remove("path"); expected.remove("table.data.path"); - expected.remove(FlinkCatalogOptions.REGISTER_TIMEOUT.key()); assertThat(catalogTable.getOptions()).isEqualTo(expected); } @Test - public void testCreateTableWithLogSystemRegister() throws Exception { - catalog.createDatabase(path1.getDatabaseName(), null, false); - - ResolvedExpression expression = - new ResolvedExpressionMock(DataTypes.INT(), () -> "test + 1"); - ResolvedSchema resolvedSchema = - new ResolvedSchema( - Arrays.asList( - Column.physical("pk", DataTypes.INT().notNull()), - Column.physical("test", DataTypes.INT()), - Column.computed("comp", expression)), - Collections.emptyList(), - UniqueConstraint.primaryKey("pk", ImmutableList.of("pk"))); - - Schema schema = - Schema.newBuilder() - .column("pk", DataTypes.INT().notNull()) - .column("test", DataTypes.INT()) - .columnByExpression("comp", "test + 1") - .primaryKey("pk") - .build(); - - Map options = new HashMap<>(); - CatalogTable catalogTable1 = - new ResolvedCatalogTable( - CatalogTable.newBuilder() - .schema(schema) - .comment("") - .partitionKeys(Collections.emptyList()) - .options(options) - .build(), - resolvedSchema); - catalog.createTable(path1, catalogTable1, false); - CatalogBaseTable storedTable1 = catalog.getTable(path1); - assertThat(storedTable1.getOptions().containsKey("testing.log.store.topic")).isFalse(); - - options.put(LOG_SYSTEM.key(), TESTING_LOG_STORE); - CatalogTable catalogTable2 = - new ResolvedCatalogTable( - CatalogTable.newBuilder() - .schema(schema) - .comment("") - .partitionKeys(Collections.emptyList()) - .options(options) - .build(), - resolvedSchema); - catalog.createTable(path3, catalogTable2, false); - - CatalogBaseTable storedTable2 = catalog.getTable(path3); - assertThat(storedTable2.getOptions().get("testing.log.store.topic")) - .isEqualTo(String.format("%s-topic", path3.getObjectName())); - assertThatThrownBy(() -> catalog.dropTable(path3, true)) - .hasMessage("Check unregister log store topic here."); - } - - @Test - public void testDisableCreateTableInDefaultDB() - throws TableAlreadyExistException, DatabaseNotExistException, - DatabaseAlreadyExistException { + public void testDisableCreateTableInDefaultDB() throws DatabaseAlreadyExistException { String path = new File(temporaryFolder.toFile(), UUID.randomUUID().toString()).toString(); Options conf = new Options(); conf.setString("warehouse", path); - conf.set(LOG_SYSTEM_AUTO_REGISTER, true); conf.set(DISABLE_CREATE_TABLE_IN_DEFAULT_DB, true); Catalog catalog = FlinkCatalogFactory.createCatalog( @@ -871,28 +790,19 @@ void testBuildPaimonTableWithCustomScheme() throws Exception { private void checkCreateTable( ObjectPath path, CatalogBaseTable expected, CatalogBaseTable actual) { - checkEquals( - path, - expected, - actual, - Collections.singletonMap( - FlinkCatalogOptions.REGISTER_TIMEOUT.key(), - FlinkCatalogOptions.REGISTER_TIMEOUT.defaultValue().toString()), - Collections.singleton(CONNECTOR.key())); + checkEquals(path, expected, actual, Collections.singleton(CONNECTOR.key())); } private void checkAlterTable(ObjectPath path, CatalogTable expected, CatalogTable actual) { - checkEquals(path, expected, actual, Collections.emptyMap(), Collections.emptySet()); + checkEquals(path, expected, actual, Collections.emptySet()); } private void checkEquals( ObjectPath path, CatalogBaseTable t1, CatalogBaseTable t2, - Map optionsToAdd, Set optionsToRemove) { Path tablePath; - Path tableDataPath; try { Map options = ((FlinkCatalog) catalog) @@ -905,7 +815,6 @@ private void checkEquals( } Map options = new HashMap<>(t1.getOptions()); options.put("path", tablePath.toString()); - options.putAll(optionsToAdd); optionsToRemove.forEach(options::remove); if (t1.getTableKind() == CatalogBaseTable.TableKind.TABLE) { t1 = ((ResolvedCatalogTable) t1).copy(options); @@ -994,54 +903,6 @@ private static Stream> optionProvider(boolean isStreaming) { return allOptions.stream(); } - /** Testing log store register factory to create {@link TestingLogStoreRegister}. */ - public static class TestingLogSoreRegisterFactory implements LogStoreTableFactory { - - @Override - public String identifier() { - return TESTING_LOG_STORE; - } - - @Override - public LogSourceProvider createSourceProvider( - DynamicTableFactory.Context context, - DynamicTableSource.Context sourceContext, - @Nullable int[][] projectFields) { - throw new UnsupportedOperationException(); - } - - @Override - public LogSinkProvider createSinkProvider( - DynamicTableFactory.Context context, DynamicTableSink.Context sinkContext) { - throw new UnsupportedOperationException(); - } - - @Override - public LogStoreRegister createRegister(RegisterContext context) { - return new TestingLogStoreRegister(context.getIdentifier()); - } - } - - /** Testing log store register. */ - private static class TestingLogStoreRegister implements LogStoreRegister { - private final Identifier table; - - private TestingLogStoreRegister(Identifier table) { - this.table = table; - } - - @Override - public Map registerTopic() { - return Collections.singletonMap( - "testing.log.store.topic", String.format("%s-topic", table.getObjectName())); - } - - @Override - public void unRegisterTopic() { - throw new UnsupportedOperationException("Check unregister log store topic here."); - } - } - private static class TestRefreshHandler implements RefreshHandler { private final String handlerString; diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ReadWriteTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ReadWriteTableITCase.java index fc7c7b7a10d2..940dd95af0b0 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ReadWriteTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ReadWriteTableITCase.java @@ -1927,8 +1927,7 @@ private void testSinkParallelism( new FlinkTableSink( context.getObjectIdentifier(), new FlinkTableFactory().buildPaimonTable(context), - context, - null); + context); assertThat(tableSink).isInstanceOf(FlinkTableSink.class); // 2. get sink provider diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperatorTest.java index ad9e9355e149..5a7a71b7b977 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperatorTest.java @@ -185,7 +185,7 @@ private void assertCommittable(Object o, long checkpointId, BinaryRow partition, assertThat(record.getValue().isLeft()).isTrue(); Committable committable = record.getValue().left(); assertThat(committable.checkpointId()).isEqualTo(checkpointId); - CommitMessageImpl message = (CommitMessageImpl) committable.wrappedCommittable(); + CommitMessageImpl message = (CommitMessageImpl) committable.commitMessage(); assertThat(message.partition()).isEqualTo(partition); assertThat(message.newFilesIncrement().deletedFiles()).isEmpty(); assertThat(message.newFilesIncrement().changelogFiles()).isEmpty(); @@ -228,7 +228,7 @@ private Committable createCommittable(long checkpointId, BinaryRow partition, in Collections.emptyList(), Collections.emptyList()), CompactIncrement.emptyIncrement()); - return new Committable(checkpointId, Committable.Kind.FILE, message); + return new Committable(checkpointId, message); } private DataFileMeta createDataFileMetaOfSize(int mb) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactCoordinateOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactCoordinateOperatorTest.java index 18ec5b4ef500..bbcc5394f99d 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactCoordinateOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactCoordinateOperatorTest.java @@ -268,7 +268,7 @@ private void assertCommittable( Committable committable = record.getValue().left(); assertThat(committable.checkpointId()).isEqualTo(1); - CommitMessageImpl message = (CommitMessageImpl) committable.wrappedCommittable(); + CommitMessageImpl message = (CommitMessageImpl) committable.commitMessage(); assertThat(message.partition()).isEqualTo(partition); assertThat(message.bucket()).isEqualTo(0); @@ -334,7 +334,7 @@ private Committable createCommittable( compactChangelogMbs.stream() .map(this::createDataFileMetaOfSize) .collect(Collectors.toList()))); - return new Committable(checkpointId, Committable.Kind.FILE, message); + return new Committable(checkpointId, message); } private DataFileMeta createDataFileMetaOfSize(int mb) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactSortOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactSortOperatorTest.java index baedafcf8af9..70530de9dc49 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactSortOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactSortOperatorTest.java @@ -81,8 +81,7 @@ public void testChangelogSorted() throws Exception { Collections.emptyList(), Collections.emptyList()), CompactIncrement.emptyIncrement()); - testHarness.processElement( - new StreamRecord<>(new Committable(1, Committable.Kind.FILE, onlyData))); + testHarness.processElement(new StreamRecord<>(new Committable(1, onlyData))); CommitMessageImpl onlyChangelogBucket0 = new CommitMessageImpl( @@ -94,9 +93,7 @@ public void testChangelogSorted() throws Exception { Collections.emptyList(), Arrays.asList(files.get(4), files.get(3))), CompactIncrement.emptyIncrement()); - testHarness.processElement( - new StreamRecord<>( - new Committable(1, Committable.Kind.FILE, onlyChangelogBucket0))); + testHarness.processElement(new StreamRecord<>(new Committable(1, onlyChangelogBucket0))); CommitMessageImpl onlyChangelogBucket1 = new CommitMessageImpl( @@ -108,9 +105,7 @@ public void testChangelogSorted() throws Exception { Collections.emptyList(), Arrays.asList(files.get(7), files.get(8))), CompactIncrement.emptyIncrement()); - testHarness.processElement( - new StreamRecord<>( - new Committable(1, Committable.Kind.FILE, onlyChangelogBucket1))); + testHarness.processElement(new StreamRecord<>(new Committable(1, onlyChangelogBucket1))); CommitMessageImpl mixed = new CommitMessageImpl( @@ -122,8 +117,7 @@ public void testChangelogSorted() throws Exception { Collections.emptyList(), Arrays.asList(files.get(6), files.get(5))), CompactIncrement.emptyIncrement()); - testHarness.processElement( - new StreamRecord<>(new Committable(1, Committable.Kind.FILE, mixed))); + testHarness.processElement(new StreamRecord<>(new Committable(1, mixed))); testHarness.prepareSnapshotPreBarrier(1); @@ -133,8 +127,7 @@ public void testChangelogSorted() throws Exception { List actual = new ArrayList<>(); for (Object o : output) { actual.add( - (CommitMessageImpl) - ((StreamRecord) o).getValue().wrappedCommittable()); + (CommitMessageImpl) ((StreamRecord) o).getValue().commitMessage()); } assertThat(actual.get(0)).isEqualTo(onlyData); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskSerializerTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskSerializerTest.java index 344200d043e7..df62115f2750 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskSerializerTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskSerializerTest.java @@ -73,7 +73,7 @@ public void testSerializer() throws Exception { put(1, newFiles(10)); } }); - serializeTask = serializer.deserialize(2, serializer.serialize(task)); + serializeTask = serializer.deserialize(serializer.getVersion(), serializer.serialize(task)); assertThat(task).isEqualTo(serializeTask); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskTest.java index 1bcccf9cbf85..a7e7fc099b9b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskTest.java @@ -169,7 +169,7 @@ public void testManyExternalPaths() throws Exception { List messages = task.doCompact(table, Executors.newFixedThreadPool(1), MemorySize.ofMebiBytes(64)) .stream() - .map(c -> (CommitMessageImpl) c.wrappedCommittable()) + .map(c -> (CommitMessageImpl) c.commitMessage()) .collect(Collectors.toList()); TableCommitImpl commit = table.newCommit("test"); commit.commit(messages); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperatorTest.java index 93baabf6b625..d2fc8e86f354 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperatorTest.java @@ -115,7 +115,7 @@ public void testAsyncCompactionWorks() throws Exception { committables.forEach( a -> Assertions.assertThat( - ((CommitMessageImpl) a.wrappedCommittable()) + ((CommitMessageImpl) a.commitMessage()) .compactIncrement() .compactAfter() .size() diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java index 27d3d9241b08..0de040f6f362 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java @@ -108,7 +108,7 @@ public void testAsyncCompactionWorks() throws Exception { committables.forEach( a -> assertThat( - ((CommitMessageImpl) a.wrappedCommittable()) + ((CommitMessageImpl) a.commitMessage()) .compactIncrement() .compactAfter() .size() diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java index d046687f6911..bbd79573ec38 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java @@ -192,8 +192,7 @@ private void processCommittable( write.write(row); } for (CommitMessage committable : write.prepareCommit(false, checkpointId)) { - testHarness.processElement( - new Committable(checkpointId, Committable.Kind.FILE, committable), timestamp); + testHarness.processElement(new Committable(checkpointId, committable), timestamp); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommittableSerializerTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommittableSerializerTest.java index 1cda7ddc181d..f3cc99292c08 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommittableSerializerTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommittableSerializerTest.java @@ -47,31 +47,11 @@ public void testFile() throws IOException { CommitMessage committable = new CommitMessageImpl(row(0), 1, 2, dataIncrement, compactIncrement); CommitMessage newCommittable = - (CommitMessage) - serializer - .deserialize( - 2, - serializer.serialize( - new Committable( - 9, Committable.Kind.FILE, committable))) - .wrappedCommittable(); - assertThat(newCommittable).isEqualTo(committable); - } - - @Test - public void testLogOffset() throws IOException { - LogOffsetCommittable committable = new LogOffsetCommittable(2, 3); - LogOffsetCommittable newCommittable = - (LogOffsetCommittable) - serializer - .deserialize( - 2, - serializer.serialize( - new Committable( - 8, - Committable.Kind.LOG_OFFSET, - committable))) - .wrappedCommittable(); + serializer + .deserialize( + serializer.getVersion(), + serializer.serialize(new Committable(9, committable))) + .commitMessage(); assertThat(newCommittable).isEqualTo(committable); } } 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 0e40ad94cf58..4c4bd4350e96 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 @@ -101,8 +101,7 @@ public void testFailIntentionallyAfterRestore() throws Exception { long timestamp = 1; for (CommitMessage committable : write.prepareCommit(false, 8)) { - testHarness.processElement( - new Committable(8, Committable.Kind.FILE, committable), timestamp++); + testHarness.processElement(new Committable(8, committable), timestamp++); } // checkpoint is completed but not notified, so no snapshot is committed OperatorSubtaskState snapshot = testHarness.snapshot(0, timestamp++); @@ -151,8 +150,7 @@ public void testCheckpointAbort() throws Exception { write.write(GenericRow.of(1, 10L)); write.write(GenericRow.of(2, 20L)); for (CommitMessage committable : write.prepareCommit(false, cpId)) { - testHarness.processElement( - new Committable(cpId, Committable.Kind.FILE, committable), 1); + testHarness.processElement(new Committable(cpId, committable), 1); } } @@ -188,8 +186,7 @@ public void testSnapshotLostWhenFailed() throws Exception { write.write(GenericRow.of(1, 10L)); write.write(GenericRow.of(2, 20L)); for (CommitMessage committable : write.prepareCommit(false, 1)) { - testHarness.processElement( - new Committable(1, Committable.Kind.FILE, committable), timestamp++); + testHarness.processElement(new Committable(1, committable), timestamp++); } testHarness.snapshot(1, timestamp++); testHarness.notifyOfCompletedCheckpoint(1); @@ -198,8 +195,7 @@ public void testSnapshotLostWhenFailed() throws Exception { write.write(GenericRow.of(3, 30L)); write.write(GenericRow.of(4, 40L)); for (CommitMessage committable : write.prepareCommit(false, 2)) { - testHarness.processElement( - new Committable(2, Committable.Kind.FILE, committable), timestamp++); + testHarness.processElement(new Committable(2, committable), timestamp++); } OperatorSubtaskState snapshot = testHarness.snapshot(2, timestamp++); @@ -216,8 +212,7 @@ public void testSnapshotLostWhenFailed() throws Exception { write.write(GenericRow.of(5, 50L)); write.write(GenericRow.of(6, 60L)); for (CommitMessage committable : write.prepareCommit(false, 3)) { - testHarness.processElement( - new Committable(3, Committable.Kind.FILE, committable), timestamp++); + testHarness.processElement(new Committable(3, committable), timestamp++); } testHarness.snapshot(3, timestamp++); testHarness.notifyOfCompletedCheckpoint(3); @@ -332,7 +327,6 @@ public void testCommitInputEnd() throws Exception { testHarness.processElement( new Committable( Long.MAX_VALUE, - Committable.Kind.FILE, new CommitMessageImpl( BinaryRow.EMPTY_ROW, 0, @@ -350,7 +344,6 @@ public void testCommitInputEnd() throws Exception { testHarness.processElement( new Committable( Long.MAX_VALUE, - Committable.Kind.FILE, new CommitMessageImpl( BinaryRow.EMPTY_ROW, 0, @@ -367,7 +360,6 @@ public void testCommitInputEnd() throws Exception { testHarness.processElement( new Committable( Long.MAX_VALUE, - Committable.Kind.FILE, new CommitMessageImpl( BinaryRow.EMPTY_ROW, 0, @@ -403,7 +395,6 @@ public void testCommitInputEnd() throws Exception { testHarness.processElement( new Committable( 0L, - Committable.Kind.FILE, new CommitMessageImpl( BinaryRow.EMPTY_ROW, 0, @@ -421,7 +412,6 @@ public void testCommitInputEnd() throws Exception { testHarness.processElement( new Committable( 0L, - Committable.Kind.FILE, new CommitMessageImpl( BinaryRow.EMPTY_ROW, 0, @@ -438,7 +428,6 @@ public void testCommitInputEnd() throws Exception { testHarness.processElement( new Committable( Long.MAX_VALUE, - Committable.Kind.FILE, new CommitMessageImpl( BinaryRow.EMPTY_ROW, 0, @@ -472,8 +461,7 @@ private static OperatorSubtaskState writeAndSnapshot( StreamTableWrite write = streamWriteBuilder.withCommitUser(commitUser).newWrite(); write.write(GenericRow.of(1, 10L)); for (CommitMessage committable : write.prepareCommit(false, 1)) { - testHarness.processElement( - new Committable(checkpoint, Committable.Kind.FILE, committable), ++timestamp); + testHarness.processElement(new Committable(checkpoint, committable), ++timestamp); } OperatorSubtaskState snapshot = testHarness.snapshot(checkpoint, ++timestamp); return snapshot; @@ -493,9 +481,7 @@ public void testWatermarkCommit() throws Exception { long cpId = 1; write.write(GenericRow.of(1, 10L)); testHarness.processElement( - new Committable( - cpId, Committable.Kind.FILE, write.prepareCommit(true, cpId).get(0)), - timestamp++); + new Committable(cpId, write.prepareCommit(true, cpId).get(0)), timestamp++); testHarness.processWatermark(new Watermark(1024)); testHarness.snapshot(cpId, timestamp++); testHarness.notifyOfCompletedCheckpoint(cpId); @@ -504,9 +490,7 @@ public void testWatermarkCommit() throws Exception { cpId = 2; write.write(GenericRow.of(1, 20L)); testHarness.processElement( - new Committable( - cpId, Committable.Kind.FILE, write.prepareCommit(true, cpId).get(0)), - timestamp++); + new Committable(cpId, write.prepareCommit(true, cpId).get(0)), timestamp++); testHarness.processWatermark(new Watermark(Long.MAX_VALUE)); testHarness.snapshot(cpId, timestamp++); testHarness.notifyOfCompletedCheckpoint(cpId); @@ -705,9 +689,7 @@ public void testCommitMetrics() throws Exception { long cpId = 1; write.write(GenericRow.of(1, 100L)); testHarness.processElement( - new Committable( - cpId, Committable.Kind.FILE, write.prepareCommit(false, cpId).get(0)), - timestamp++); + new Committable(cpId, write.prepareCommit(false, cpId).get(0)), timestamp++); testHarness.snapshot(cpId, timestamp++); testHarness.notifyOfCompletedCheckpoint(cpId); @@ -742,9 +724,7 @@ public void testCommitMetrics() throws Exception { // real compaction write.compact(BinaryRow.EMPTY_ROW, 0, true); testHarness.processElement( - new Committable( - cpId, Committable.Kind.FILE, write.prepareCommit(true, cpId).get(0)), - timestamp++); + new Committable(cpId, write.prepareCommit(true, cpId).get(0)), timestamp++); testHarness.snapshot(cpId, timestamp++); testHarness.notifyOfCompletedCheckpoint(cpId); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactionTaskSimpleSerializerTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactionTaskSimpleSerializerTest.java index 7c22f125fff0..b85b5492373d 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactionTaskSimpleSerializerTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactionTaskSimpleSerializerTest.java @@ -50,7 +50,8 @@ public class CompactionTaskSimpleSerializerTest { public void testSerializer() throws IOException { AppendCompactTask task1 = new AppendCompactTask(partition, newFiles(20)); - AppendCompactTask task2 = serializer.deserialize(2, serializer.serialize(task1)); + AppendCompactTask task2 = + serializer.deserialize(serializer.getVersion(), serializer.serialize(task1)); assertThat(task1).isEqualTo(task2); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/LogOffsetCommittableTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/LogOffsetCommittableTest.java deleted file mode 100644 index 74dda8ac6502..000000000000 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/LogOffsetCommittableTest.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.sink; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Test for {@link LogOffsetCommittable}. */ -public class LogOffsetCommittableTest { - - @Test - public void test() { - LogOffsetCommittable committable = new LogOffsetCommittable(5, 9); - LogOffsetCommittable deser = LogOffsetCommittable.fromBytes(committable.toBytes()); - assertThat(deser.bucket()).isEqualTo(committable.bucket()); - assertThat(deser.offset()).isEqualTo(committable.offset()); - } -} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/MultiTableCommittableSerializerTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/MultiTableCommittableSerializerTest.java index 549595ec7b01..086041c94054 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/MultiTableCommittableSerializerTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/MultiTableCommittableSerializerTest.java @@ -50,7 +50,7 @@ public void testDeserialize() { CompactIncrement compactIncrement = randomCompactIncrement(); CommitMessage commitMessage = new CommitMessageImpl(row(0), 1, 2, dataIncrement, compactIncrement); - Committable committable = new Committable(9, Committable.Kind.FILE, commitMessage); + Committable committable = new Committable(9, commitMessage); Arrays.asList(Tuple2.of("测试数据库", "用户信息表"), Tuple2.of("database", "table")) .forEach( @@ -64,7 +64,8 @@ public void testDeserialize() { try { deserializeCommittable = serializer.deserialize( - 2, serializer.serialize(multiTableCommittable)); + serializer.getVersion(), + serializer.serialize(multiTableCommittable)); } catch (IOException e) { throw new RuntimeException(e); } @@ -83,7 +84,7 @@ public void testSerialize() { CompactIncrement compactIncrement = randomCompactIncrement(); CommitMessage commitMessage = new CommitMessageImpl(row(0), 1, 2, newFilesIncrement, compactIncrement); - Committable committable = new Committable(9, Committable.Kind.FILE, commitMessage); + Committable committable = new Committable(9, commitMessage); Arrays.asList(Tuple2.of("测试数据库", "用户信息表"), Tuple2.of("database", "table")) .forEach( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/RowDataChannelComputerTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/RowDataChannelComputerTest.java index f90a3cabbfb8..687407141bf1 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/RowDataChannelComputerTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/RowDataChannelComputerTest.java @@ -38,10 +38,8 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.ThreadLocalRandom; import static org.assertj.core.api.Assertions.assertThat; @@ -113,8 +111,7 @@ private void testImpl(TableSchema schema, List input) { FixedBucketRowKeyExtractor extractor = new FixedBucketRowKeyExtractor(schema); int numChannels = random.nextInt(10) + 1; - boolean hasLogSink = random.nextBoolean(); - RowDataChannelComputer channelComputer = new RowDataChannelComputer(schema, hasLogSink); + RowDataChannelComputer channelComputer = new RowDataChannelComputer(schema); channelComputer.setup(numChannels); // assert that channel(record) and channel(partition, bucket) gives the same result @@ -150,22 +147,5 @@ private void testImpl(TableSchema schema, List input) { int min = bucketsPerChannel.values().stream().min(Integer::compareTo).get(); assertThat(max - min).isLessThanOrEqualTo(1); } - - // log sinks like Kafka only consider bucket and don't care about partition - // so same bucket, even from different partition, must go to the same channel - - if (hasLogSink) { - Map> channelsPerBucket = new HashMap<>(); - for (InternalRow rowData : input) { - extractor.setRecord(rowData); - int bucket = extractor.bucket(); - channelsPerBucket - .computeIfAbsent(bucket, k -> new HashSet<>()) - .add(channelComputer.channel(rowData)); - } - for (Set channels : channelsPerBucket.values()) { - assertThat(channels).hasSize(1); - } - } } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreCompactOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreCompactOperatorTest.java index 08ffc85b1443..a89cfa66f011 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreCompactOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreCompactOperatorTest.java @@ -257,11 +257,6 @@ public SinkRecord write(InternalRow rowData, int bucket) { return null; } - @Override - public SinkRecord toLogRecord(SinkRecord record) { - return null; - } - @Override public void compact(BinaryRow partition, int bucket, boolean fullCompaction) { compactTime++; 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 7ab11df3b72a..f09f145ad1dd 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 @@ -178,8 +178,7 @@ public void testFailIntentionallyAfterRestore() throws Exception { long timestamp = 1; for (CommitMessage committable : write.prepareCommit(false, 8)) { testHarness.processElement( - getMultiTableCommittable( - firstTable, new Committable(8, Committable.Kind.FILE, committable)), + getMultiTableCommittable(firstTable, new Committable(8, committable)), timestamp++); } // checkpoint is completed but not notified, so no snapshot is committed @@ -219,8 +218,7 @@ firstTable, new Committable(8, Committable.Kind.FILE, committable)), for (CommitMessage committable : write.prepareCommit(false, 9)) { testHarness.processElement( - getMultiTableCommittable( - secondTable, new Committable(9, Committable.Kind.FILE, committable)), + getMultiTableCommittable(secondTable, new Committable(9, committable)), timestamp++); } @@ -280,9 +278,7 @@ public void testCheckpointAbort() throws Exception { write1.write(GenericRow.of(2, 20L)); for (CommitMessage committable : write1.prepareCommit(false, cpId)) { testHarness.processElement( - getMultiTableCommittable( - firstTable, - new Committable(cpId, Committable.Kind.FILE, committable)), + getMultiTableCommittable(firstTable, new Committable(cpId, committable)), 1); } } @@ -309,17 +305,13 @@ public void testCheckpointAbort() throws Exception { write2.write(GenericRow.of(3, 40.0, BinaryString.fromString("s4"))); for (CommitMessage committable : write1.prepareCommit(false, cpId)) { testHarness.processElement( - getMultiTableCommittable( - firstTable, - new Committable(cpId, Committable.Kind.FILE, committable)), + getMultiTableCommittable(firstTable, new Committable(cpId, committable)), 1); } for (CommitMessage committable : write2.prepareCommit(false, cpId)) { testHarness.processElement( - getMultiTableCommittable( - secondTable, - new Committable(cpId, Committable.Kind.FILE, committable)), + getMultiTableCommittable(secondTable, new Committable(cpId, committable)), 1); } } @@ -364,8 +356,7 @@ public void testSnapshotLostWhenFailed() throws Exception { write1.write(GenericRow.of(2, 20L)); for (CommitMessage committable : write1.prepareCommit(false, 1)) { testHarness.processElement( - getMultiTableCommittable( - firstTable, new Committable(1, Committable.Kind.FILE, committable)), + getMultiTableCommittable(firstTable, new Committable(1, committable)), timestamp++); } testHarness.snapshot(1, timestamp++); @@ -378,14 +369,12 @@ firstTable, new Committable(1, Committable.Kind.FILE, committable)), write2.write(GenericRow.of(3, 40.0, BinaryString.fromString("s4"))); for (CommitMessage committable : write1.prepareCommit(false, 2)) { testHarness.processElement( - getMultiTableCommittable( - firstTable, new Committable(2, Committable.Kind.FILE, committable)), + getMultiTableCommittable(firstTable, new Committable(2, committable)), timestamp++); } for (CommitMessage committable : write2.prepareCommit(false, 2)) { testHarness.processElement( - getMultiTableCommittable( - secondTable, new Committable(2, Committable.Kind.FILE, committable)), + getMultiTableCommittable(secondTable, new Committable(2, committable)), timestamp++); } OperatorSubtaskState snapshot = testHarness.snapshot(2, timestamp++); @@ -408,14 +397,12 @@ secondTable, new Committable(2, Committable.Kind.FILE, committable)), write2.write(GenericRow.of(6, 60.0, BinaryString.fromString("s6"))); for (CommitMessage committable : write1.prepareCommit(false, 3)) { testHarness.processElement( - getMultiTableCommittable( - firstTable, new Committable(3, Committable.Kind.FILE, committable)), + getMultiTableCommittable(firstTable, new Committable(3, committable)), timestamp++); } for (CommitMessage committable : write2.prepareCommit(false, 2)) { testHarness.processElement( - getMultiTableCommittable( - secondTable, new Committable(2, Committable.Kind.FILE, committable)), + getMultiTableCommittable(secondTable, new Committable(2, committable)), timestamp++); } testHarness.snapshot(3, timestamp); @@ -450,11 +437,7 @@ public void testWatermarkCommit() throws Exception { write1.write(GenericRow.of(1, 10L)); testHarness.processElement( getMultiTableCommittable( - firstTable, - new Committable( - cpId, - Committable.Kind.FILE, - write1.prepareCommit(true, cpId).get(0))), + firstTable, new Committable(cpId, write1.prepareCommit(true, cpId).get(0))), timestamp++); testHarness.processWatermark(new Watermark(1024)); testHarness.snapshot(cpId, timestamp++); @@ -469,11 +452,7 @@ public void testWatermarkCommit() throws Exception { write2.write(GenericRow.of(1, 20.0, BinaryString.fromString("s2"))); testHarness.processElement( getMultiTableCommittable( - firstTable, - new Committable( - cpId, - Committable.Kind.FILE, - write1.prepareCommit(true, cpId).get(0))), + firstTable, new Committable(cpId, write1.prepareCommit(true, cpId).get(0))), timestamp++); testHarness.processWatermark(new Watermark(2048)); testHarness.snapshot(cpId, timestamp); @@ -508,19 +487,12 @@ public void testEmptyCommit() throws Exception { write2.write(GenericRow.of(1, 20.0, BinaryString.fromString("s2"))); testHarness.processElement( getMultiTableCommittable( - firstTable, - new Committable( - cpId, - Committable.Kind.FILE, - write1.prepareCommit(true, cpId).get(0))), + firstTable, new Committable(cpId, write1.prepareCommit(true, cpId).get(0))), timestamp++); testHarness.processElement( getMultiTableCommittable( secondTable, - new Committable( - cpId, - Committable.Kind.FILE, - write2.prepareCommit(true, cpId).get(0))), + new Committable(cpId, write2.prepareCommit(true, cpId).get(0))), timestamp++); testHarness.processWatermark(new Watermark(2048)); testHarness.snapshot(cpId, timestamp++); @@ -570,19 +542,12 @@ public void testCommitMetrics() throws Exception { write2.compact(BinaryRow.EMPTY_ROW, 0, true); testHarness.processElement( getMultiTableCommittable( - firstTable, - new Committable( - cpId, - Committable.Kind.FILE, - write1.prepareCommit(true, cpId).get(0))), + firstTable, new Committable(cpId, write1.prepareCommit(true, cpId).get(0))), timestamp++); testHarness.processElement( getMultiTableCommittable( secondTable, - new Committable( - cpId, - Committable.Kind.FILE, - write2.prepareCommit(true, cpId).get(0))), + new Committable(cpId, write2.prepareCommit(true, cpId).get(0))), timestamp++); testHarness.snapshot(cpId, timestamp); testHarness.notifyOfCompletedCheckpoint(cpId); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WrappedManifestCommittableSerializerTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WrappedManifestCommittableSerializerTest.java index f82bfc4278ac..7390f7c4da32 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WrappedManifestCommittableSerializerTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WrappedManifestCommittableSerializerTest.java @@ -30,8 +30,6 @@ import org.junit.jupiter.api.Test; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; @@ -85,7 +83,6 @@ public static ManifestCommittable createManifestCommittable() { public static void addFileCommittables( ManifestCommittable committable, BinaryRow partition, int bucket, int totalBuckets) { - List commitMessages = new ArrayList<>(); int length = ThreadLocalRandom.current().nextInt(10) + 1; for (int i = 0; i < length; i++) { DataIncrement dataIncrement = randomNewFilesIncrement(); @@ -93,14 +90,7 @@ public static void addFileCommittables( CommitMessage commitMessage = new CommitMessageImpl( partition, bucket, totalBuckets, dataIncrement, compactIncrement); - commitMessages.add(commitMessage); committable.addFileCommittable(commitMessage); } - - if (!committable.logOffsets().containsKey(bucket)) { - int offset = ID.incrementAndGet(); - committable.addLogOffset(bucket, offset, false); - assertThat(committable.logOffsets().get(bucket)).isEqualTo(offset); - } } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java index 276bf45193f3..c7cca8eceb1d 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java @@ -489,7 +489,7 @@ public void testNumWritersMetric() throws Exception { commit.commit( 1, harness.extractOutputValues().stream() - .map(c -> (CommitMessage) c.wrappedCommittable()) + .map(Committable::commitMessage) .collect(Collectors.toList())); assertThat(numWriters.getValue()).isEqualTo(3); @@ -507,7 +507,7 @@ public void testNumWritersMetric() throws Exception { commit.commit( 2, harness.extractOutputValues().stream() - .map(c -> (CommitMessage) c.wrappedCommittable()) + .map(Committable::commitMessage) .collect(Collectors.toList())); harness.prepareSnapshotPreBarrier(3); @@ -529,7 +529,6 @@ private RowDataStoreWriteOperator.Factory getStoreSinkWriteOperatorFactory( FileStoreTable fileStoreTable) { return new RowDataStoreWriteOperator.Factory( fileStoreTable, - null, (table, commitUser, state, ioManager, memoryPool, metricGroup) -> new StoreSinkWriteImpl( table, @@ -548,7 +547,6 @@ private RowDataStoreWriteOperator.Factory getLookupWriteOperatorFactory( FileStoreTable fileStoreTable, boolean waitCompaction) { return new RowDataStoreWriteOperator.Factory( fileStoreTable, - null, (table, commitUser, state, ioManager, memoryPoolFactory, metricGroup) -> new LookupSinkWrite( table, @@ -572,8 +570,7 @@ private void commitAll( while (!harness.getOutput().isEmpty()) { Committable committable = ((StreamRecord) harness.getOutput().poll()).getValue(); - assertThat(committable.kind()).isEqualTo(Committable.Kind.FILE); - commitMessages.add((CommitMessage) committable.wrappedCommittable()); + commitMessages.add(committable.commitMessage()); } commit.commit(commitIdentifier, commitMessages); } @@ -587,8 +584,7 @@ private void commitAppend( while (!harness.getOutput().isEmpty()) { Committable committable = ((StreamRecord) harness.getOutput().poll()).getValue(); - assertThat(committable.kind()).isEqualTo(Committable.Kind.FILE); - CommitMessageImpl message = (CommitMessageImpl) committable.wrappedCommittable(); + CommitMessageImpl message = (CommitMessageImpl) committable.commitMessage(); CommitMessageImpl newMessage = new CommitMessageImpl( message.partition(), diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/DataTableSourceTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/DataTableSourceTest.java index 568e7f748bb7..260da03f63a9 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/DataTableSourceTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/DataTableSourceTest.java @@ -74,11 +74,7 @@ void testInferScanParallelism() throws Exception { DataTableSource tableSource = new DataTableSource( - ObjectIdentifier.of("cat", "db", "table"), - fileStoreTable, - true, - null, - null); + ObjectIdentifier.of("cat", "db", "table"), fileStoreTable, true, null); PaimonDataStreamScanProvider runtimeProvider = runtimeProvider(tableSource); StreamExecutionEnvironment sEnv1 = StreamExecutionEnvironment.createLocalEnvironment(); sEnv1.setParallelism(-1); @@ -108,11 +104,7 @@ public void testInferStreamParallelism() throws Exception { DataTableSource tableSource = new DataTableSource( - ObjectIdentifier.of("cat", "db", "table"), - fileStoreTable, - true, - null, - null); + ObjectIdentifier.of("cat", "db", "table"), fileStoreTable, true, null); PaimonDataStreamScanProvider runtimeProvider = runtimeProvider(tableSource); StreamExecutionEnvironment sEnv1 = StreamExecutionEnvironment.createLocalEnvironment(); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceMetricsTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceMetricsTest.java index 348a194c025c..164ab844665c 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceMetricsTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceMetricsTest.java @@ -126,20 +126,6 @@ public void continuousFileStoreSourceScanMetricsTest() throws Exception { .isEqualTo(1L); } - @Test - public void logHybridFileStoreSourceScanMetricsTest() throws Exception { - writeOnce(); - FlinkSource logHybridFileStoreSource = - LogHybridSourceFactory.buildHybridFirstSource(table, null, null, null, null); - logHybridFileStoreSource.restoreEnumerator(context, null); - assertThat(TestingMetricUtils.getGauge(scanMetricGroup, "lastScannedManifests").getValue()) - .isEqualTo(1L); - assertThat( - TestingMetricUtils.getGauge(scanMetricGroup, "lastScanResultedTableFiles") - .getValue()) - .isEqualTo(1L); - } - private void writeOnce() throws Exception { InnerTableWrite writer = table.newWrite("test"); TableCommitImpl commit = table.newCommit("test"); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FlinkTableSourceTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FlinkTableSourceTest.java index cff9ab6f4d25..4965cc41350a 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FlinkTableSourceTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FlinkTableSourceTest.java @@ -54,7 +54,7 @@ public void testApplyFilterNonPartitionTable() throws Exception { Table table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath, tableSchema); DataTableSource tableSource = new DataTableSource( - ObjectIdentifier.of("catalog1", "db1", "T"), table, false, null, null); + ObjectIdentifier.of("catalog1", "db1", "T"), table, false, null); // col1 = 1 List filters = ImmutableList.of(col1Equal1()); @@ -78,7 +78,7 @@ public void testApplyPartitionTable() throws Exception { Table table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath, tableSchema); FlinkTableSource tableSource = new DataTableSource( - ObjectIdentifier.of("catalog1", "db1", "T"), table, false, null, null); + ObjectIdentifier.of("catalog1", "db1", "T"), table, false, null); // col1 = 1 && p1 = 1 => [p1 = 1] List filters = ImmutableList.of(col1Equal1(), p1Equal1()); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/FileStoreTableStatisticsTestBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/FileStoreTableStatisticsTestBase.java index 90e5e6e81138..05435c35f81e 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/FileStoreTableStatisticsTestBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/FileStoreTableStatisticsTestBase.java @@ -89,7 +89,7 @@ public void testTableScanStatistics() throws Exception { latestSnapshot.id(), latestSnapshot.schemaId(), 9L, null, colStatsMap); commit.updateStatistics(colStats); commit.close(); - DataTableSource scanSource = new DataTableSource(identifier, table, false, null, null); + DataTableSource scanSource = new DataTableSource(identifier, table, false, null); Assertions.assertThat(scanSource.reportStatistics().getRowCount()).isEqualTo(9L); Map expectedColStats = new HashMap<>(); expectedColStats.put( @@ -131,7 +131,7 @@ public void testTableScanStatistics() throws Exception { @Test public void testTableStreamingStatistics() throws Exception { FileStoreTable table = writeData(); - DataTableSource streamSource = new DataTableSource(identifier, table, true, null, null); + DataTableSource streamSource = new DataTableSource(identifier, table, true, null); Assertions.assertThat(streamSource.reportStatistics()).isEqualTo(TableStats.UNKNOWN); } @@ -145,7 +145,6 @@ public void testTableFilterPartitionStatistics() throws Exception { table, false, null, - null, builder.equal(0, 1), null, null, @@ -223,7 +222,6 @@ public void testTableFilterKeyStatistics() throws Exception { table, false, null, - null, builder.equal(1, 50), null, null, @@ -301,7 +299,6 @@ public void testTableFilterValueStatistics() throws Exception { table, false, null, - null, builder.greaterThan(2, 500L), null, null, diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/PrimaryKeyTableStatisticsTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/PrimaryKeyTableStatisticsTest.java index ea47df2d9d72..cac7a718a9bf 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/PrimaryKeyTableStatisticsTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/PrimaryKeyTableStatisticsTest.java @@ -46,7 +46,6 @@ public void testTableFilterValueStatistics() throws Exception { table, false, null, - null, builder.greaterThan(2, 500L), null, null, diff --git a/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory index 39300b1797a3..f9cb87a5209f 100644 --- a/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -13,8 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.paimon.flink.FlinkCatalogTest$TestingLogSoreRegisterFactory - # Catalog lock factory org.apache.paimon.flink.FileSystemCatalogITCase$FileSystemCatalogDummyLockFactory diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/flink/table/hive/LegacyHiveClasses.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/flink/table/hive/LegacyHiveClasses.java deleted file mode 100644 index 7a0ba8f8868e..000000000000 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/flink/table/hive/LegacyHiveClasses.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.hive; - -import org.apache.hadoop.hive.metastore.api.Table; - -/** Legacy hive classes of table store 0.3. */ -@Deprecated -public class LegacyHiveClasses { - - private static final String LEGACY_INPUT_FORMAT_CLASS_NAME = - "org.apache.flink.table.store.mapred.TableStoreInputFormat"; - private static final String LEGACY_OUTPUT_FORMAT_CLASS_NAME = - "org.apache.flink.table.store.mapred.TableStoreOutputFormat"; - - public static boolean isPaimonTable(Table table) { - return LEGACY_INPUT_FORMAT_CLASS_NAME.equals(table.getSd().getInputFormat()) - && LEGACY_OUTPUT_FORMAT_CLASS_NAME.equals(table.getSd().getOutputFormat()); - } -} 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 50fc94983564..f63cd4846447 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 @@ -58,7 +58,6 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; -import org.apache.flink.table.hive.LegacyHiveClasses; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hive.conf.HiveConf; @@ -1382,10 +1381,8 @@ public Table getHmsTable(Identifier identifier) } private static boolean isPaimonTable(Table table) { - boolean isPaimonTable = - INPUT_FORMAT_CLASS_NAME.equals(table.getSd().getInputFormat()) - && OUTPUT_FORMAT_CLASS_NAME.equals(table.getSd().getOutputFormat()); - return isPaimonTable || LegacyHiveClasses.isPaimonTable(table); + return INPUT_FORMAT_CLASS_NAME.equals(table.getSd().getInputFormat()) + && OUTPUT_FORMAT_CLASS_NAME.equals(table.getSd().getOutputFormat()); } private boolean isFormatTable(Table table) { diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/hive/TableStoreHiveStorageHandler.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/hive/TableStoreHiveStorageHandler.java deleted file mode 100644 index 0842b280af28..000000000000 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/hive/TableStoreHiveStorageHandler.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.store.hive; - -import org.apache.paimon.hive.PaimonStorageHandler; - -/** A {@link PaimonStorageHandler} to be compatible to table store 0.3. */ -@Deprecated -public class TableStoreHiveStorageHandler extends PaimonStorageHandler {} diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/hive/TableStoreSerDe.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/hive/TableStoreSerDe.java deleted file mode 100644 index 38e44958b69b..000000000000 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/hive/TableStoreSerDe.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.store.hive; - -import org.apache.paimon.hive.PaimonSerDe; - -/** A {@link PaimonSerDe} to be compatible to table store 0.3. */ -@Deprecated -public class TableStoreSerDe extends PaimonSerDe {} diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/mapred/TableStoreInputFormat.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/mapred/TableStoreInputFormat.java deleted file mode 100644 index 0a0bf3669a73..000000000000 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/mapred/TableStoreInputFormat.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.store.mapred; - -import org.apache.paimon.hive.mapred.PaimonInputFormat; - -/** A {@link PaimonInputFormat} to be compatible to table store 0.3. */ -@Deprecated -public class TableStoreInputFormat extends PaimonInputFormat {} diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/mapred/TableStoreOutputFormat.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/mapred/TableStoreOutputFormat.java deleted file mode 100644 index 49629832e3c2..000000000000 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/flink/table/store/mapred/TableStoreOutputFormat.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.store.mapred; - -import org.apache.paimon.hive.mapred.PaimonOutputFormat; - -/** A {@link PaimonOutputFormat} to be compatible to table store 0.3. */ -@Deprecated -public class TableStoreOutputFormat extends PaimonOutputFormat {} diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkReadITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkReadITCase.java index 25d7a396227d..5ed50328ec29 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkReadITCase.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkReadITCase.java @@ -108,7 +108,7 @@ public void testSnapshotsTableWithRecordCount() { "delta_record_count", "changelog_record_count") .collectAsList(); - assertThat(rows.toString()).isEqualTo("[[1,3,3,0]]"); + assertThat(rows.toString()).isEqualTo("[[1,3,3,null]]"); } @Test diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/TableValuedFunctionsTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/TableValuedFunctionsTest.scala index ecddb8ac6323..b3012e2f90b7 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/TableValuedFunctionsTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/TableValuedFunctionsTest.scala @@ -157,30 +157,15 @@ class TableValuedFunctionsTest extends PaimonHiveTestBase { write.write(GenericRow.of(1, BinaryString.fromString("a"))) var commitMessages = write.prepareCommit(false, 0) - commit.commit( - new ManifestCommittable( - 0, - utcMills("2024-12-02T10:00:00"), - Collections.emptyMap[Integer, java.lang.Long], - commitMessages)) + commit.commit(new ManifestCommittable(0, utcMills("2024-12-02T10:00:00"), commitMessages)) write.write(GenericRow.of(2, BinaryString.fromString("b"))) commitMessages = write.prepareCommit(false, 1) - commit.commit( - new ManifestCommittable( - 1, - utcMills("2024-12-03T10:00:00"), - Collections.emptyMap[Integer, java.lang.Long], - commitMessages)) + commit.commit(new ManifestCommittable(1, utcMills("2024-12-03T10:00:00"), commitMessages)) write.write(GenericRow.of(3, BinaryString.fromString("c"))) commitMessages = write.prepareCommit(false, 2) - commit.commit( - new ManifestCommittable( - 2, - utcMills("2024-12-05T10:00:00"), - Collections.emptyMap[Integer, java.lang.Long], - commitMessages)) + commit.commit(new ManifestCommittable(2, utcMills("2024-12-05T10:00:00"), commitMessages)) checkAnswer( sql(s"SELECT * FROM paimon_incremental_to_auto_tag('t', '2024-12-01') ORDER BY a"), From b502e5be58067a446772207138d955326c65e96d Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Mon, 29 Dec 2025 14:45:44 +0800 Subject: [PATCH 2/3] fix --- .../paimon/flink/CatalogTableITCase.java | 2 +- .../flink/ContinuousFileStoreITCase.java | 37 ++++--------------- 2 files changed, 8 insertions(+), 31 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java index a79f7dc28123..813a0e67fe9a 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java @@ -148,7 +148,7 @@ public void testSnapshotsTableWithRecordCount() throws Exception { sql( "SELECT snapshot_id, total_record_count, delta_record_count, changelog_record_count FROM T$snapshots"); assertThat(result) - .containsExactlyInAnyOrder(Row.of(1L, 1L, 1L, 0L), Row.of(2L, 2L, 1L, 0L)); + .containsExactlyInAnyOrder(Row.of(1L, 1L, 1L, null), Row.of(2L, 2L, 1L, null)); } @Test diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java index 317ca2417535..5980d627d1b6 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java @@ -27,7 +27,6 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; import org.apache.flink.table.api.StatementSet; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.planner.factories.TestValuesTableFactory; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; @@ -227,7 +226,7 @@ public void testContinuousLatest() throws Exception { BlockingIterator iterator = BlockingIterator.of( - streamSqlIter("SELECT * FROM T1 /*+ OPTIONS('log.scan'='latest') */")); + streamSqlIter("SELECT * FROM T1 /*+ OPTIONS('scan.mode'='latest') */")); batchSql("INSERT INTO T1 VALUES ('7', '8', '9'), ('10', '11', '12')"); assertThat(iterator.collect(2)) @@ -255,7 +254,7 @@ public void testContinuousLatestStartingFromEmpty() throws Exception { @Test public void testContinuousFromTimestamp() throws Exception { String sql = - "SELECT * FROM T1 /*+ OPTIONS('log.scan'='from-timestamp', 'log.scan.timestamp-millis'='%s') */"; + "SELECT * FROM T1 /*+ OPTIONS('scan.mode'='from-timestamp', 'scan.timestamp-millis'='%s') */"; // empty table BlockingIterator iterator = BlockingIterator.of(streamSqlIter(sql, 0)); @@ -323,7 +322,7 @@ public void testLackStartupTimestamp() { assertThatThrownBy( () -> streamSqlIter( - "SELECT * FROM T1 /*+ OPTIONS('log.scan'='from-timestamp') */")) + "SELECT * FROM T1 /*+ OPTIONS('scan.mode'='from-timestamp') */")) .hasCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage( "must set only one key in [scan.timestamp-millis,scan.timestamp] when you use from-timestamp for scan.mode"); @@ -331,11 +330,11 @@ public void testLackStartupTimestamp() { @Test public void testConfigureStartupTimestamp() throws Exception { - // Configure 'log.scan.timestamp-millis' without 'log.scan'. + // Configure 'scan.timestamp-millis' without 'scan.mode'. BlockingIterator iterator = BlockingIterator.of( streamSqlIter( - "SELECT * FROM T1 /*+ OPTIONS('log.scan.timestamp-millis'='%s') */", + "SELECT * FROM T1 /*+ OPTIONS('scan.timestamp-millis'='%s') */", 0)); batchSql("INSERT INTO T1 VALUES ('1', '2', '3'), ('4', '5', '6')"); batchSql("INSERT INTO T1 VALUES ('7', '8', '9'), ('10', '11', '12')"); @@ -343,11 +342,11 @@ public void testConfigureStartupTimestamp() throws Exception { .containsExactlyInAnyOrder(Row.of("1", "2", "3"), Row.of("4", "5", "6")); iterator.close(); - // Configure 'log.scan.timestamp-millis' with 'log.scan=latest'. + // Configure 'scan.timestamp-millis' with 'scan.mode=latest'. assertThatThrownBy( () -> streamSqlIter( - "SELECT * FROM T1 /*+ OPTIONS('log.scan'='latest', 'log.scan.timestamp-millis'='%s') */", + "SELECT * FROM T1 /*+ OPTIONS('scan.mode'='latest', 'scan.timestamp-millis'='%s') */", 0)) .hasCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage( @@ -466,28 +465,6 @@ public void testIgnoreOverwrite() throws Exception { iterator.close(); } - @Test - public void testUnsupportedUpsert() { - assertThatThrownBy( - () -> - streamSqlIter( - "SELECT * FROM T1 /*+ OPTIONS('log.changelog-mode'='upsert') */")) - .hasCauseInstanceOf(ValidationException.class) - .hasRootCauseMessage( - "File store continuous reading does not support upsert changelog mode."); - } - - @Test - public void testUnsupportedEventual() { - assertThatThrownBy( - () -> - streamSqlIter( - "SELECT * FROM T1 /*+ OPTIONS('log.consistency'='eventual') */")) - .hasCauseInstanceOf(ValidationException.class) - .hasRootCauseMessage( - "File store continuous reading does not support eventual consistency mode."); - } - @Test public void testFlinkMemoryPool() { // Check if the configuration is effective From 9961b826d6efdb56f8a44a70d5df16756a1a1161 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Mon, 29 Dec 2025 15:18:09 +0800 Subject: [PATCH 3/3] fix docs --- .../generated/core_configuration.html | 6 --- .../generated/kafka_log_configuration.html | 42 ------------------- .../ConfigOptionsDocGenerator.java | 2 - 3 files changed, 50 deletions(-) delete mode 100644 docs/layouts/shortcodes/generated/kafka_log_configuration.html diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index 23491cc468a7..f4e0a6d1cd43 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -1272,12 +1272,6 @@ Boolean Whether to read the delta from append table's overwrite commit in streaming mode. - -

streaming-read-mode
- (none) -

Enum

- The mode of streaming read that specifies to read the data of table file or log.

Possible values:
  • "log": Read from the data of table log store.
  • "file": Read from the data of table file store.
-
streaming-read-overwrite
false diff --git a/docs/layouts/shortcodes/generated/kafka_log_configuration.html b/docs/layouts/shortcodes/generated/kafka_log_configuration.html deleted file mode 100644 index 663ec9848872..000000000000 --- a/docs/layouts/shortcodes/generated/kafka_log_configuration.html +++ /dev/null @@ -1,42 +0,0 @@ -{{/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/}} - - - - - - - - - - - - - - - - - - - - - - - -
KeyDefaultTypeDescription
kafka.bootstrap.servers
(none)StringRequired Kafka server connection string.
kafka.topic
(none)StringTopic of this kafka 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 3710d14a37c9..3e7db2d89a13 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 @@ -81,8 +81,6 @@ public class ConfigOptionsDocGenerator { new OptionsClassLocation("paimon-format", "org.apache.paimon.format"), new OptionsClassLocation( "paimon-flink/paimon-flink-common", "org.apache.paimon.flink"), - new OptionsClassLocation( - "paimon-flink/paimon-flink-cdc", "org.apache.paimon.flink.kafka"), new OptionsClassLocation( "paimon-flink/paimon-flink-cdc", "org.apache.paimon.flink.pipeline.cdc"), new OptionsClassLocation(