diff --git a/docs/content/cdc-ingestion/kafka-cdc.md b/docs/content/cdc-ingestion/kafka-cdc.md index b037937c554f..26a5be340942 100644 --- a/docs/content/cdc-ingestion/kafka-cdc.md +++ b/docs/content/cdc-ingestion/kafka-cdc.md @@ -199,7 +199,9 @@ To use this feature through `flink run`, run the following shell command. --warehouse \ --database \ [--table_mapping =] \ + [--table_prefix_db ] \ [--table_prefix ] \ + [--table_suffix_db ] \ [--table_suffix ] \ [--including_tables ] \ [--excluding_tables ] \ diff --git a/docs/content/concepts/spec/datafile.md b/docs/content/concepts/spec/datafile.md index 6ba43a421fc5..923a8da58259 100644 --- a/docs/content/concepts/spec/datafile.md +++ b/docs/content/concepts/spec/datafile.md @@ -83,11 +83,45 @@ relationship between various table types and buckets in Paimon: The name of data file is `data-${uuid}-${id}.${format}`. For the append table, the file stores the data of the table without adding any new columns. But for the primary key table, each row of data stores additional system columns: -1. `_VALUE_KIND`: row is deleted or added. Similar to RocksDB, each row of data can be deleted or added, which will be +## Table with Primary key Data File + +1. Primary key columns, `_KEY_` prefix to key columns, this is to avoid conflicts with columns of the table. It's optional, + Paimon version 1.0 and above will retrieve the primary key fields from value_columns. +2. `_VALUE_KIND`: TINYINT, row is deleted or added. Similar to RocksDB, each row of data can be deleted or added, which will be used for updating the primary key table. -2. `_SEQUENCE_NUMBER`: this number is used for comparison during updates, determining which data came first and which +3. `_SEQUENCE_NUMBER`: BIGINT, this number is used for comparison during updates, determining which data came first and which data came later. -3. `_KEY_` prefix to key columns, this is to avoid conflicts with columns of the table. +4. Value columns. All columns declared in the table. + +For example, data file for table: + +```sql +CREATE TABLE T ( + a INT PRIMARY KEY NOT ENFORCED, + b INT, + c INT +); +``` + +Its file has 6 columns: `_KEY_a`, `_VALUE_KIND`, `_SEQUENCE_NUMBER`, `a`, `b`, `c`. + +When `data-file.thin-mode` enabled, its file has 5 columns: `_VALUE_KIND`, `_SEQUENCE_NUMBER`, `a`, `b`, `c`. + +## Table w/o Primary key Data File + +- Value columns. All columns declared in the table. + +For example, data file for table: + +```sql +CREATE TABLE T ( + a INT, + b INT, + c INT +); +``` + +Its file has 3 columns: `a`, `b`, `c`. ## Changelog File diff --git a/docs/content/concepts/spec/manifest.md b/docs/content/concepts/spec/manifest.md index bc7318331a51..9cc5afca0f58 100644 --- a/docs/content/concepts/spec/manifest.md +++ b/docs/content/concepts/spec/manifest.md @@ -35,13 +35,13 @@ under the License. Manifest List includes meta of several manifest files. Its name contains UUID, it is a avro file, the schema is: -1. fileName: manifest file name. -2. fileSize: manifest file size. -3. numAddedFiles: number added files in manifest. -4. numDeletedFiles: number deleted files in manifest. -5. partitionStats: partition stats, the minimum and maximum values of partition fields in this manifest are beneficial +1. _FILE_NAME: STRING, manifest file name. +2. _FILE_SIZE: BIGINT, manifest file size. +3. _NUM_ADDED_FILES: BIGINT, number added files in manifest. +4. _NUM_DELETED_FILES: BIGINT, number deleted files in manifest. +5. _PARTITION_STATS: SimpleStats, partition stats, the minimum and maximum values of partition fields in this manifest are beneficial for skipping certain manifest files during queries, it is a SimpleStats. -6. schemaId: schema id when writing this manifest file. +6. _SCHEMA_ID: BIGINT, schema id when writing this manifest file. ## Manifest @@ -63,31 +63,31 @@ Data Manifest includes meta of several data files or changelog files. The schema is: -1. kind: ADD or DELETE, -2. partition: partition spec, a BinaryRow. -3. bucket: bucket of this file. -4. totalBuckets: total buckets when write this file, it is used for verification after bucket changes. -5. file: data file meta. +1. _KIND: TINYINT, ADD or DELETE, +2. _PARTITION: BYTES, partition spec, a BinaryRow. +3. _BUCKET: INT, bucket of this file. +4. _TOTAL_BUCKETS: INT, total buckets when write this file, it is used for verification after bucket changes. +5. _FILE: data file meta. The data file meta is: -1. fileName: file name. -2. fileSize: file size. -3. rowCount: total number of rows (including add & delete) in this file. -4. minKey: the minimum key of this file. -5. maxKey: the maximum key of this file. -6. keyStats: the statistics of the key. -7. valueStats: the statistics of the value. -8. minSequenceNumber: the minimum sequence number. -9. maxSequenceNumber: the maximum sequence number. -10. schemaId: schema id when write this file. -11. level: level of this file, in LSM. -12. extraFiles: extra files for this file, for example, data file index file. -13. creationTime: creation time of this file. -14. deleteRowCount: rowCount = addRowCount + deleteRowCount. -15. embeddedIndex: if data file index is too small, store the index in manifest. -16. fileSource: indicate whether this file is generated as an APPEND or COMPACT file -17. valueStatsCols: statistical column in metadata +1. _FILE_NAME: STRING, file name. +2. _FILE_SIZE: BIGINT, file size. +3. _ROW_COUNT: BIGINT, total number of rows (including add & delete) in this file. +4. _MIN_KEY: STRING, the minimum key of this file. +5. _MAX_KEY: STRING, the maximum key of this file. +6. _KEY_STATS: SimpleStats, the statistics of the key. +7. _VALUE_STATS: SimpleStats, the statistics of the value. +8. _MIN_SEQUENCE_NUMBER: BIGINT, the minimum sequence number. +9. _MAX_SEQUENCE_NUMBER: BIGINT, the maximum sequence number. +10. _SCHEMA_ID: BIGINT, schema id when write this file. +11. _LEVEL: INT, level of this file, in LSM. +12. _EXTRA_FILES: ARRAY, extra files for this file, for example, data file index file. +13. _CREATION_TIME: TIMESTAMP_MILLIS, creation time of this file. +14. _DELETE_ROW_COUNT: BIGINT, rowCount = addRowCount + deleteRowCount. +15. _EMBEDDED_FILE_INDEX: BYTES, if data file index is too small, store the index in manifest. +16. _FILE_SOURCE: TINYINT, indicate whether this file is generated as an APPEND or COMPACT file +17. _VALUE_STATS_COLS: ARRAY, statistical column in metadata ### Index Manifest @@ -100,16 +100,35 @@ Index Manifest includes meta of several [table-index]({{< ref "concepts/spec/tab The schema is: -1. kind: ADD or DELETE, -2. partition: partition spec, a BinaryRow. -3. bucket: bucket of this file. -4. indexFile: index file meta. +1. _KIND: TINYINT, ADD or DELETE, +2. _PARTITION: BYTES, partition spec, a BinaryRow. +3. _BUCKET: INT, bucket of this file. +4. _INDEX_TYPE: STRING, "HASH" or "DELETION_VECTORS". +5. _FILE_NAME: STRING, file name. +6. _FILE_SIZE: BIGINT, file size. +7. _ROW_COUNT: BIGINT, total number of rows. +8. _DELETIONS_VECTORS_RANGES: Metadata only used by "DELETION_VECTORS", is an array of deletion vector meta, the schema of each deletion vector meta is: + 1. f0: the data file name corresponding to this deletion vector. + 2. f1: the starting offset of this deletion vector in the index file. + 3. f2: the length of this deletion vector in the index file. + 4. _CARDINALITY: the number of deleted rows. -The index file meta is: +## Appendix -1. indexType: string, "HASH" or "DELETION_VECTORS". -2. fileName: file name. -3. fileSize: file size. -4. rowCount: total number of rows. -5. deletionVectorsRanges: Metadata only used by "DELETION_VECTORS", Stores offset and length of each data file, - The schema is `ARRAY>`. +### SimpleStats + +SimpleStats is nested row, the schema is: + +1. _MIN_VALUES: BYTES, BinaryRow, the minimum values of the columns. +2. _MAX_VALUES: BYTES, BinaryRow, the maximum values of the columns. +3. _NULL_COUNTS: ARRAY, the number of nulls of the columns. + +### BinaryRow + +BinaryRow is backed by bytes instead of Object. It can significantly reduce the serialization/deserialization of Java +objects. + +A Row has two part: Fixed-length part and variable-length part. Fixed-length part contains 1 byte header and null bit +set and field values. Null bit set is used for null tracking and is aligned to 8-byte word boundaries. `Field values` +holds fixed-length primitive types and variable-length values which can be stored in 8 bytes inside. If it do not fit +the variable-length field, then store the length and offset of variable-length part. diff --git a/docs/content/flink/procedures.md b/docs/content/flink/procedures.md index 7a9b23807392..8eb1786a08b3 100644 --- a/docs/content/flink/procedures.md +++ b/docs/content/flink/procedures.md @@ -434,6 +434,27 @@ All available procedures are listed below. CALL sys.rollback_to_watermark(`table` => 'default.T', watermark => 1730292023000) + + purge_files + + -- for Flink 1.18
+ -- clear table with purge files directly.
+ CALL sys.purge_files('identifier')

+ -- for Flink 1.19 and later
+ -- clear table with purge files directly.
+ CALL sys.purge_files(`table` => 'default.T')

+ + + To clear table with purge files directly. Argument: +
  • identifier: the target table identifier. Cannot be empty.
  • + + + -- for Flink 1.18
    + CALL sys.purge_files('default.T') + -- for Flink 1.19 and later
    + CALL sys.purge_files(`table` => 'default.T') + + expire_snapshots diff --git a/docs/content/maintenance/configurations.md b/docs/content/maintenance/configurations.md index 3849d70a5862..99f797e68fa6 100644 --- a/docs/content/maintenance/configurations.md +++ b/docs/content/maintenance/configurations.md @@ -38,12 +38,6 @@ Options for paimon catalog. {{< generated/catalog_configuration >}} -### FilesystemCatalogOptions - -Options for Filesystem catalog. - -{{< generated/file_system_catalog_configuration >}} - ### HiveCatalogOptions Options for Hive catalog. diff --git a/docs/content/program-api/flink-api.md b/docs/content/program-api/flink-api.md index 6ecac3909ced..3451b40d5880 100644 --- a/docs/content/program-api/flink-api.md +++ b/docs/content/program-api/flink-api.md @@ -221,7 +221,7 @@ public class WriteCdcToTable { Identifier identifier = Identifier.create("my_db", "T"); Options catalogOptions = new Options(); catalogOptions.set("warehouse", "/path/to/warehouse"); - Catalog.Loader catalogLoader = + CatalogLoader catalogLoader = () -> FlinkCatalogFactory.createPaimonCatalog(catalogOptions); Table table = catalogLoader.load().getTable(identifier); diff --git a/docs/content/spark/procedures.md b/docs/content/spark/procedures.md index 5b0efd5f90a6..bf7b8ae2d572 100644 --- a/docs/content/spark/procedures.md +++ b/docs/content/spark/procedures.md @@ -190,6 +190,16 @@ This section introduce all available spark procedures about paimon. CALL sys.rollback_to_watermark(table => 'default.T', watermark => 1730292023000)

    + + purge_files + + To clear table with purge files directly. Argument: +
  • table: the target table identifier. Cannot be empty.
  • + + + CALL sys.purge_files(table => 'default.T')

    + + migrate_database diff --git a/docs/layouts/shortcodes/generated/catalog_configuration.html b/docs/layouts/shortcodes/generated/catalog_configuration.html index 63f7adda1e0d..6355c9558653 100644 --- a/docs/layouts/shortcodes/generated/catalog_configuration.html +++ b/docs/layouts/shortcodes/generated/catalog_configuration.html @@ -26,12 +26,6 @@ - -
    allow-upper-case
    - (none) - Boolean - Indicates whether this catalog allow upper case, its default value depends on the implementation of the specific catalog. -
    cache-enabled
    true @@ -74,6 +68,12 @@ Integer Controls the max number for snapshots per table in the catalog are cached. + +
    case-sensitive
    + (none) + Boolean + Indicates whether this catalog is case-sensitive. +
    client-pool-size
    2 diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index b2bd3a976d66..1133de289fa3 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -230,6 +230,12 @@ String Specify the file name prefix of data files. + +
    data-file.thin-mode
    + false + Boolean + Enable data file thin mode to avoid duplicate columns storage. +
    delete-file.thread-num
    (none) diff --git a/docs/layouts/shortcodes/generated/file_system_catalog_configuration.html b/docs/layouts/shortcodes/generated/file_system_catalog_configuration.html deleted file mode 100644 index c416ed6da557..000000000000 --- a/docs/layouts/shortcodes/generated/file_system_catalog_configuration.html +++ /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. -*/}} - - - - - - - - - - - - - - - - - -
    KeyDefaultTypeDescription
    case-sensitive
    trueBooleanIs case sensitive. If case insensitive, you need to set this option to false, and the table name and fields be converted to lowercase.
    diff --git a/docs/layouts/shortcodes/generated/kafka_sync_database.html b/docs/layouts/shortcodes/generated/kafka_sync_database.html index 6c90f1d7f7d8..3664128a26ca 100644 --- a/docs/layouts/shortcodes/generated/kafka_sync_database.html +++ b/docs/layouts/shortcodes/generated/kafka_sync_database.html @@ -41,13 +41,21 @@
    --table_mapping
    The table name mapping between source database and Paimon. For example, if you want to synchronize a source table named "test" to a Paimon table named "paimon_test", you can specify "--table_mapping test=paimon_test". Multiple mappings could be specified with multiple "--table_mapping" options. "--table_mapping" has higher priority than "--table_prefix" and "--table_suffix". + +
    --table_prefix_db
    + The prefix of the Paimon tables to be synchronized from the specified db. For example, if you want to prefix the tables from db1 with "ods_db1_", you can specify "--table_prefix_db db1=ods_db1_". "--table_prefix_db" has higher priority than "--table_prefix". +
    --table_prefix
    - The prefix of all Paimon tables to be synchronized. For example, if you want all synchronized tables to have "ods_" as prefix, you can specify "--table_prefix ods_". + The prefix of all Paimon tables to be synchronized except those specified by "--table_mapping" or "--table_prefix_db". For example, if you want all synchronized tables to have "ods_" as prefix, you can specify "--table_prefix ods_". + + +
    --table_suffix_db
    + The suffix of the Paimon tables to be synchronized from the specified db. The usage is same as "--table_prefix_db".
    --table_suffix
    - The suffix of all Paimon tables to be synchronized. The usage is same as "--table_prefix". + The suffix of all Paimon tables to be synchronized except those specified by "--table_mapping" or "--table_suffix_db". The usage is same as "--table_prefix".
    --including_tables
    diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowBundleRecords.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowBundleRecords.java index 9627bbd85d8c..25f6603ec22e 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowBundleRecords.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowBundleRecords.java @@ -32,13 +32,13 @@ public class ArrowBundleRecords implements BundleRecords { private final VectorSchemaRoot vectorSchemaRoot; private final RowType rowType; - private final boolean allowUpperCase; + private final boolean caseSensitive; public ArrowBundleRecords( - VectorSchemaRoot vectorSchemaRoot, RowType rowType, boolean allowUpperCase) { + VectorSchemaRoot vectorSchemaRoot, RowType rowType, boolean caseSensitive) { this.vectorSchemaRoot = vectorSchemaRoot; this.rowType = rowType; - this.allowUpperCase = allowUpperCase; + this.caseSensitive = caseSensitive; } public VectorSchemaRoot getVectorSchemaRoot() { @@ -52,7 +52,7 @@ public long rowCount() { @Override public Iterator iterator() { - ArrowBatchReader arrowBatchReader = new ArrowBatchReader(rowType, allowUpperCase); + ArrowBatchReader arrowBatchReader = new ArrowBatchReader(rowType, caseSensitive); return arrowBatchReader.readBatch(vectorSchemaRoot).iterator(); } } diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java index b3925a0a769e..0f6a98b7a2e0 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java @@ -55,6 +55,8 @@ import java.util.List; import java.util.stream.Collectors; +import static org.apache.paimon.utils.StringUtils.toLowerCaseIfNeed; + /** Utilities for creating Arrow objects. */ public class ArrowUtils { @@ -66,13 +68,13 @@ public static VectorSchemaRoot createVectorSchemaRoot( } public static VectorSchemaRoot createVectorSchemaRoot( - RowType rowType, BufferAllocator allocator, boolean allowUpperCase) { + RowType rowType, BufferAllocator allocator, boolean caseSensitive) { List fields = rowType.getFields().stream() .map( f -> toArrowField( - allowUpperCase ? f.name() : f.name().toLowerCase(), + toLowerCaseIfNeed(f.name(), caseSensitive), f.id(), f.type(), 0)) @@ -81,9 +83,9 @@ public static VectorSchemaRoot createVectorSchemaRoot( } public static FieldVector createVector( - DataField dataField, BufferAllocator allocator, boolean allowUpperCase) { + DataField dataField, BufferAllocator allocator, boolean caseSensitive) { return toArrowField( - allowUpperCase ? dataField.name() : dataField.name().toLowerCase(), + toLowerCaseIfNeed(dataField.name(), caseSensitive), dataField.id(), dataField.type(), 0) diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/reader/ArrowBatchReader.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/reader/ArrowBatchReader.java index 9d20062b437b..b626758dedfc 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/reader/ArrowBatchReader.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/reader/ArrowBatchReader.java @@ -34,6 +34,8 @@ import java.util.Iterator; import java.util.List; +import static org.apache.paimon.utils.StringUtils.toLowerCaseIfNeed; + /** Reader from a {@link VectorSchemaRoot} to paimon rows. */ public class ArrowBatchReader { @@ -41,9 +43,9 @@ public class ArrowBatchReader { private final VectorizedColumnBatch batch; private final Arrow2PaimonVectorConverter[] convertors; private final RowType projectedRowType; - private final boolean allowUpperCase; + private final boolean caseSensitive; - public ArrowBatchReader(RowType rowType, boolean allowUpperCase) { + public ArrowBatchReader(RowType rowType, boolean caseSensitive) { this.internalRowSerializer = new InternalRowSerializer(rowType); ColumnVector[] columnVectors = new ColumnVector[rowType.getFieldCount()]; this.convertors = new Arrow2PaimonVectorConverter[rowType.getFieldCount()]; @@ -53,7 +55,7 @@ public ArrowBatchReader(RowType rowType, boolean allowUpperCase) { for (int i = 0; i < columnVectors.length; i++) { this.convertors[i] = Arrow2PaimonVectorConverter.construct(rowType.getTypeAt(i)); } - this.allowUpperCase = allowUpperCase; + this.caseSensitive = caseSensitive; } public Iterable readBatch(VectorSchemaRoot vsr) { @@ -63,8 +65,7 @@ public Iterable readBatch(VectorSchemaRoot vsr) { for (int i = 0; i < dataFields.size(); ++i) { try { String fieldName = dataFields.get(i).name(); - Field field = - arrowSchema.findField(allowUpperCase ? fieldName : fieldName.toLowerCase()); + Field field = arrowSchema.findField(toLowerCaseIfNeed(fieldName, caseSensitive)); int idx = arrowSchema.getFields().indexOf(field); mapping[i] = idx; } catch (IllegalArgumentException e) { diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatCWriter.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatCWriter.java index 10afcaf6917a..442457813ace 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatCWriter.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatCWriter.java @@ -37,8 +37,8 @@ public class ArrowFormatCWriter implements AutoCloseable { private final ArrowSchema schema; private final ArrowFormatWriter realWriter; - public ArrowFormatCWriter(RowType rowType, int writeBatchSize, boolean allowUpperCase) { - this.realWriter = new ArrowFormatWriter(rowType, writeBatchSize, allowUpperCase); + public ArrowFormatCWriter(RowType rowType, int writeBatchSize, boolean caseSensitive) { + this.realWriter = new ArrowFormatWriter(rowType, writeBatchSize, caseSensitive); RootAllocator allocator = realWriter.getAllocator(); array = ArrowArray.allocateNew(allocator); schema = ArrowSchema.allocateNew(allocator); diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatWriter.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatWriter.java index acdb5d0dcb1d..9f557921979b 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatWriter.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatWriter.java @@ -43,10 +43,10 @@ public class ArrowFormatWriter implements AutoCloseable { private final RootAllocator allocator; private int rowId; - public ArrowFormatWriter(RowType rowType, int writeBatchSize, boolean allowUpperCase) { + public ArrowFormatWriter(RowType rowType, int writeBatchSize, boolean caseSensitive) { allocator = new RootAllocator(); - vectorSchemaRoot = ArrowUtils.createVectorSchemaRoot(rowType, allocator, allowUpperCase); + vectorSchemaRoot = ArrowUtils.createVectorSchemaRoot(rowType, allocator, caseSensitive); fieldWriters = new ArrowFieldWriter[rowType.getFieldCount()]; diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java index 5db809cff1d1..dd5632c18b42 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -1452,6 +1452,13 @@ public class CoreOptions implements Serializable { "For DELETE manifest entry in manifest file, drop stats to reduce memory and storage." + " Default value is false only for compatibility of old reader."); + public static final ConfigOption DATA_FILE_THIN_MODE = + key("data-file.thin-mode") + .booleanType() + .defaultValue(false) + .withDescription( + "Enable data file thin mode to avoid duplicate columns storage."); + @ExcludeFromDocumentation("Only used internally to support materialized table") public static final ConfigOption MATERIALIZED_TABLE_DEFINITION_QUERY = key("materialized-table.definition-query") @@ -2356,6 +2363,10 @@ public boolean statsDenseStore() { return options.get(METADATA_STATS_DENSE_STORE); } + public boolean dataFileThinMode() { + return options.get(DATA_FILE_THIN_MODE); + } + /** Specifies the merge engine for table with primary key. */ public enum MergeEngine implements DescribedEnum { DEDUPLICATE("deduplicate", "De-duplicate and keep the last row."), diff --git a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java index bb8cfae68284..b22274e011fb 100644 --- a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java @@ -128,13 +128,12 @@ public class CatalogOptions { .withDescription( "Controls the max number for snapshots per table in the catalog are cached."); - public static final ConfigOption ALLOW_UPPER_CASE = - ConfigOptions.key("allow-upper-case") + public static final ConfigOption CASE_SENSITIVE = + ConfigOptions.key("case-sensitive") .booleanType() .noDefaultValue() - .withDescription( - "Indicates whether this catalog allow upper case, " - + "its default value depends on the implementation of the specific catalog."); + .withFallbackKeys("allow-upper-case") + .withDescription("Indicates whether this catalog is case-sensitive."); public static final ConfigOption SYNC_ALL_PROPERTIES = ConfigOptions.key("sync-all-properties") diff --git a/paimon-common/src/main/java/org/apache/paimon/types/ArrayType.java b/paimon-common/src/main/java/org/apache/paimon/types/ArrayType.java index 00cf0c072886..62fb9ce65b69 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/ArrayType.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/ArrayType.java @@ -100,6 +100,21 @@ public boolean equals(Object o) { return elementType.equals(arrayType.elementType); } + @Override + public boolean equalsIgnoreFieldId(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + ArrayType arrayType = (ArrayType) o; + return elementType.equalsIgnoreFieldId(arrayType.elementType); + } + @Override public boolean isPrunedFrom(Object o) { if (this == o) { diff --git a/paimon-common/src/main/java/org/apache/paimon/types/DataField.java b/paimon-common/src/main/java/org/apache/paimon/types/DataField.java index e9052684f33f..209118023ba5 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/DataField.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/DataField.java @@ -148,17 +148,29 @@ public boolean equals(Object o) { && Objects.equals(description, field.description); } - public boolean isPrunedFrom(DataField field) { - if (this == field) { + public boolean equalsIgnoreFieldId(DataField other) { + if (this == other) { return true; } - if (field == null) { + if (other == null) { return false; } - return Objects.equals(id, field.id) - && Objects.equals(name, field.name) - && type.isPrunedFrom(field.type) - && Objects.equals(description, field.description); + return Objects.equals(name, other.name) + && type.equalsIgnoreFieldId(other.type) + && Objects.equals(description, other.description); + } + + public boolean isPrunedFrom(DataField other) { + if (this == other) { + return true; + } + if (other == null) { + return false; + } + return Objects.equals(id, other.id) + && Objects.equals(name, other.name) + && type.isPrunedFrom(other.type) + && Objects.equals(description, other.description); } @Override diff --git a/paimon-common/src/main/java/org/apache/paimon/types/DataType.java b/paimon-common/src/main/java/org/apache/paimon/types/DataType.java index aff150090e63..dd9a4685ef5b 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/DataType.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/DataType.java @@ -145,6 +145,10 @@ public boolean equals(Object o) { return isNullable == that.isNullable && typeRoot == that.typeRoot; } + public boolean equalsIgnoreFieldId(Object o) { + return equals(o); + } + /** * Determine whether the current type is the result of the target type after pruning (e.g. * select some fields from a nested type) or just the same. diff --git a/paimon-common/src/main/java/org/apache/paimon/types/MapType.java b/paimon-common/src/main/java/org/apache/paimon/types/MapType.java index 57cb1a9724b3..b715d49284fa 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/MapType.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/MapType.java @@ -109,6 +109,22 @@ public boolean equals(Object o) { return keyType.equals(mapType.keyType) && valueType.equals(mapType.valueType); } + @Override + public boolean equalsIgnoreFieldId(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + MapType mapType = (MapType) o; + return keyType.equalsIgnoreFieldId(mapType.keyType) + && valueType.equalsIgnoreFieldId(mapType.valueType); + } + @Override public boolean isPrunedFrom(Object o) { if (this == o) { diff --git a/paimon-common/src/main/java/org/apache/paimon/types/RowType.java b/paimon-common/src/main/java/org/apache/paimon/types/RowType.java index f3fce0db6df1..625a4634b320 100644 --- a/paimon-common/src/main/java/org/apache/paimon/types/RowType.java +++ b/paimon-common/src/main/java/org/apache/paimon/types/RowType.java @@ -210,13 +210,25 @@ public boolean equals(Object o) { return false; } RowType rowType = (RowType) o; - // For nested RowTypes e.g. DataField.dataType = RowType we need to ignoreIds as they can be - // different - if (fields.size() != rowType.fields.size()) { + return fields.equals(rowType.fields); + } + + public boolean equalsIgnoreFieldId(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { return false; } - for (int i = 0; i < fields.size(); ++i) { - if (!DataField.dataFieldEqualsIgnoreId(fields.get(i), rowType.fields.get(i))) { + if (!super.equals(o)) { + return false; + } + RowType other = (RowType) o; + if (fields.size() != other.fields.size()) { + return false; + } + for (int i = 0; i < fields.size(); i++) { + if (!fields.get(i).equalsIgnoreFieldId(other.fields.get(i))) { return false; } } @@ -236,7 +248,7 @@ public boolean isPrunedFrom(Object o) { } RowType rowType = (RowType) o; for (DataField field : fields) { - if (!field.isPrunedFrom(rowType.getField(field.name()))) { + if (!field.isPrunedFrom(rowType.getField(field.id()))) { return false; } } diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/StringUtils.java b/paimon-common/src/main/java/org/apache/paimon/utils/StringUtils.java index e184624c0bbf..c4e07e0a6972 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/StringUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/StringUtils.java @@ -542,8 +542,8 @@ public static String quote(String str) { return "`" + str + "`"; } - public static String caseSensitiveConversion(String str, boolean allowUpperCase) { - return allowUpperCase ? str : str.toLowerCase(); + public static String toLowerCaseIfNeed(String str, boolean caseSensitive) { + return caseSensitive ? str : str.toLowerCase(); } public static boolean isNumeric(final CharSequence cs) { diff --git a/paimon-core/src/main/java/org/apache/paimon/KeyValueThinSerializer.java b/paimon-core/src/main/java/org/apache/paimon/KeyValueThinSerializer.java new file mode 100644 index 000000000000..6dd41a42506a --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/KeyValueThinSerializer.java @@ -0,0 +1,58 @@ +/* + * 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; + +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.JoinedRow; +import org.apache.paimon.types.RowKind; +import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.ObjectSerializer; + +/** Serialize KeyValue to InternalRow with ignorance of key. Only used to write KeyValue to disk. */ +public class KeyValueThinSerializer extends ObjectSerializer { + + private static final long serialVersionUID = 1L; + + private final GenericRow reusedMeta; + private final JoinedRow reusedKeyWithMeta; + + public KeyValueThinSerializer(RowType keyType, RowType valueType) { + super(KeyValue.schema(keyType, valueType)); + + this.reusedMeta = new GenericRow(2); + this.reusedKeyWithMeta = new JoinedRow(); + } + + public InternalRow toRow(KeyValue record) { + return toRow(record.sequenceNumber(), record.valueKind(), record.value()); + } + + public InternalRow toRow(long sequenceNumber, RowKind valueKind, InternalRow value) { + reusedMeta.setField(0, sequenceNumber); + reusedMeta.setField(1, valueKind.toByteValue()); + return reusedKeyWithMeta.replace(reusedMeta, value); + } + + @Override + public KeyValue fromRow(InternalRow row) { + throw new UnsupportedOperationException( + "KeyValue cannot be deserialized from InternalRow by this serializer."); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index c76d0286b595..892e77735b4b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -60,7 +60,6 @@ import static org.apache.paimon.CoreOptions.TYPE; import static org.apache.paimon.CoreOptions.createCommitUser; -import static org.apache.paimon.options.CatalogOptions.ALLOW_UPPER_CASE; import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED; import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE; import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH; @@ -82,7 +81,7 @@ protected AbstractCatalog(FileIO fileIO) { protected AbstractCatalog(FileIO fileIO, Options options) { this.fileIO = fileIO; - this.tableDefaultOptions = Catalog.tableDefaultOptions(options.toMap()); + this.tableDefaultOptions = CatalogUtils.tableDefaultOptions(options.toMap()); this.catalogOptions = options; } @@ -123,11 +122,6 @@ protected boolean lockEnabled() { return catalogOptions.getOptional(LOCK_ENABLED).orElse(fileIO.isObjectStore()); } - @Override - public boolean allowUpperCase() { - return catalogOptions.getOptional(ALLOW_UPPER_CASE).orElse(true); - } - protected boolean allowCustomTablePath() { return false; } @@ -516,7 +510,6 @@ public Optional metastoreClientFactory( return Optional.empty(); } - @Override public Path getTableLocation(Identifier identifier) { return new Path(newDatabasePath(identifier.getDatabaseName()), identifier.getTableName()); } @@ -580,8 +573,9 @@ protected void checkNotSystemDatabase(String database) { } protected void validateIdentifierNameCaseInsensitive(Identifier identifier) { - Catalog.validateCaseInsensitive(allowUpperCase(), "Database", identifier.getDatabaseName()); - Catalog.validateCaseInsensitive(allowUpperCase(), "Table", identifier.getObjectName()); + CatalogUtils.validateCaseInsensitive( + caseSensitive(), "Database", identifier.getDatabaseName()); + CatalogUtils.validateCaseInsensitive(caseSensitive(), "Table", identifier.getObjectName()); } private void validateFieldNameCaseInsensitiveInSchemaChange(List changes) { @@ -599,7 +593,7 @@ private void validateFieldNameCaseInsensitiveInSchemaChange(List c } protected void validateFieldNameCaseInsensitive(List fieldNames) { - Catalog.validateCaseInsensitive(allowUpperCase(), "Field", fieldNames); + CatalogUtils.validateCaseInsensitive(caseSensitive(), "Field", fieldNames); } private void validateAutoCreateClose(Map options) { diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java index a90fb86ac2d3..37ea6fa5e203 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java @@ -20,22 +20,15 @@ import org.apache.paimon.annotation.Public; import org.apache.paimon.fs.FileIO; -import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.table.Table; import org.apache.paimon.view.View; -import java.io.Serializable; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; - -import static org.apache.paimon.options.OptionsUtils.convertToPropertiesPrefixKey; -import static org.apache.paimon.utils.Preconditions.checkArgument; /** * This interface is responsible for reading and writing metadata such as database/table from a @@ -47,30 +40,38 @@ @Public public interface Catalog extends AutoCloseable { - String DEFAULT_DATABASE = "default"; - + // constants for system table and database String SYSTEM_TABLE_SPLITTER = "$"; String SYSTEM_DATABASE_NAME = "sys"; String SYSTEM_BRANCH_PREFIX = "branch_"; - String TABLE_DEFAULT_OPTION_PREFIX = "table-default."; - String DB_SUFFIX = ".db"; + // constants for table and database String COMMENT_PROP = "comment"; String OWNER_PROP = "owner"; + + // constants for database + String DEFAULT_DATABASE = "default"; + String DB_SUFFIX = ".db"; String DB_LOCATION_PROP = "location"; + + // constants for table + String TABLE_DEFAULT_OPTION_PREFIX = "table-default."; String NUM_ROWS_PROP = "numRows"; String NUM_FILES_PROP = "numFiles"; String TOTAL_SIZE_PROP = "totalSize"; String LAST_UPDATE_TIME_PROP = "lastUpdateTime"; - String HIVE_LAST_UPDATE_TIME_PROP = "transient_lastDdlTime"; - /** Warehouse root path containing all database directories in this catalog. */ + /** Warehouse root path for creating new databases. */ String warehouse(); - /** Catalog options. */ + /** {@link FileIO} of this catalog. It can access {@link #warehouse()} path. */ + FileIO fileIO(); + + /** Catalog options for re-creating this catalog. */ Map options(); - FileIO fileIO(); + /** Return a boolean that indicates whether this catalog is case-sensitive. */ + boolean caseSensitive(); /** * Get the names of all databases in this catalog. @@ -149,14 +150,6 @@ void alterDatabase(String name, List changes, boolean ignoreIfNo */ Table getTable(Identifier identifier) throws TableNotExistException; - /** - * Get the table location in this catalog. If the table exists, return the location of the - * table; If the table does not exist, construct the location for table. - * - * @return the table location - */ - Path getTableLocation(Identifier identifier); - /** * Get names of all tables under this database. An empty list is returned if none exists. * @@ -347,44 +340,30 @@ default void renameView(Identifier fromView, Identifier toView, boolean ignoreIf throw new UnsupportedOperationException(); } - /** Return a boolean that indicates whether this catalog allow upper case. */ - boolean allowUpperCase(); - + /** + * Repair the entire Catalog, repair the metadata in the metastore consistent with the metadata + * in the filesystem, register missing tables in the metastore. + */ default void repairCatalog() { throw new UnsupportedOperationException(); } + /** + * Repair the entire database, repair the metadata in the metastore consistent with the metadata + * in the filesystem, register missing tables in the metastore. + */ default void repairDatabase(String databaseName) { throw new UnsupportedOperationException(); } + /** + * Repair the table, repair the metadata in the metastore consistent with the metadata in the + * filesystem. + */ default void repairTable(Identifier identifier) throws TableNotExistException { throw new UnsupportedOperationException(); } - static Map tableDefaultOptions(Map options) { - return convertToPropertiesPrefixKey(options, TABLE_DEFAULT_OPTION_PREFIX); - } - - /** Validate database, table and field names must be lowercase when not case-sensitive. */ - static void validateCaseInsensitive(boolean caseSensitive, String type, String... names) { - validateCaseInsensitive(caseSensitive, type, Arrays.asList(names)); - } - - /** Validate database, table and field names must be lowercase when not case-sensitive. */ - static void validateCaseInsensitive(boolean caseSensitive, String type, List names) { - if (caseSensitive) { - return; - } - List illegalNames = - names.stream().filter(f -> !f.equals(f.toLowerCase())).collect(Collectors.toList()); - checkArgument( - illegalNames.isEmpty(), - String.format( - "%s name %s cannot contain upper case in the catalog.", - type, illegalNames)); - } - /** Exception for trying to drop on a database that is not empty. */ class DatabaseNotEmptyException extends Exception { private static final String MSG = "Database %s is not empty."; @@ -621,10 +600,4 @@ public Identifier identifier() { return identifier; } } - - /** Loader of {@link Catalog}. */ - @FunctionalInterface - interface Loader extends Serializable { - Catalog load(); - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLoader.java similarity index 55% rename from paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalogOptions.java rename to paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLoader.java index e656742b42e9..c8de08139cb7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalogOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLoader.java @@ -18,19 +18,11 @@ package org.apache.paimon.catalog; -import org.apache.paimon.options.ConfigOption; -import org.apache.paimon.options.ConfigOptions; +import java.io.Serializable; -/** Options for filesystem catalog. */ -public final class FileSystemCatalogOptions { +/** Loader for creating a {@link Catalog}. */ +@FunctionalInterface +public interface CatalogLoader extends Serializable { - public static final ConfigOption CASE_SENSITIVE = - ConfigOptions.key("case-sensitive") - .booleanType() - .defaultValue(true) - .withFallbackKeys("allow-upper-case") - .withDescription( - "Is case sensitive. If case insensitive, you need to set this option to false, and the table name and fields be converted to lowercase."); - - private FileSystemCatalogOptions() {} + Catalog load(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java index 39f81833a9eb..bae23c627607 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java @@ -21,6 +21,15 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.schema.SchemaManager; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.paimon.catalog.Catalog.TABLE_DEFAULT_OPTION_PREFIX; +import static org.apache.paimon.options.OptionsUtils.convertToPropertiesPrefixKey; +import static org.apache.paimon.utils.Preconditions.checkArgument; + /** Utils for {@link Catalog}. */ public class CatalogUtils { @@ -51,4 +60,29 @@ public static String table(Path path) { public static String table(String path) { return SchemaManager.identifierFromPath(path, false).getObjectName(); } + + public static Map tableDefaultOptions(Map options) { + return convertToPropertiesPrefixKey(options, TABLE_DEFAULT_OPTION_PREFIX); + } + + /** Validate database, table and field names must be lowercase when not case-sensitive. */ + public static void validateCaseInsensitive( + boolean caseSensitive, String type, String... names) { + validateCaseInsensitive(caseSensitive, type, Arrays.asList(names)); + } + + /** Validate database, table and field names must be lowercase when not case-sensitive. */ + public static void validateCaseInsensitive( + boolean caseSensitive, String type, List names) { + if (caseSensitive) { + return; + } + List illegalNames = + names.stream().filter(f -> !f.equals(f.toLowerCase())).collect(Collectors.toList()); + checkArgument( + illegalNames.isEmpty(), + String.format( + "%s name %s cannot contain upper case in the catalog.", + type, illegalNames)); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java index 8a80e020e8b8..968f00cfcae5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java @@ -19,7 +19,6 @@ package org.apache.paimon.catalog; import org.apache.paimon.fs.FileIO; -import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; @@ -43,8 +42,8 @@ public Catalog wrapped() { } @Override - public boolean allowUpperCase() { - return wrapped.allowUpperCase(); + public boolean caseSensitive() { + return wrapped.caseSensitive(); } @Override @@ -153,11 +152,6 @@ public void renameView(Identifier fromView, Identifier toView, boolean ignoreIfN wrapped.renameView(fromView, toView, ignoreIfNotExists); } - @Override - public Path getTableLocation(Identifier identifier) { - return wrapped.getTableLocation(identifier); - } - @Override public void createPartition(Identifier identifier, Map partitions) throws TableNotExistException { diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java index 0d1a2c4c6621..cb0c358259f8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java @@ -34,7 +34,7 @@ import java.util.Map; import java.util.concurrent.Callable; -import static org.apache.paimon.catalog.FileSystemCatalogOptions.CASE_SENSITIVE; +import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; /** A catalog implementation for {@link FileIO}. */ public class FileSystemCatalog extends AbstractCatalog { @@ -163,7 +163,7 @@ public String warehouse() { } @Override - public boolean allowUpperCase() { - return catalogOptions.get(CASE_SENSITIVE); + public boolean caseSensitive() { + return catalogOptions.getOptional(CASE_SENSITIVE).orElse(true); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/BitmapDeletionVector.java b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/BitmapDeletionVector.java index a2c592596646..51ae729c2193 100644 --- a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/BitmapDeletionVector.java +++ b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/BitmapDeletionVector.java @@ -117,4 +117,9 @@ public boolean equals(Object o) { BitmapDeletionVector that = (BitmapDeletionVector) o; return Objects.equals(this.roaringBitmap, that.roaringBitmap); } + + @Override + public int hashCode() { + return Objects.hashCode(roaringBitmap); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorIndexFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorIndexFileWriter.java index f8c8330f190c..5246d35d4b31 100644 --- a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorIndexFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorIndexFileWriter.java @@ -20,9 +20,9 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.index.DeletionVectorMeta; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.options.MemorySize; -import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.PathFactory; import org.apache.paimon.utils.Preconditions; @@ -104,13 +104,13 @@ private class SingleIndexFileWriter implements Closeable { private final Path path; private final DataOutputStream dataOutputStream; - private final LinkedHashMap> dvRanges; + private final LinkedHashMap dvMetas; private SingleIndexFileWriter() throws IOException { this.path = indexPathFactory.newPath(); this.dataOutputStream = new DataOutputStream(fileIO.newOutputStream(path, true)); dataOutputStream.writeByte(VERSION_ID_V1); - this.dvRanges = new LinkedHashMap<>(); + this.dvMetas = new LinkedHashMap<>(); } private long writtenSizeInBytes() { @@ -121,7 +121,10 @@ private void write(String key, DeletionVector deletionVector) throws IOException Preconditions.checkNotNull(dataOutputStream); byte[] data = deletionVector.serializeToBytes(); int size = data.length; - dvRanges.put(key, Pair.of(dataOutputStream.size(), size)); + dvMetas.put( + key, + new DeletionVectorMeta( + key, dataOutputStream.size(), size, deletionVector.getCardinality())); dataOutputStream.writeInt(size); dataOutputStream.write(data); dataOutputStream.writeInt(calculateChecksum(data)); @@ -132,8 +135,8 @@ public IndexFileMeta writtenIndexFile() { DELETION_VECTORS_INDEX, path.getName(), writtenSizeInBytes(), - dvRanges.size(), - dvRanges); + dvMetas.size(), + dvMetas); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorsIndexFile.java b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorsIndexFile.java index 798404e001e5..77abb2d72985 100644 --- a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorsIndexFile.java +++ b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorsIndexFile.java @@ -21,11 +21,11 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.SeekableInputStream; +import org.apache.paimon.index.DeletionVectorMeta; import org.apache.paimon.index.IndexFile; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.options.MemorySize; import org.apache.paimon.table.source.DeletionFile; -import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.PathFactory; import java.io.DataInputStream; @@ -63,9 +63,9 @@ public DeletionVectorsIndexFile( * @throws UncheckedIOException If an I/O error occurs while reading from the file. */ public Map readAllDeletionVectors(IndexFileMeta fileMeta) { - LinkedHashMap> deletionVectorRanges = - fileMeta.deletionVectorsRanges(); - checkNotNull(deletionVectorRanges); + LinkedHashMap deletionVectorMetas = + fileMeta.deletionVectorMetas(); + checkNotNull(deletionVectorMetas); String indexFileName = fileMeta.fileName(); Map deletionVectors = new HashMap<>(); @@ -73,18 +73,17 @@ public Map readAllDeletionVectors(IndexFileMeta fileMeta try (SeekableInputStream inputStream = fileIO.newInputStream(filePath)) { checkVersion(inputStream); DataInputStream dataInputStream = new DataInputStream(inputStream); - for (Map.Entry> entry : - deletionVectorRanges.entrySet()) { + for (DeletionVectorMeta deletionVectorMeta : deletionVectorMetas.values()) { deletionVectors.put( - entry.getKey(), - readDeletionVector(dataInputStream, entry.getValue().getRight())); + deletionVectorMeta.dataFileName(), + readDeletionVector(dataInputStream, deletionVectorMeta.length())); } } catch (Exception e) { throw new RuntimeException( "Unable to read deletion vectors from file: " + filePath - + ", deletionVectorRanges: " - + deletionVectorRanges, + + ", deletionVectorMetas: " + + deletionVectorMetas, e); } return deletionVectors; diff --git a/paimon-core/src/main/java/org/apache/paimon/index/DeletionVectorMeta.java b/paimon-core/src/main/java/org/apache/paimon/index/DeletionVectorMeta.java new file mode 100644 index 000000000000..9eb38818f694 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/index/DeletionVectorMeta.java @@ -0,0 +1,103 @@ +/* + * 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.index; + +import org.apache.paimon.types.BigIntType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.RowType; + +import javax.annotation.Nullable; + +import java.util.Objects; + +import static org.apache.paimon.utils.SerializationUtils.newStringType; + +/** Metadata of deletion vector. */ +public class DeletionVectorMeta { + + public static final RowType SCHEMA = + RowType.of( + new DataField(0, "f0", newStringType(false)), + new DataField(1, "f1", new IntType(false)), + new DataField(2, "f2", new IntType(false)), + new DataField(3, "_CARDINALITY", new BigIntType(true))); + + private final String dataFileName; + private final int offset; + private final int length; + @Nullable private final Long cardinality; + + public DeletionVectorMeta( + String dataFileName, int start, int size, @Nullable Long cardinality) { + this.dataFileName = dataFileName; + this.offset = start; + this.length = size; + this.cardinality = cardinality; + } + + public String dataFileName() { + return dataFileName; + } + + public int offset() { + return offset; + } + + public int length() { + return length; + } + + @Nullable + public Long cardinality() { + return cardinality; + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + DeletionVectorMeta that = (DeletionVectorMeta) o; + return offset == that.offset + && length == that.length + && Objects.equals(dataFileName, that.dataFileName) + && Objects.equals(cardinality, that.cardinality); + } + + @Override + public int hashCode() { + return Objects.hash(dataFileName, offset, length, cardinality); + } + + @Override + public String toString() { + return "DeletionVectorMeta{" + + "dataFileName='" + + dataFileName + + '\'' + + ", offset=" + + offset + + ", length=" + + length + + ", cardinality=" + + cardinality + + '}'; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileHandler.java b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileHandler.java index 7e5efccdd813..8b0e5c5021f6 100644 --- a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileHandler.java +++ b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileHandler.java @@ -100,15 +100,16 @@ public Map scanDVIndex( if (meta.indexType().equals(DELETION_VECTORS_INDEX) && file.partition().equals(partition) && file.bucket() == bucket) { - LinkedHashMap> dvRanges = - meta.deletionVectorsRanges(); - checkNotNull(dvRanges); - for (String dataFile : dvRanges.keySet()) { - Pair pair = dvRanges.get(dataFile); - DeletionFile deletionFile = + LinkedHashMap dvMetas = meta.deletionVectorMetas(); + checkNotNull(dvMetas); + for (DeletionVectorMeta dvMeta : dvMetas.values()) { + result.put( + dvMeta.dataFileName(), new DeletionFile( - filePath(meta).toString(), pair.getLeft(), pair.getRight()); - result.put(dataFile, deletionFile); + filePath(meta).toString(), + dvMeta.offset(), + dvMeta.length(), + dvMeta.cardinality())); } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMeta.java b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMeta.java index 24ba6992a5d9..aae4f8c4731b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMeta.java +++ b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMeta.java @@ -23,9 +23,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.DataField; -import org.apache.paimon.types.IntType; import org.apache.paimon.types.RowType; -import org.apache.paimon.utils.Pair; import javax.annotation.Nullable; @@ -54,12 +52,7 @@ public class IndexFileMeta { new DataField( 4, "_DELETIONS_VECTORS_RANGES", - new ArrayType( - true, - RowType.of( - newStringType(false), - new IntType(false), - new IntType(false)))))); + new ArrayType(true, DeletionVectorMeta.SCHEMA)))); private final String indexType; private final String fileName; @@ -68,9 +61,9 @@ public class IndexFileMeta { /** * Metadata only used by {@link DeletionVectorsIndexFile}, use LinkedHashMap to ensure that the - * order of DeletionVectorRanges and the written DeletionVectors is consistent. + * order of DeletionVectorMetas and the written DeletionVectors is consistent. */ - private final @Nullable LinkedHashMap> deletionVectorsRanges; + private final @Nullable LinkedHashMap deletionVectorMetas; public IndexFileMeta(String indexType, String fileName, long fileSize, long rowCount) { this(indexType, fileName, fileSize, rowCount, null); @@ -81,12 +74,12 @@ public IndexFileMeta( String fileName, long fileSize, long rowCount, - @Nullable LinkedHashMap> deletionVectorsRanges) { + @Nullable LinkedHashMap deletionVectorMetas) { this.indexType = indexType; this.fileName = fileName; this.fileSize = fileSize; this.rowCount = rowCount; - this.deletionVectorsRanges = deletionVectorsRanges; + this.deletionVectorMetas = deletionVectorMetas; } public String indexType() { @@ -105,8 +98,8 @@ public long rowCount() { return rowCount; } - public @Nullable LinkedHashMap> deletionVectorsRanges() { - return deletionVectorsRanges; + public @Nullable LinkedHashMap deletionVectorMetas() { + return deletionVectorMetas; } @Override @@ -122,12 +115,12 @@ public boolean equals(Object o) { && Objects.equals(fileName, that.fileName) && fileSize == that.fileSize && rowCount == that.rowCount - && Objects.equals(deletionVectorsRanges, that.deletionVectorsRanges); + && Objects.equals(deletionVectorMetas, that.deletionVectorMetas); } @Override public int hashCode() { - return Objects.hash(indexType, fileName, fileSize, rowCount, deletionVectorsRanges); + return Objects.hash(indexType, fileName, fileSize, rowCount, deletionVectorMetas); } @Override @@ -142,8 +135,8 @@ public String toString() { + fileSize + ", rowCount=" + rowCount - + ", deletionVectorsRanges=" - + deletionVectorsRanges + + ", deletionVectorMetas=" + + deletionVectorMetas + '}'; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMeta09Serializer.java b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMeta09Serializer.java new file mode 100644 index 000000000000..915d904569d7 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMeta09Serializer.java @@ -0,0 +1,104 @@ +/* + * 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.index; + +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.serializer.InternalRowSerializer; +import org.apache.paimon.data.serializer.InternalSerializers; +import org.apache.paimon.io.DataInputView; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.BigIntType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.RowType; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; + +import static org.apache.paimon.utils.SerializationUtils.newStringType; + +/** Serializer for {@link IndexFileMeta} with 0.9 version. */ +public class IndexFileMeta09Serializer implements Serializable { + + private static final long serialVersionUID = 1L; + + public static final RowType SCHEMA = + new RowType( + false, + Arrays.asList( + new DataField(0, "_INDEX_TYPE", newStringType(false)), + new DataField(1, "_FILE_NAME", newStringType(false)), + new DataField(2, "_FILE_SIZE", new BigIntType(false)), + new DataField(3, "_ROW_COUNT", new BigIntType(false)), + new DataField( + 4, + "_DELETIONS_VECTORS_RANGES", + new ArrayType( + true, + RowType.of( + newStringType(false), + new IntType(false), + new IntType(false)))))); + + protected final InternalRowSerializer rowSerializer; + + public IndexFileMeta09Serializer() { + this.rowSerializer = InternalSerializers.create(SCHEMA); + } + + public IndexFileMeta fromRow(InternalRow row) { + return new IndexFileMeta( + row.getString(0).toString(), + row.getString(1).toString(), + row.getLong(2), + row.getLong(3), + row.isNullAt(4) ? null : rowArrayDataToDvMetas(row.getArray(4))); + } + + public final List deserializeList(DataInputView source) throws IOException { + int size = source.readInt(); + List records = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + records.add(deserialize(source)); + } + return records; + } + + public IndexFileMeta deserialize(DataInputView in) throws IOException { + return fromRow(rowSerializer.deserialize(in)); + } + + public static LinkedHashMap rowArrayDataToDvMetas( + InternalArray arrayData) { + LinkedHashMap dvMetas = new LinkedHashMap<>(arrayData.size()); + for (int i = 0; i < arrayData.size(); i++) { + InternalRow row = arrayData.getRow(i, 3); + dvMetas.put( + row.getString(0).toString(), + new DeletionVectorMeta( + row.getString(0).toString(), row.getInt(1), row.getInt(2), null)); + } + return dvMetas; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMetaSerializer.java b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMetaSerializer.java index 4b52932623f2..db4a44838fbf 100644 --- a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMetaSerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMetaSerializer.java @@ -24,9 +24,9 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalRow; import org.apache.paimon.utils.ObjectSerializer; -import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.VersionedObjectSerializer; +import java.util.Collection; import java.util.LinkedHashMap; /** A {@link VersionedObjectSerializer} for {@link IndexFileMeta}. */ @@ -43,9 +43,9 @@ public InternalRow toRow(IndexFileMeta record) { BinaryString.fromString(record.fileName()), record.fileSize(), record.rowCount(), - record.deletionVectorsRanges() == null + record.deletionVectorMetas() == null ? null - : dvRangesToRowArrayData(record.deletionVectorsRanges())); + : dvMetasToRowArrayData(record.deletionVectorMetas().values())); } @Override @@ -55,30 +55,35 @@ public IndexFileMeta fromRow(InternalRow row) { row.getString(1).toString(), row.getLong(2), row.getLong(3), - row.isNullAt(4) ? null : rowArrayDataToDvRanges(row.getArray(4))); + row.isNullAt(4) ? null : rowArrayDataToDvMetas(row.getArray(4))); } - public static InternalArray dvRangesToRowArrayData( - LinkedHashMap> dvRanges) { + public static InternalArray dvMetasToRowArrayData(Collection dvMetas) { return new GenericArray( - dvRanges.entrySet().stream() + dvMetas.stream() .map( - entry -> + dvMeta -> GenericRow.of( - BinaryString.fromString(entry.getKey()), - entry.getValue().getLeft(), - entry.getValue().getRight())) + BinaryString.fromString(dvMeta.dataFileName()), + dvMeta.offset(), + dvMeta.length(), + dvMeta.cardinality())) .toArray(GenericRow[]::new)); } - public static LinkedHashMap> rowArrayDataToDvRanges( + public static LinkedHashMap rowArrayDataToDvMetas( InternalArray arrayData) { - LinkedHashMap> dvRanges = - new LinkedHashMap<>(arrayData.size()); + LinkedHashMap dvMetas = new LinkedHashMap<>(arrayData.size()); for (int i = 0; i < arrayData.size(); i++) { - InternalRow row = arrayData.getRow(i, 3); - dvRanges.put(row.getString(0).toString(), Pair.of(row.getInt(1), row.getInt(2))); + InternalRow row = arrayData.getRow(i, DeletionVectorMeta.SCHEMA.getFieldCount()); + dvMetas.put( + row.getString(0).toString(), + new DeletionVectorMeta( + row.getString(0).toString(), + row.getInt(1), + row.getInt(2), + row.isNullAt(3) ? null : row.getLong(3))); } - return dvRanges; + return dvMetas; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java index ce0b3b02840b..651c6a6f7b56 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java @@ -42,7 +42,6 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.function.Function; @@ -56,13 +55,13 @@ *

    NOTE: records given to the writer must be sorted because it does not compare the min max keys * to produce {@link DataFileMeta}. */ -public class KeyValueDataFileWriter +public abstract class KeyValueDataFileWriter extends StatsCollectingSingleFileWriter { private static final Logger LOG = LoggerFactory.getLogger(KeyValueDataFileWriter.class); - private final RowType keyType; - private final RowType valueType; + protected final RowType keyType; + protected final RowType valueType; private final long schemaId; private final int level; @@ -85,6 +84,7 @@ public KeyValueDataFileWriter( Function converter, RowType keyType, RowType valueType, + RowType writeRowType, @Nullable SimpleStatsExtractor simpleStatsExtractor, long schemaId, int level, @@ -97,11 +97,11 @@ public KeyValueDataFileWriter( factory, path, converter, - KeyValue.schema(keyType, valueType), + writeRowType, simpleStatsExtractor, compression, StatsCollectorFactories.createStatsFactories( - options, KeyValue.schema(keyType, valueType).getFieldNames()), + options, writeRowType.getFieldNames(), keyType.getFieldNames()), options.asyncFileWrite()); this.keyType = keyType; @@ -166,17 +166,11 @@ public DataFileMeta result() throws IOException { return null; } - SimpleColStats[] rowStats = fieldStats(); - int numKeyFields = keyType.getFieldCount(); - - SimpleColStats[] keyFieldStats = Arrays.copyOfRange(rowStats, 0, numKeyFields); - SimpleStats keyStats = keyStatsConverter.toBinaryAllMode(keyFieldStats); - - SimpleColStats[] valFieldStats = - Arrays.copyOfRange(rowStats, numKeyFields + 2, rowStats.length); + Pair keyValueStats = fetchKeyValueStats(fieldStats()); + SimpleStats keyStats = keyStatsConverter.toBinaryAllMode(keyValueStats.getKey()); Pair, SimpleStats> valueStatsPair = - valueStatsConverter.toBinary(valFieldStats); + valueStatsConverter.toBinary(keyValueStats.getValue()); DataFileIndexWriter.FileIndexResult indexResult = dataFileIndexWriter == null @@ -204,6 +198,8 @@ public DataFileMeta result() throws IOException { valueStatsPair.getKey()); } + abstract Pair fetchKeyValueStats(SimpleColStats[] rowStats); + @Override public void close() throws IOException { if (dataFileIndexWriter != null) { diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriterImpl.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriterImpl.java new file mode 100644 index 000000000000..27a1aef64e36 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriterImpl.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.io; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.KeyValue; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.fileindex.FileIndexOptions; +import org.apache.paimon.format.FormatWriterFactory; +import org.apache.paimon.format.SimpleColStats; +import org.apache.paimon.format.SimpleStatsExtractor; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.manifest.FileSource; +import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.Pair; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.function.Function; + +/** Write data files containing {@link KeyValue}s. */ +public class KeyValueDataFileWriterImpl extends KeyValueDataFileWriter { + + public KeyValueDataFileWriterImpl( + FileIO fileIO, + FormatWriterFactory factory, + Path path, + Function converter, + RowType keyType, + RowType valueType, + @Nullable SimpleStatsExtractor simpleStatsExtractor, + long schemaId, + int level, + String compression, + CoreOptions options, + FileSource fileSource, + FileIndexOptions fileIndexOptions) { + super( + fileIO, + factory, + path, + converter, + keyType, + valueType, + KeyValue.schema(keyType, valueType), + simpleStatsExtractor, + schemaId, + level, + compression, + options, + fileSource, + fileIndexOptions); + } + + @Override + Pair fetchKeyValueStats(SimpleColStats[] rowStats) { + int numKeyFields = keyType.getFieldCount(); + return Pair.of( + Arrays.copyOfRange(rowStats, 0, numKeyFields), + Arrays.copyOfRange(rowStats, numKeyFields + 2, rowStats.length)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileWriterFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileWriterFactory.java index a6fddb43283a..a6aae3985bd4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileWriterFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileWriterFactory.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.KeyValue; import org.apache.paimon.KeyValueSerializer; +import org.apache.paimon.KeyValueThinSerializer; import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.fileindex.FileIndexOptions; @@ -31,6 +32,8 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.FileSource; import org.apache.paimon.statistics.SimpleColStatsCollector; +import org.apache.paimon.table.SpecialFields; +import org.apache.paimon.types.DataField; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.FileStorePathFactory; import org.apache.paimon.utils.StatsCollectorFactories; @@ -38,10 +41,13 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.function.Function; +import java.util.stream.Collectors; /** A factory to create {@link FileWriter}s for writing {@link KeyValue} files. */ public class KeyValueFileWriterFactory { @@ -58,15 +64,13 @@ public class KeyValueFileWriterFactory { private KeyValueFileWriterFactory( FileIO fileIO, long schemaId, - RowType keyType, - RowType valueType, WriteFormatContext formatContext, long suggestedFileSize, CoreOptions options) { this.fileIO = fileIO; this.schemaId = schemaId; - this.keyType = keyType; - this.valueType = valueType; + this.keyType = formatContext.keyType; + this.valueType = formatContext.valueType; this.formatContext = formatContext; this.suggestedFileSize = suggestedFileSize; this.options = options; @@ -107,21 +111,35 @@ public RollingFileWriter createRollingChangelogFileWrite private KeyValueDataFileWriter createDataFileWriter( Path path, int level, FileSource fileSource) { - KeyValueSerializer kvSerializer = new KeyValueSerializer(keyType, valueType); - return new KeyValueDataFileWriter( - fileIO, - formatContext.writerFactory(level), - path, - kvSerializer::toRow, - keyType, - valueType, - formatContext.extractor(level), - schemaId, - level, - formatContext.compression(level), - options, - fileSource, - fileIndexOptions); + return formatContext.thinModeEnabled() + ? new KeyValueThinDataFileWriterImpl( + fileIO, + formatContext.writerFactory(level), + path, + new KeyValueThinSerializer(keyType, valueType)::toRow, + keyType, + valueType, + formatContext.extractor(level), + schemaId, + level, + formatContext.compression(level), + options, + fileSource, + fileIndexOptions) + : new KeyValueDataFileWriterImpl( + fileIO, + formatContext.writerFactory(level), + path, + new KeyValueSerializer(keyType, valueType)::toRow, + keyType, + valueType, + formatContext.extractor(level), + schemaId, + level, + formatContext.compression(level), + options, + fileSource, + fileIndexOptions); } public void deleteFile(String filename, int level) { @@ -191,17 +209,17 @@ private Builder( public KeyValueFileWriterFactory build( BinaryRow partition, int bucket, CoreOptions options) { - RowType fileRowType = KeyValue.schema(keyType, valueType); WriteFormatContext context = new WriteFormatContext( partition, bucket, - fileRowType, + keyType, + valueType, fileFormat, format2PathFactory, options); return new KeyValueFileWriterFactory( - fileIO, schemaId, keyType, valueType, context, suggestedFileSize, options); + fileIO, schemaId, context, suggestedFileSize, options); } } @@ -214,13 +232,24 @@ private static class WriteFormatContext { private final Map format2PathFactory; private final Map format2WriterFactory; + private final RowType keyType; + private final RowType valueType; + private final boolean thinModeEnabled; + private WriteFormatContext( BinaryRow partition, int bucket, - RowType rowType, + RowType keyType, + RowType valueType, FileFormat defaultFormat, Map parentFactories, CoreOptions options) { + this.keyType = keyType; + this.valueType = valueType; + this.thinModeEnabled = + options.dataFileThinMode() && supportsThinMode(keyType, valueType); + RowType writeRowType = + KeyValue.schema(thinModeEnabled ? RowType.of() : keyType, valueType); Map fileFormatPerLevel = options.fileFormatPerLevel(); this.level2Format = level -> @@ -236,7 +265,10 @@ private WriteFormatContext( this.format2PathFactory = new HashMap<>(); this.format2WriterFactory = new HashMap<>(); SimpleColStatsCollector.Factory[] statsCollectorFactories = - StatsCollectorFactories.createStatsFactories(options, rowType.getFieldNames()); + StatsCollectorFactories.createStatsFactories( + options, + writeRowType.getFieldNames(), + thinModeEnabled ? keyType.getFieldNames() : Collections.emptyList()); for (String format : parentFactories.keySet()) { format2PathFactory.put( format, @@ -252,11 +284,30 @@ private WriteFormatContext( format.equals("avro") ? Optional.empty() : fileFormat.createStatsExtractor( - rowType, statsCollectorFactories)); - format2WriterFactory.put(format, fileFormat.createWriterFactory(rowType)); + writeRowType, statsCollectorFactories)); + format2WriterFactory.put(format, fileFormat.createWriterFactory(writeRowType)); } } + private boolean supportsThinMode(RowType keyType, RowType valueType) { + Set keyFieldIds = + valueType.getFields().stream().map(DataField::id).collect(Collectors.toSet()); + + for (DataField field : keyType.getFields()) { + if (!SpecialFields.isKeyField(field.name())) { + return false; + } + if (!keyFieldIds.contains(field.id() - SpecialFields.KEY_FIELD_ID_START)) { + return false; + } + } + return true; + } + + private boolean thinModeEnabled() { + return thinModeEnabled; + } + @Nullable private SimpleStatsExtractor extractor(int level) { return format2Extractor.get(level2Format.apply(level)).orElse(null); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueThinDataFileWriterImpl.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueThinDataFileWriterImpl.java new file mode 100644 index 000000000000..dd7ebb006764 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueThinDataFileWriterImpl.java @@ -0,0 +1,128 @@ +/* + * 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.io; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.KeyValue; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.fileindex.FileIndexOptions; +import org.apache.paimon.format.FormatWriterFactory; +import org.apache.paimon.format.SimpleColStats; +import org.apache.paimon.format.SimpleStatsExtractor; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.manifest.FileSource; +import org.apache.paimon.table.SpecialFields; +import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.Pair; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.function.Function; + +/** + * Implementation of KeyValueDataFileWriter for thin data files. Thin data files only contain + * _SEQUENCE_NUMBER_, _ROW_KIND_ and value fields. + */ +public class KeyValueThinDataFileWriterImpl extends KeyValueDataFileWriter { + + private final int[] keyStatMapping; + + /** + * Constructs a KeyValueThinDataFileWriterImpl. + * + * @param fileIO The file IO interface. + * @param factory The format writer factory. + * @param path The path to the file. + * @param converter The function to convert KeyValue to InternalRow. + * @param keyType The row type of the key. + * @param valueType The row type of the value. + * @param simpleStatsExtractor The simple stats extractor, can be null. + * @param schemaId The schema ID. + * @param level The level. + * @param compression The compression type. + * @param options The core options. + * @param fileSource The file source. + * @param fileIndexOptions The file index options. + */ + public KeyValueThinDataFileWriterImpl( + FileIO fileIO, + FormatWriterFactory factory, + Path path, + Function converter, + RowType keyType, + RowType valueType, + @Nullable SimpleStatsExtractor simpleStatsExtractor, + long schemaId, + int level, + String compression, + CoreOptions options, + FileSource fileSource, + FileIndexOptions fileIndexOptions) { + super( + fileIO, + factory, + path, + converter, + keyType, + valueType, + KeyValue.schema(RowType.of(), valueType), + simpleStatsExtractor, + schemaId, + level, + compression, + options, + fileSource, + fileIndexOptions); + Map idToIndex = new HashMap<>(valueType.getFieldCount()); + for (int i = 0; i < valueType.getFieldCount(); i++) { + idToIndex.put(valueType.getFields().get(i).id(), i); + } + this.keyStatMapping = new int[keyType.getFieldCount()]; + for (int i = 0; i < keyType.getFieldCount(); i++) { + keyStatMapping[i] = + idToIndex.get( + keyType.getFields().get(i).id() - SpecialFields.KEY_FIELD_ID_START); + } + } + + /** + * Fetches the key and value statistics. + * + * @param rowStats The row statistics. + * @return A pair of key statistics and value statistics. + */ + @Override + Pair fetchKeyValueStats(SimpleColStats[] rowStats) { + int numKeyFields = keyType.getFieldCount(); + // In thin mode, there is no key stats in rowStats, so we only jump + // _SEQUNCE_NUMBER_ and _ROW_KIND_ stats. Therefore, the 'from' value is 2. + SimpleColStats[] valFieldStats = Arrays.copyOfRange(rowStats, 2, rowStats.length); + // Thin mode on, so need to map value stats to key stats. + SimpleColStats[] keyStats = new SimpleColStats[numKeyFields]; + for (int i = 0; i < keyStatMapping.length; i++) { + keyStats[i] = valFieldStats[keyStatMapping[i]]; + } + + return Pair.of(keyStats, valFieldStats); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java index 5543f9c3e230..63cb54c180f5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java @@ -365,7 +365,7 @@ protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotE } @Override - public boolean allowUpperCase() { + public boolean caseSensitive() { return false; } diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntry.java b/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntry.java index a52d9e8af40f..2431a1c26412 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntry.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntry.java @@ -20,6 +20,7 @@ import org.apache.paimon.annotation.Public; import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.index.DeletionVectorMeta; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; @@ -57,12 +58,7 @@ public class IndexManifestEntry { new DataField( 7, "_DELETIONS_VECTORS_RANGES", - new ArrayType( - true, - RowType.of( - newStringType(false), - new IntType(false), - new IntType(false)))))); + new ArrayType(true, DeletionVectorMeta.SCHEMA)))); private final FileKind kind; private final BinaryRow partition; diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntrySerializer.java b/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntrySerializer.java index 574e935550eb..6f2ec17dda8c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntrySerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntrySerializer.java @@ -22,10 +22,9 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.index.IndexFileMeta; +import org.apache.paimon.index.IndexFileMetaSerializer; import org.apache.paimon.utils.VersionedObjectSerializer; -import static org.apache.paimon.index.IndexFileMetaSerializer.dvRangesToRowArrayData; -import static org.apache.paimon.index.IndexFileMetaSerializer.rowArrayDataToDvRanges; import static org.apache.paimon.utils.SerializationUtils.deserializeBinaryRow; import static org.apache.paimon.utils.SerializationUtils.serializeBinaryRow; @@ -52,9 +51,10 @@ public InternalRow convertTo(IndexManifestEntry record) { BinaryString.fromString(indexFile.fileName()), indexFile.fileSize(), indexFile.rowCount(), - record.indexFile().deletionVectorsRanges() == null + record.indexFile().deletionVectorMetas() == null ? null - : dvRangesToRowArrayData(record.indexFile().deletionVectorsRanges())); + : IndexFileMetaSerializer.dvMetasToRowArrayData( + record.indexFile().deletionVectorMetas().values())); } @Override @@ -72,6 +72,8 @@ public IndexManifestEntry convertFrom(int version, InternalRow row) { row.getString(4).toString(), row.getLong(5), row.getLong(6), - row.isNullAt(7) ? null : rowArrayDataToDvRanges(row.getArray(7)))); + row.isNullAt(7) + ? null + : IndexFileMetaSerializer.rowArrayDataToDvMetas(row.getArray(7)))); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java b/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java index 60e28c59f45d..75f7af5abbdc 100644 --- a/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java @@ -55,7 +55,8 @@ default void addPartitionsSpec(List> partitionSpec default void alterPartition( LinkedHashMap partitionSpec, Map parameters, - long modifyTime) + long modifyTime, + boolean ignoreIfNotExist) throws Exception { throw new UnsupportedOperationException(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreWrite.java index 3ce019c91638..4a6196453df6 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreWrite.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreWrite.java @@ -212,6 +212,9 @@ protected void forceBufferSpill() throws Exception { if (ioManager == null) { return; } + if (forceBufferSpill) { + return; + } forceBufferSpill = true; LOG.info( "Force buffer spill for append-only file store write, writer number is: {}", diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java index c7204f484058..8b53bef8486b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -23,7 +23,6 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.catalog.PropertyChange; import org.apache.paimon.fs.FileIO; -import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; @@ -58,6 +57,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; +import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; import static org.apache.paimon.utils.ThreadPoolUtils.createScheduledThreadPool; /** A catalog implementation for REST. */ @@ -67,7 +67,7 @@ public class RESTCatalog implements Catalog { private final RESTClient client; private final ResourcePaths resourcePaths; - private final Map options; + private final Options options; private final Map baseHeader; private final AuthSession catalogAuth; @@ -105,7 +105,7 @@ public RESTCatalog(Options options) { } Map initHeaders = RESTUtil.merge(configHeaders(options.toMap()), this.catalogAuth.getHeaders()); - this.options = fetchOptionsFromServer(initHeaders, options.toMap()); + this.options = new Options(fetchOptionsFromServer(initHeaders, options.toMap())); this.resourcePaths = ResourcePaths.forCatalogProperties( this.options.get(RESTCatalogInternalOptions.PREFIX)); @@ -118,7 +118,7 @@ public String warehouse() { @Override public Map options() { - return this.options; + return this.options.toMap(); } @Override @@ -210,11 +210,6 @@ public Table getTable(Identifier identifier) throws TableNotExistException { throw new UnsupportedOperationException(); } - @Override - public Path getTableLocation(Identifier identifier) { - throw new UnsupportedOperationException(); - } - @Override public List listTables(String databaseName) throws DatabaseNotExistException { return new ArrayList(); @@ -262,8 +257,8 @@ public List listPartitions(Identifier identifier) } @Override - public boolean allowUpperCase() { - return false; + public boolean caseSensitive() { + return options.getOptional(CASE_SENSITIVE).orElse(true); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java b/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java index 33309a7cecc9..a3b30d81a3dd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java @@ -68,7 +68,6 @@ public Schema( this.partitionKeys = normalizePartitionKeys(partitionKeys); this.primaryKeys = normalizePrimaryKeys(primaryKeys); this.fields = normalizeFields(fields, this.primaryKeys, this.partitionKeys); - this.comment = comment; } diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java index 2139dca4a990..325e0c392d12 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java @@ -214,7 +214,7 @@ public TableSchema createTable(Schema schema, boolean externalTable) throws Exce if (latest.isPresent()) { TableSchema latestSchema = latest.get(); if (externalTable) { - checkSchemaForExternalTable(latestSchema, schema); + checkSchemaForExternalTable(latestSchema.toSchema(), schema); return latestSchema; } else { throw new IllegalStateException( @@ -248,14 +248,15 @@ public TableSchema createTable(Schema schema, boolean externalTable) throws Exce } } - private void checkSchemaForExternalTable(TableSchema existsSchema, Schema newSchema) { + private void checkSchemaForExternalTable(Schema existsSchema, Schema newSchema) { // When creating an external table, if the table already exists in the location, we can // choose not to specify the fields. - if (newSchema.fields().isEmpty() - // When the fields are explicitly specified, we need check for consistency. - || (Objects.equals(existsSchema.fields(), newSchema.fields()) - && Objects.equals(existsSchema.partitionKeys(), newSchema.partitionKeys()) - && Objects.equals(existsSchema.primaryKeys(), newSchema.primaryKeys()))) { + if ((newSchema.fields().isEmpty() + || newSchema.rowType().equalsIgnoreFieldId(existsSchema.rowType())) + && (newSchema.partitionKeys().isEmpty() + || Objects.equals(newSchema.partitionKeys(), existsSchema.partitionKeys())) + && (newSchema.primaryKeys().isEmpty() + || Objects.equals(newSchema.primaryKeys(), existsSchema.primaryKeys()))) { // check for options Map existsOptions = existsSchema.options(); Map newOptions = newSchema.options(); diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java index a0a149d1ae9b..791269dc73b5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java @@ -331,6 +331,10 @@ public static List newFields(RowType rowType) { return rowType.getFields(); } + public Schema toSchema() { + return new Schema(fields, partitionKeys, primaryKeys, options, comment); + } + // =================== Utils for reading ========================= public static TableSchema fromJson(String json) { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java index 7918914b2c63..9fc251c36672 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java @@ -19,6 +19,8 @@ package org.apache.paimon.table.sink; import org.apache.paimon.data.serializer.VersionedSerializer; +import org.apache.paimon.index.IndexFileMeta; +import org.apache.paimon.index.IndexFileMeta09Serializer; import org.apache.paimon.index.IndexFileMetaSerializer; import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataFileMeta; @@ -45,12 +47,14 @@ /** {@link VersionedSerializer} for {@link CommitMessage}. */ public class CommitMessageSerializer implements VersionedSerializer { - private static final int CURRENT_VERSION = 4; + private static final int CURRENT_VERSION = 5; private final DataFileMetaSerializer dataFileSerializer; private final IndexFileMetaSerializer indexEntrySerializer; + private DataFileMeta09Serializer dataFile09Serializer; private DataFileMeta08Serializer dataFile08Serializer; + private IndexFileMeta09Serializer indexEntry09Serializer; public CommitMessageSerializer() { this.dataFileSerializer = new DataFileMetaSerializer(); @@ -107,48 +111,48 @@ public List deserializeList(int version, DataInputView view) thro } private CommitMessage deserialize(int version, DataInputView view) throws IOException { - if (version >= 3) { - IOExceptionSupplier> fileDeserializer = - () -> dataFileSerializer.deserializeList(view); - if (version == 3) { - DataFileMeta09Serializer serializer = new DataFileMeta09Serializer(); - fileDeserializer = () -> serializer.deserializeList(view); - } - return new CommitMessageImpl( - deserializeBinaryRow(view), - view.readInt(), - new DataIncrement( - fileDeserializer.get(), fileDeserializer.get(), fileDeserializer.get()), - new CompactIncrement( - fileDeserializer.get(), fileDeserializer.get(), fileDeserializer.get()), - new IndexIncrement( - indexEntrySerializer.deserializeList(view), - indexEntrySerializer.deserializeList(view))); - } else { - return deserialize08(version, view); - } - } - - private CommitMessage deserialize08(int version, DataInputView view) throws IOException { - if (dataFile08Serializer == null) { - dataFile08Serializer = new DataFileMeta08Serializer(); - } + IOExceptionSupplier> fileDeserializer = fileDeserializer(version, view); + IOExceptionSupplier> indexEntryDeserializer = + indexEntryDeserializer(version, view); return new CommitMessageImpl( deserializeBinaryRow(view), view.readInt(), new DataIncrement( - dataFile08Serializer.deserializeList(view), - dataFile08Serializer.deserializeList(view), - dataFile08Serializer.deserializeList(view)), + fileDeserializer.get(), fileDeserializer.get(), fileDeserializer.get()), new CompactIncrement( - dataFile08Serializer.deserializeList(view), - dataFile08Serializer.deserializeList(view), - dataFile08Serializer.deserializeList(view)), + fileDeserializer.get(), fileDeserializer.get(), fileDeserializer.get()), new IndexIncrement( - indexEntrySerializer.deserializeList(view), - version <= 2 - ? Collections.emptyList() - : indexEntrySerializer.deserializeList(view))); + indexEntryDeserializer.get(), + version <= 2 ? Collections.emptyList() : indexEntryDeserializer.get())); + } + + private IOExceptionSupplier> fileDeserializer( + int version, DataInputView view) { + if (version >= 4) { + return () -> dataFileSerializer.deserializeList(view); + } else if (version == 3) { + if (dataFile09Serializer == null) { + dataFile09Serializer = new DataFileMeta09Serializer(); + } + return () -> dataFile09Serializer.deserializeList(view); + } else { + if (dataFile08Serializer == null) { + dataFile08Serializer = new DataFileMeta08Serializer(); + } + return () -> dataFile08Serializer.deserializeList(view); + } + } + + private IOExceptionSupplier> indexEntryDeserializer( + int version, DataInputView view) { + if (version >= 5) { + return () -> indexEntrySerializer.deserializeList(view); + } else { + if (indexEntry09Serializer == null) { + indexEntry09Serializer = new IndexFileMeta09Serializer(); + } + return () -> indexEntry09Serializer.deserializeList(view); + } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java b/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java index 1dac6584d698..b9460f28b4e7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java @@ -44,13 +44,14 @@ import static org.apache.paimon.io.DataFilePathFactory.INDEX_PATH_SUFFIX; import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.apache.paimon.utils.Preconditions.checkState; /** Input splits. Needed by most batch computation engines. */ public class DataSplit implements Split { private static final long serialVersionUID = 7L; private static final long MAGIC = -2394839472490812314L; - private static final int VERSION = 3; + private static final int VERSION = 4; private long snapshotId = 0; private BinaryRow partition; @@ -126,6 +127,45 @@ public long rowCount() { return rowCount; } + /** Whether it is possible to calculate the merged row count. */ + public boolean mergedRowCountAvailable() { + return rawConvertible + && (dataDeletionFiles == null + || dataDeletionFiles.stream() + .allMatch(f -> f == null || f.cardinality() != null)); + } + + public long mergedRowCount() { + checkState(mergedRowCountAvailable()); + return partialMergedRowCount(); + } + + /** + * Obtain merged row count as much as possible. There are two scenarios where accurate row count + * can be calculated: + * + *

    1. raw file and no deletion file. + * + *

    2. raw file + deletion file with cardinality. + */ + public long partialMergedRowCount() { + long sum = 0L; + if (rawConvertible) { + List rawFiles = convertToRawFiles().orElse(null); + if (rawFiles != null) { + for (int i = 0; i < rawFiles.size(); i++) { + RawFile rawFile = rawFiles.get(i); + if (dataDeletionFiles == null || dataDeletionFiles.get(i) == null) { + sum += rawFile.rowCount(); + } else if (dataDeletionFiles.get(i).cardinality() != null) { + sum += rawFile.rowCount() - dataDeletionFiles.get(i).cardinality(); + } + } + } + } + return sum; + } + @Override public Optional> convertToRawFiles() { if (rawConvertible) { @@ -272,13 +312,16 @@ public static DataSplit deserialize(DataInputView in) throws IOException { FunctionWithIOException dataFileSer = getFileMetaSerde(version); + FunctionWithIOException deletionFileSerde = + getDeletionFileSerde(version); int beforeNumber = in.readInt(); List beforeFiles = new ArrayList<>(beforeNumber); for (int i = 0; i < beforeNumber; i++) { beforeFiles.add(dataFileSer.apply(in)); } - List beforeDeletionFiles = DeletionFile.deserializeList(in); + List beforeDeletionFiles = + DeletionFile.deserializeList(in, deletionFileSerde); int fileNumber = in.readInt(); List dataFiles = new ArrayList<>(fileNumber); @@ -286,7 +329,7 @@ public static DataSplit deserialize(DataInputView in) throws IOException { dataFiles.add(dataFileSer.apply(in)); } - List dataDeletionFiles = DeletionFile.deserializeList(in); + List dataDeletionFiles = DeletionFile.deserializeList(in, deletionFileSerde); boolean isStreaming = in.readBoolean(); boolean rawConvertible = in.readBoolean(); @@ -319,16 +362,22 @@ private static FunctionWithIOException getFileMetaS } else if (version == 2) { DataFileMeta09Serializer serializer = new DataFileMeta09Serializer(); return serializer::deserialize; - } else if (version == 3) { + } else if (version >= 3) { DataFileMetaSerializer serializer = new DataFileMetaSerializer(); return serializer::deserialize; } else { - throw new UnsupportedOperationException( - "Expecting DataSplit version to be smaller or equal than " - + VERSION - + ", but found " - + version - + "."); + throw new UnsupportedOperationException("Unsupported version: " + version); + } + } + + private static FunctionWithIOException getDeletionFileSerde( + int version) { + if (version >= 1 && version <= 3) { + return DeletionFile::deserializeV3; + } else if (version >= 4) { + return DeletionFile::deserialize; + } else { + throw new UnsupportedOperationException("Unsupported version: " + version); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableBatchScan.java b/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableBatchScan.java index 635802cc9dcb..a4fe6d73bba1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableBatchScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableBatchScan.java @@ -28,7 +28,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Objects; import static org.apache.paimon.CoreOptions.MergeEngine.FIRST_ROW; @@ -103,9 +102,9 @@ private StartingScanner.Result applyPushDownLimit(StartingScanner.Result result) List limitedSplits = new ArrayList<>(); for (DataSplit dataSplit : splits) { if (dataSplit.rawConvertible()) { - long splitRowCount = getRowCountForSplit(dataSplit); + long partialMergedRowCount = dataSplit.partialMergedRowCount(); limitedSplits.add(dataSplit); - scannedRowCount += splitRowCount; + scannedRowCount += partialMergedRowCount; if (scannedRowCount >= pushDownLimit) { SnapshotReader.Plan newPlan = new PlanImpl(plan.watermark(), plan.snapshotId(), limitedSplits); @@ -117,20 +116,6 @@ private StartingScanner.Result applyPushDownLimit(StartingScanner.Result result) return result; } - /** - * 0 represents that we can't compute the row count of this split: 1. the split needs to be - * merged; 2. the table enabled deletion vector and there are some deletion files. - */ - private long getRowCountForSplit(DataSplit split) { - if (split.deletionFiles().isPresent() - && split.deletionFiles().get().stream().anyMatch(Objects::nonNull)) { - return 0L; - } - return split.convertToRawFiles() - .map(files -> files.stream().map(RawFile::rowCount).reduce(Long::sum).orElse(0L)) - .orElse(0L); - } - @Override public DataTableScan withShard(int indexOfThisSubtask, int numberOfParallelSubtasks) { snapshotReader.withShard(indexOfThisSubtask, numberOfParallelSubtasks); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/DeletionFile.java b/paimon-core/src/main/java/org/apache/paimon/table/source/DeletionFile.java index 94dfc615729c..5bcf6898ed99 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/DeletionFile.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/DeletionFile.java @@ -22,6 +22,7 @@ import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataInputView; import org.apache.paimon.io.DataOutputView; +import org.apache.paimon.utils.FunctionWithIOException; import javax.annotation.Nullable; @@ -52,11 +53,13 @@ public class DeletionFile implements Serializable { private final String path; private final long offset; private final long length; + @Nullable private final Long cardinality; - public DeletionFile(String path, long offset, long length) { + public DeletionFile(String path, long offset, long length, @Nullable Long cardinality) { this.path = path; this.offset = offset; this.length = length; + this.cardinality = cardinality; } /** Path of the file. */ @@ -74,6 +77,12 @@ public long length() { return length; } + /** the number of deleted rows. */ + @Nullable + public Long cardinality() { + return cardinality; + } + public static void serialize(DataOutputView out, @Nullable DeletionFile file) throws IOException { if (file == null) { @@ -83,6 +92,7 @@ public static void serialize(DataOutputView out, @Nullable DeletionFile file) out.writeUTF(file.path); out.writeLong(file.offset); out.writeLong(file.length); + out.writeLong(file.cardinality == null ? -1 : file.cardinality); } } @@ -108,17 +118,32 @@ public static DeletionFile deserialize(DataInputView in) throws IOException { String path = in.readUTF(); long offset = in.readLong(); long length = in.readLong(); - return new DeletionFile(path, offset, length); + long cardinality = in.readLong(); + return new DeletionFile(path, offset, length, cardinality == -1 ? null : cardinality); } @Nullable - public static List deserializeList(DataInputView in) throws IOException { + public static DeletionFile deserializeV3(DataInputView in) throws IOException { + if (in.readByte() == 0) { + return null; + } + + String path = in.readUTF(); + long offset = in.readLong(); + long length = in.readLong(); + return new DeletionFile(path, offset, length, null); + } + + @Nullable + public static List deserializeList( + DataInputView in, FunctionWithIOException deserialize) + throws IOException { List files = null; if (in.readByte() == 1) { int size = in.readInt(); files = new ArrayList<>(size); for (int i = 0; i < size; i++) { - files.add(DeletionFile.deserialize(in)); + files.add(deserialize.apply(in)); } } return files; @@ -126,22 +151,34 @@ public static List deserializeList(DataInputView in) throws IOExce @Override public boolean equals(Object o) { - if (!(o instanceof DeletionFile)) { + if (o == null || getClass() != o.getClass()) { return false; } - - DeletionFile other = (DeletionFile) o; - return Objects.equals(path, other.path) && offset == other.offset && length == other.length; + DeletionFile that = (DeletionFile) o; + return offset == that.offset + && length == that.length + && Objects.equals(path, that.path) + && Objects.equals(cardinality, that.cardinality); } @Override public int hashCode() { - return Objects.hash(path, offset, length); + return Objects.hash(path, offset, length, cardinality); } @Override public String toString() { - return String.format("{path = %s, offset = %d, length = %d}", path, offset, length); + return "DeletionFile{" + + "path='" + + path + + '\'' + + ", offset=" + + offset + + ", length=" + + length + + ", cardinality=" + + cardinality + + '}'; } static Factory emptyFactory() { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java index ce01bdba9447..bf19ba10c689 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java @@ -24,6 +24,7 @@ import org.apache.paimon.codegen.RecordComparator; import org.apache.paimon.consumer.ConsumerManager; import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.index.DeletionVectorMeta; import org.apache.paimon.index.IndexFileHandler; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.io.DataFileMeta; @@ -492,23 +493,24 @@ private List getDeletionFiles( List deletionFiles = new ArrayList<>(dataFiles.size()); Map dataFileToIndexFileMeta = new HashMap<>(); for (IndexFileMeta indexFileMeta : indexFileMetas) { - if (indexFileMeta.deletionVectorsRanges() != null) { - for (String dataFileName : indexFileMeta.deletionVectorsRanges().keySet()) { - dataFileToIndexFileMeta.put(dataFileName, indexFileMeta); + if (indexFileMeta.deletionVectorMetas() != null) { + for (DeletionVectorMeta dvMeta : indexFileMeta.deletionVectorMetas().values()) { + dataFileToIndexFileMeta.put(dvMeta.dataFileName(), indexFileMeta); } } } for (DataFileMeta file : dataFiles) { IndexFileMeta indexFileMeta = dataFileToIndexFileMeta.get(file.fileName()); if (indexFileMeta != null) { - Map> ranges = indexFileMeta.deletionVectorsRanges(); - if (ranges != null && ranges.containsKey(file.fileName())) { - Pair range = ranges.get(file.fileName()); + LinkedHashMap dvMetas = + indexFileMeta.deletionVectorMetas(); + if (dvMetas != null && dvMetas.containsKey(file.fileName())) { deletionFiles.add( new DeletionFile( indexFileHandler.filePath(indexFileMeta).toString(), - range.getKey(), - range.getValue())); + dvMetas.get(file.fileName()).offset(), + dvMetas.get(file.fileName()).length(), + dvMetas.get(file.fileName()).cardinality())); continue; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/splitread/IncrementalChangelogReadProvider.java b/paimon-core/src/main/java/org/apache/paimon/table/source/splitread/IncrementalChangelogReadProvider.java index 308c09d14204..eb41d02669fc 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/splitread/IncrementalChangelogReadProvider.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/splitread/IncrementalChangelogReadProvider.java @@ -60,20 +60,20 @@ private SplitRead create(Supplier supplier) { ConcatRecordReader.create( () -> new ReverseReader( - read.createNoMergeReader( + read.createMergeReader( split.partition(), split.bucket(), split.beforeFiles(), split.beforeDeletionFiles() .orElse(null), - true)), + false)), () -> - read.createNoMergeReader( + read.createMergeReader( split.partition(), split.bucket(), split.dataFiles(), split.deletionFiles().orElse(null), - true)); + false)); return unwrap(reader); }; diff --git a/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoCreation.java b/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoCreation.java index 58241033f5fb..3989786bd277 100644 --- a/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoCreation.java +++ b/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoCreation.java @@ -150,22 +150,28 @@ public void run() { private void tryToCreateTags(Snapshot snapshot) { Optional timeOptional = timeExtractor.extract(snapshot.timeMillis(), snapshot.watermark()); + LOG.info("Starting to create a tag for snapshot {}.", snapshot.id()); if (!timeOptional.isPresent()) { return; } LocalDateTime time = timeOptional.get(); + LOG.info("The time of snapshot {} is {}.", snapshot.id(), time); + LOG.info("The next tag time is {}.", nextTag); if (nextTag == null || isAfterOrEqual(time.minus(delay), periodHandler.nextTagTime(nextTag))) { LocalDateTime thisTag = periodHandler.normalizeToPreviousTag(time); + LOG.info("Create tag for snapshot {} with time {}.", snapshot.id(), thisTag); if (automaticCompletion && nextTag != null) { thisTag = nextTag; } String tagName = periodHandler.timeToTag(thisTag); + LOG.info("The tag name is {}.", tagName); if (!tagManager.tagExists(tagName)) { tagManager.createTag(snapshot, tagName, defaultTimeRetained, callbacks); } nextTag = periodHandler.nextTagTime(thisTag); + LOG.info("The next tag time after this is {}.", nextTag); if (numRetainedMax != null) { // only handle auto-created tags here diff --git a/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoManager.java b/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoManager.java index 1ed1b3f2d4a2..817c20af4612 100644 --- a/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoManager.java @@ -52,6 +52,7 @@ public static TagAutoManager create( TagManager tagManager, TagDeletion tagDeletion, List callbacks) { + TagTimeExtractor extractor = TagTimeExtractor.createForAutoTag(options); return new TagAutoManager( diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java index 8896ec328680..f255762cfd3c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java @@ -130,20 +130,19 @@ public DataFilePathFactory createDataFilePathFactory(BinaryRow partition, int bu } public Path bucketPath(BinaryRow partition, int bucket) { - Path dataFileRoot = this.root; - if (dataFilePathDirectory != null) { - dataFileRoot = new Path(dataFileRoot, dataFilePathDirectory); - } - return new Path(dataFileRoot + "/" + relativePartitionAndBucketPath(partition, bucket)); + return new Path(root, relativeBucketPath(partition, bucket)); } - public Path relativePartitionAndBucketPath(BinaryRow partition, int bucket) { + public Path relativeBucketPath(BinaryRow partition, int bucket) { + Path relativeBucketPath = new Path(BUCKET_PATH_PREFIX + bucket); String partitionPath = getPartitionString(partition); - String fullPath = - partitionPath.isEmpty() - ? BUCKET_PATH_PREFIX + bucket - : partitionPath + "/" + BUCKET_PATH_PREFIX + bucket; - return new Path(fullPath); + if (!partitionPath.isEmpty()) { + relativeBucketPath = new Path(partitionPath, relativeBucketPath); + } + if (dataFilePathDirectory != null) { + relativeBucketPath = new Path(dataFilePathDirectory, relativeBucketPath); + } + return relativeBucketPath; } /** IMPORTANT: This method is NOT THREAD SAFE. */ diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java b/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java index eb7333366fec..49da83bfe48a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java @@ -45,6 +45,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.function.BinaryOperator; import java.util.function.Consumer; @@ -883,6 +884,23 @@ public void commitEarliestHint(long snapshotId) throws IOException { private void commitHint(long snapshotId, String fileName, Path dir) throws IOException { Path hintFile = new Path(dir, fileName); - fileIO.overwriteFileUtf8(hintFile, String.valueOf(snapshotId)); + int loopTime = 3; + while (loopTime-- > 0) { + try { + fileIO.overwriteFileUtf8(hintFile, String.valueOf(snapshotId)); + return; + } catch (IOException e) { + try { + Thread.sleep(ThreadLocalRandom.current().nextInt(1000) + 500); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + // throw root cause + throw new RuntimeException(e); + } + if (loopTime == 0) { + throw e; + } + } + } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/StatsCollectorFactories.java b/paimon-core/src/main/java/org/apache/paimon/utils/StatsCollectorFactories.java index de94b2e23eff..abb1d686073f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/StatsCollectorFactories.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/StatsCollectorFactories.java @@ -24,6 +24,7 @@ import org.apache.paimon.statistics.TruncateSimpleColStatsCollector; import org.apache.paimon.table.SpecialFields; +import java.util.Collections; import java.util.List; import static org.apache.paimon.CoreOptions.FIELDS_PREFIX; @@ -35,6 +36,11 @@ public class StatsCollectorFactories { public static SimpleColStatsCollector.Factory[] createStatsFactories( CoreOptions options, List fields) { + return createStatsFactories(options, fields, Collections.emptyList()); + } + + public static SimpleColStatsCollector.Factory[] createStatsFactories( + CoreOptions options, List fields, List keyNames) { Options cfg = options.toConfiguration(); SimpleColStatsCollector.Factory[] modes = new SimpleColStatsCollector.Factory[fields.size()]; @@ -47,7 +53,11 @@ public static SimpleColStatsCollector.Factory[] createStatsFactories( .noDefaultValue()); if (fieldMode != null) { modes[i] = SimpleColStatsCollector.from(fieldMode); - } else if (SpecialFields.isSystemField(field)) { + } else if (SpecialFields.isSystemField(field) + || + // If we config DATA_FILE_THIN_MODE to true, we need to maintain the + // stats for key fields. + keyNames.contains(SpecialFields.KEY_FIELD_PREFIX + field)) { modes[i] = () -> new TruncateSimpleColStatsCollector(128); } else { modes[i] = SimpleColStatsCollector.from(cfg.get(CoreOptions.METADATA_STATS_MODE)); diff --git a/paimon-core/src/test/java/org/apache/paimon/TestKeyValueGenerator.java b/paimon-core/src/test/java/org/apache/paimon/TestKeyValueGenerator.java index 657c791351a4..587204cd7616 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestKeyValueGenerator.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestKeyValueGenerator.java @@ -95,10 +95,12 @@ public class TestKeyValueGenerator { public static final RowType KEY_TYPE = RowType.of( new DataField( - 2 + SpecialFields.KEY_FIELD_ID_START, "key_shopId", new IntType(false)), + 2 + SpecialFields.KEY_FIELD_ID_START, + SpecialFields.KEY_FIELD_PREFIX + "shopId", + new IntType(false)), new DataField( 3 + SpecialFields.KEY_FIELD_ID_START, - "key_orderId", + SpecialFields.KEY_FIELD_PREFIX + "orderId", new BigIntType(false))); public static final InternalRowSerializer DEFAULT_ROW_SERIALIZER = @@ -281,7 +283,7 @@ public BinaryRow getPartition(KeyValue kv) { public static List getPrimaryKeys(GeneratorMode mode) { List trimmedPk = KEY_TYPE.getFieldNames().stream() - .map(f -> f.replaceFirst("key_", "")) + .map(f -> f.replaceFirst(SpecialFields.KEY_FIELD_PREFIX, "")) .collect(Collectors.toList()); if (mode != NON_PARTITIONED) { trimmedPk = new ArrayList<>(trimmedPk); @@ -394,7 +396,7 @@ public List keyFields(TableSchema schema) { f -> new DataField( f.id() + SpecialFields.KEY_FIELD_ID_START, - "key_" + f.name(), + SpecialFields.KEY_FIELD_PREFIX + f.name(), f.type(), f.description())) .collect(Collectors.toList()); diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java index aef2f8f485ce..7045daca8e86 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java @@ -39,12 +39,12 @@ public class FileSystemCatalogTest extends CatalogTestBase { public void setUp() throws Exception { super.setUp(); Options catalogOptions = new Options(); - catalogOptions.set(CatalogOptions.ALLOW_UPPER_CASE, false); + catalogOptions.set(CatalogOptions.CASE_SENSITIVE, false); catalog = new FileSystemCatalog(fileIO, new Path(warehouse), catalogOptions); } @Test - public void testCreateTableAllowUpperCase() throws Exception { + public void testCreateTableCaseSensitive() throws Exception { catalog.createDatabase("test_db", false); Identifier identifier = Identifier.create("test_db", "new_table"); Schema schema = diff --git a/paimon-core/src/test/java/org/apache/paimon/deletionvectors/append/AppendDeletionFileMaintainerTest.java b/paimon-core/src/test/java/org/apache/paimon/deletionvectors/append/AppendDeletionFileMaintainerTest.java index 6c674352b8d3..a52819c80515 100644 --- a/paimon-core/src/test/java/org/apache/paimon/deletionvectors/append/AppendDeletionFileMaintainerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/deletionvectors/append/AppendDeletionFileMaintainerTest.java @@ -23,12 +23,12 @@ import org.apache.paimon.deletionvectors.DeletionVector; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.index.DeletionVectorMeta; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.manifest.FileKind; import org.apache.paimon.manifest.IndexManifestEntry; import org.apache.paimon.table.sink.CommitMessageImpl; import org.apache.paimon.table.source.DeletionFile; -import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.PathFactory; import org.junit.jupiter.api.Test; @@ -94,7 +94,7 @@ public void test() throws Exception { assertThat(res.size()).isEqualTo(3); IndexManifestEntry entry = res.stream().filter(file -> file.kind() == FileKind.ADD).findAny().get(); - assertThat(entry.indexFile().deletionVectorsRanges().containsKey("f2")).isTrue(); + assertThat(entry.indexFile().deletionVectorMetas().containsKey("f2")).isTrue(); entry = res.stream() .filter(file -> file.kind() == FileKind.DELETE) @@ -117,14 +117,15 @@ private Map createDeletionFileMapFromIndexFileMetas( PathFactory indexPathFactory, List fileMetas) { Map dataFileToDeletionFiles = new HashMap<>(); for (IndexFileMeta indexFileMeta : fileMetas) { - for (Map.Entry> range : - indexFileMeta.deletionVectorsRanges().entrySet()) { + for (Map.Entry dvMeta : + indexFileMeta.deletionVectorMetas().entrySet()) { dataFileToDeletionFiles.put( - range.getKey(), + dvMeta.getKey(), new DeletionFile( indexPathFactory.toPath(indexFileMeta.fileName()).toString(), - range.getValue().getLeft(), - range.getValue().getRight())); + dvMeta.getValue().offset(), + dvMeta.getValue().length(), + dvMeta.getValue().cardinality())); } } return dataFileToDeletionFiles; diff --git a/paimon-core/src/test/java/org/apache/paimon/format/ThinModeReadWriteTest.java b/paimon-core/src/test/java/org/apache/paimon/format/ThinModeReadWriteTest.java new file mode 100644 index 000000000000..3f8015b33b2d --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/format/ThinModeReadWriteTest.java @@ -0,0 +1,158 @@ +/* + * 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.format; + +import org.apache.paimon.data.Decimal; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.manifest.FileKind; +import org.apache.paimon.manifest.ManifestEntry; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.TableTestBase; +import org.apache.paimon.types.DataTypes; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; + +/** This class test the compatibility and effectiveness of storage thin mode. */ +public class ThinModeReadWriteTest extends TableTestBase { + + private Table createTable(String format, Boolean thinMode) throws Exception { + catalog.createTable(identifier(), schema(format, thinMode), true); + return catalog.getTable(identifier()); + } + + private Schema schema(String format, Boolean thinMode) { + Schema.Builder schemaBuilder = Schema.newBuilder(); + schemaBuilder.column("f0", DataTypes.INT()); + schemaBuilder.column("f1", DataTypes.INT()); + schemaBuilder.column("f2", DataTypes.SMALLINT()); + schemaBuilder.column("f3", DataTypes.STRING()); + schemaBuilder.column("f4", DataTypes.DOUBLE()); + schemaBuilder.column("f5", DataTypes.CHAR(100)); + schemaBuilder.column("f6", DataTypes.VARCHAR(100)); + schemaBuilder.column("f7", DataTypes.BOOLEAN()); + schemaBuilder.column("f8", DataTypes.INT()); + schemaBuilder.column("f9", DataTypes.TIME()); + schemaBuilder.column("f10", DataTypes.TIMESTAMP()); + schemaBuilder.column("f11", DataTypes.DECIMAL(10, 2)); + schemaBuilder.column("f12", DataTypes.BYTES()); + schemaBuilder.column("f13", DataTypes.FLOAT()); + schemaBuilder.column("f14", DataTypes.BINARY(100)); + schemaBuilder.column("f15", DataTypes.VARBINARY(100)); + schemaBuilder.primaryKey( + "f0", "f1", "f2", "f3", "f4", "f5", "f6", "f7", "f8", "f9", "f10", "f11", "f12", + "f13"); + schemaBuilder.option("bucket", "1"); + schemaBuilder.option("bucket-key", "f1"); + schemaBuilder.option("file.format", format); + schemaBuilder.option("data-file.thin-mode", thinMode.toString()); + return schemaBuilder.build(); + } + + @Test + public void testThinModeWorks() throws Exception { + + InternalRow[] datas = datas(200000); + + Table table = createTable("orc", true); + write(table, datas); + + long size1 = tableSize(table); + dropTableDefault(); + + table = createTable("orc", false); + write(table, datas); + long size2 = tableSize(table); + dropTableDefault(); + + Assertions.assertThat(size2).isGreaterThan(size1); + } + + @Test + public void testAllFormatReadWrite() throws Exception { + testFormat("orc"); + testFormat("parquet"); + testFormat("avro"); + } + + private void testFormat(String format) throws Exception { + testReadWrite(format, true); + testReadWrite(format, true); + testReadWrite(format, false); + testReadWrite(format, false); + } + + private void testReadWrite(String format, boolean writeThin) throws Exception { + Table table = createTable(format, writeThin); + + InternalRow[] datas = datas(2000); + + write(table, datas); + + List readed = read(table); + + Assertions.assertThat(readed).containsExactlyInAnyOrder(datas); + dropTableDefault(); + } + + InternalRow[] datas(int i) { + InternalRow[] arrays = new InternalRow[i]; + for (int j = 0; j < i; j++) { + arrays[j] = data(); + } + return arrays; + } + + protected InternalRow data() { + return GenericRow.of( + RANDOM.nextInt(), + RANDOM.nextInt(), + (short) RANDOM.nextInt(), + randomString(), + RANDOM.nextDouble(), + randomString(), + randomString(), + RANDOM.nextBoolean(), + RANDOM.nextInt(), + RANDOM.nextInt(), + Timestamp.now(), + Decimal.zero(10, 2), + randomBytes(), + (float) RANDOM.nextDouble(), + randomBytes(), + randomBytes()); + } + + public static long tableSize(Table table) { + long count = 0; + List files = + ((FileStoreTable) table).store().newScan().plan().files(FileKind.ADD); + for (ManifestEntry file : files) { + count += file.file().fileSize(); + } + + return count; + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/index/IndexFileMetaSerializerTest.java b/paimon-core/src/test/java/org/apache/paimon/index/IndexFileMetaSerializerTest.java index 724d5b416359..a7e692d2e554 100644 --- a/paimon-core/src/test/java/org/apache/paimon/index/IndexFileMetaSerializerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/index/IndexFileMetaSerializerTest.java @@ -21,7 +21,6 @@ import org.apache.paimon.deletionvectors.DeletionVectorsIndexFile; import org.apache.paimon.utils.ObjectSerializer; import org.apache.paimon.utils.ObjectSerializerTestBase; -import org.apache.paimon.utils.Pair; import java.util.LinkedHashMap; import java.util.Random; @@ -59,14 +58,20 @@ public static IndexFileMeta randomHashIndexFile() { public static IndexFileMeta randomDeletionVectorIndexFile() { Random rnd = new Random(); - LinkedHashMap> deletionVectorsRanges = new LinkedHashMap<>(); - deletionVectorsRanges.put("my_file_name1", Pair.of(rnd.nextInt(), rnd.nextInt())); - deletionVectorsRanges.put("my_file_name2", Pair.of(rnd.nextInt(), rnd.nextInt())); + LinkedHashMap deletionVectorMetas = new LinkedHashMap<>(); + deletionVectorMetas.put( + "my_file_name1", + new DeletionVectorMeta( + "my_file_name1", rnd.nextInt(), rnd.nextInt(), rnd.nextLong())); + deletionVectorMetas.put( + "my_file_name2", + new DeletionVectorMeta( + "my_file_name2", rnd.nextInt(), rnd.nextInt(), rnd.nextLong())); return new IndexFileMeta( DeletionVectorsIndexFile.DELETION_VECTORS_INDEX, "deletion_vectors_index_file_name" + rnd.nextLong(), rnd.nextInt(), rnd.nextInt(), - deletionVectorsRanges); + deletionVectorMetas); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java index b648e2af8972..e43cd898dbc2 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java @@ -37,6 +37,7 @@ import org.apache.paimon.options.Options; import org.apache.paimon.reader.RecordReaderIterator; import org.apache.paimon.stats.StatsTestUtils; +import org.apache.paimon.table.SpecialFields; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.CloseableIterator; import org.apache.paimon.utils.FailingFileIO; @@ -158,7 +159,7 @@ public void testReadKeyType() throws Exception { List actualMetas = writer.result(); // projection: (shopId, orderId) -> (orderId) - RowType readKeyType = KEY_TYPE.project("key_orderId"); + RowType readKeyType = KEY_TYPE.project(SpecialFields.KEY_FIELD_PREFIX + "orderId"); KeyValueFileReaderFactory readerFactory = createReaderFactory(tempDir.toString(), "avro", readKeyType, null); InternalRowSerializer projectedKeySerializer = new InternalRowSerializer(readKeyType); 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 bd272b745dc4..fbc02b2d73f2 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 @@ -19,6 +19,7 @@ package org.apache.paimon.manifest; import org.apache.paimon.data.Timestamp; +import org.apache.paimon.index.DeletionVectorMeta; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataFileMeta; @@ -27,7 +28,6 @@ import org.apache.paimon.stats.SimpleStats; import org.apache.paimon.table.sink.CommitMessageImpl; import org.apache.paimon.utils.IOUtils; -import org.apache.paimon.utils.Pair; import org.junit.jupiter.api.Test; @@ -78,11 +78,11 @@ public void testProduction() throws IOException { Arrays.asList("field1", "field2", "field3")); List dataFiles = Collections.singletonList(dataFile); - LinkedHashMap> dvRanges = new LinkedHashMap<>(); - dvRanges.put("dv_key1", Pair.of(1, 2)); - dvRanges.put("dv_key2", Pair.of(3, 4)); + LinkedHashMap dvMetas = new LinkedHashMap<>(); + dvMetas.put("dv_key1", new DeletionVectorMeta("dv_key1", 1, 2, 3L)); + dvMetas.put("dv_key2", new DeletionVectorMeta("dv_key2", 3, 4, 5L)); IndexFileMeta indexFile = - new IndexFileMeta("my_index_type", "my_index_file", 1024 * 100, 1002, dvRanges); + new IndexFileMeta("my_index_type", "my_index_file", 1024 * 100, 1002, dvMetas); List indexFiles = Collections.singletonList(indexFile); CommitMessageImpl commitMessage = @@ -106,6 +106,76 @@ public void testProduction() throws IOException { assertThat(deserialized).isEqualTo(manifestCommittable); } + @Test + public void testCompatibilityToVersion4() throws IOException { + 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 = + new DataFileMeta( + "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")); + List dataFiles = Collections.singletonList(dataFile); + + LinkedHashMap dvMetas = new LinkedHashMap<>(); + dvMetas.put("dv_key1", new DeletionVectorMeta("dv_key1", 1, 2, null)); + dvMetas.put("dv_key2", new DeletionVectorMeta("dv_key2", 3, 4, null)); + IndexFileMeta indexFile = + new IndexFileMeta("my_index_type", "my_index_file", 1024 * 100, 1002, dvMetas); + List indexFiles = Collections.singletonList(indexFile); + + CommitMessageImpl commitMessage = + new CommitMessageImpl( + singleColumn("my_partition"), + 11, + new DataIncrement(dataFiles, dataFiles, dataFiles), + new CompactIncrement(dataFiles, dataFiles, dataFiles), + new IndexIncrement(indexFiles)); + + ManifestCommittable manifestCommittable = + new ManifestCommittable( + 5, + 202020L, + Collections.singletonMap(5, 555L), + Collections.singletonList(commitMessage)); + + ManifestCommittableSerializer serializer = new ManifestCommittableSerializer(); + byte[] bytes = serializer.serialize(manifestCommittable); + ManifestCommittable deserialized = serializer.deserialize(3, bytes); + assertThat(deserialized).isEqualTo(manifestCommittable); + + byte[] v2Bytes = + IOUtils.readFully( + ManifestCommittableSerializerCompatibilityTest.class + .getClassLoader() + .getResourceAsStream("compatibility/manifest-committable-v4"), + true); + deserialized = serializer.deserialize(2, v2Bytes); + assertThat(deserialized).isEqualTo(manifestCommittable); + } + @Test public void testCompatibilityToVersion3() throws IOException { SimpleStats keyStats = @@ -139,11 +209,11 @@ public void testCompatibilityToVersion3() throws IOException { null); List dataFiles = Collections.singletonList(dataFile); - LinkedHashMap> dvRanges = new LinkedHashMap<>(); - dvRanges.put("dv_key1", Pair.of(1, 2)); - dvRanges.put("dv_key2", Pair.of(3, 4)); + LinkedHashMap dvMetas = new LinkedHashMap<>(); + dvMetas.put("dv_key1", new DeletionVectorMeta("dv_key1", 1, 2, null)); + dvMetas.put("dv_key2", new DeletionVectorMeta("dv_key2", 3, 4, null)); IndexFileMeta indexFile = - new IndexFileMeta("my_index_type", "my_index_file", 1024 * 100, 1002, dvRanges); + new IndexFileMeta("my_index_type", "my_index_file", 1024 * 100, 1002, dvMetas); List indexFiles = Collections.singletonList(indexFile); CommitMessageImpl commitMessage = @@ -209,11 +279,11 @@ public void testCompatibilityToVersion2() throws IOException { null); List dataFiles = Collections.singletonList(dataFile); - LinkedHashMap> dvRanges = new LinkedHashMap<>(); - dvRanges.put("dv_key1", Pair.of(1, 2)); - dvRanges.put("dv_key2", Pair.of(3, 4)); + LinkedHashMap dvMetas = new LinkedHashMap<>(); + dvMetas.put("dv_key1", new DeletionVectorMeta("dv_key1", 1, 2, null)); + dvMetas.put("dv_key2", new DeletionVectorMeta("dv_key2", 3, 4, null)); IndexFileMeta indexFile = - new IndexFileMeta("my_index_type", "my_index_file", 1024 * 100, 1002, dvRanges); + new IndexFileMeta("my_index_type", "my_index_file", 1024 * 100, 1002, dvMetas); List indexFiles = Collections.singletonList(indexFile); CommitMessageImpl commitMessage = diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java index 0ab636c33aa3..be49311427a0 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java @@ -41,6 +41,7 @@ import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.SchemaEvolutionTableTestBase; +import org.apache.paimon.table.SpecialFields; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowKind; @@ -77,7 +78,12 @@ public class ContainsLevelsTest { private final Comparator comparator = Comparator.comparingInt(o -> o.getInt(0)); - private final RowType keyType = DataTypes.ROW(DataTypes.FIELD(0, "_key", DataTypes.INT())); + private final RowType keyType = + DataTypes.ROW( + DataTypes.FIELD( + SpecialFields.KEY_FIELD_ID_START, + SpecialFields.KEY_FIELD_PREFIX + "key", + DataTypes.INT())); private final RowType rowType = DataTypes.ROW( DataTypes.FIELD(0, "key", DataTypes.INT()), diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java index 2dce81ce56b4..a678534042eb 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java @@ -41,6 +41,7 @@ import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.SchemaEvolutionTableTestBase; +import org.apache.paimon.table.SpecialFields; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowKind; @@ -79,7 +80,9 @@ public class LookupLevelsTest { private final Comparator comparator = Comparator.comparingInt(o -> o.getInt(0)); - private final RowType keyType = DataTypes.ROW(DataTypes.FIELD(0, "_key", DataTypes.INT())); + private final RowType keyType = + DataTypes.ROW( + DataTypes.FIELD(SpecialFields.KEY_FIELD_ID_START, "_KEY_key", DataTypes.INT())); private final RowType rowType = DataTypes.ROW( DataTypes.FIELD(0, "key", DataTypes.INT()), diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/MergeFileSplitReadTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/MergeFileSplitReadTest.java index 46b64422fd9b..59f848a296cf 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/MergeFileSplitReadTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/MergeFileSplitReadTest.java @@ -37,6 +37,7 @@ import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.SpecialFields; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.DataField; @@ -284,7 +285,12 @@ private TestFileStore createStore( ? Collections.emptyList() : Stream.concat( keyType.getFieldNames().stream() - .map(field -> field.replace("key_", "")), + .map( + field -> + field.replace( + SpecialFields + .KEY_FIELD_PREFIX, + "")), partitionType.getFieldNames().stream()) .collect(Collectors.toList()), Collections.emptyMap(), diff --git a/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java b/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java index 25282d898a3d..ce8cfc9228ad 100644 --- a/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java @@ -33,7 +33,8 @@ import org.apache.paimon.table.TableTestBase; import org.apache.paimon.types.DataTypes; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import static org.apache.paimon.CoreOptions.METADATA_STATS_DENSE_STORE; import static org.apache.paimon.CoreOptions.METADATA_STATS_MODE; @@ -42,13 +43,15 @@ /** Test for table stats mode. */ public class StatsTableTest extends TableTestBase { - @Test - public void testPartitionStatsNotDense() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testPartitionStatsNotDense(boolean thinMode) throws Exception { Identifier identifier = identifier("T"); Options options = new Options(); options.set(METADATA_STATS_MODE, "NONE"); options.set(METADATA_STATS_DENSE_STORE, false); options.set(CoreOptions.BUCKET, 1); + options.set(CoreOptions.DATA_FILE_THIN_MODE, thinMode); Schema schema = Schema.newBuilder() .column("pt", DataTypes.INT()) @@ -86,19 +89,25 @@ public void testPartitionStatsNotDense() throws Exception { manifestFile.read(manifest.fileName(), manifest.fileSize()).get(0).file(); SimpleStats recordStats = file.valueStats(); assertThat(recordStats.minValues().isNullAt(0)).isTrue(); - assertThat(recordStats.minValues().isNullAt(1)).isTrue(); + assertThat(recordStats.minValues().isNullAt(1)).isEqualTo(!thinMode); assertThat(recordStats.minValues().isNullAt(2)).isTrue(); assertThat(recordStats.maxValues().isNullAt(0)).isTrue(); - assertThat(recordStats.maxValues().isNullAt(1)).isTrue(); + assertThat(recordStats.maxValues().isNullAt(1)).isEqualTo(!thinMode); assertThat(recordStats.maxValues().isNullAt(2)).isTrue(); + + SimpleStats keyStats = file.keyStats(); + assertThat(keyStats.minValues().isNullAt(0)).isFalse(); + assertThat(keyStats.maxValues().isNullAt(0)).isFalse(); } - @Test - public void testPartitionStatsDenseMode() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testPartitionStatsDenseMode(boolean thinMode) throws Exception { Identifier identifier = identifier("T"); Options options = new Options(); options.set(METADATA_STATS_MODE, "NONE"); options.set(CoreOptions.BUCKET, 1); + options.set(CoreOptions.DATA_FILE_THIN_MODE, thinMode); Schema schema = Schema.newBuilder() .column("pt", DataTypes.INT()) @@ -135,9 +144,10 @@ public void testPartitionStatsDenseMode() throws Exception { DataFileMeta file = manifestFile.read(manifest.fileName(), manifest.fileSize()).get(0).file(); SimpleStats recordStats = file.valueStats(); - assertThat(file.valueStatsCols()).isEmpty(); - assertThat(recordStats.minValues().getFieldCount()).isEqualTo(0); - assertThat(recordStats.maxValues().getFieldCount()).isEqualTo(0); - assertThat(recordStats.nullCounts().size()).isEqualTo(0); + int count = thinMode ? 1 : 0; + assertThat(file.valueStatsCols().size()).isEqualTo(count); + assertThat(recordStats.minValues().getFieldCount()).isEqualTo(count); + assertThat(recordStats.maxValues().getFieldCount()).isEqualTo(count); + assertThat(recordStats.nullCounts().size()).isEqualTo(count); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/table/TableTestBase.java b/paimon-core/src/test/java/org/apache/paimon/table/TableTestBase.java index 7f850a7725b4..7d7617cf8bd1 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/TableTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/TableTestBase.java @@ -160,6 +160,10 @@ protected void compact( } } + public void dropTableDefault() throws Exception { + catalog.dropTable(identifier(), true); + } + protected List read(Table table, Pair, String>... dynamicOptions) throws Exception { return read(table, null, dynamicOptions); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/sink/CommitMessageSerializerTest.java b/paimon-core/src/test/java/org/apache/paimon/table/sink/CommitMessageSerializerTest.java index eb9105189b71..1f87838aea31 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/sink/CommitMessageSerializerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/sink/CommitMessageSerializerTest.java @@ -48,7 +48,7 @@ public void test() throws IOException { CommitMessageImpl committable = new CommitMessageImpl(row(0), 1, dataIncrement, compactIncrement, indexIncrement); CommitMessageImpl newCommittable = - (CommitMessageImpl) serializer.deserialize(3, serializer.serialize(committable)); + (CommitMessageImpl) serializer.deserialize(5, serializer.serialize(committable)); assertThat(newCommittable.compactIncrement()).isEqualTo(committable.compactIncrement()); assertThat(newCommittable.newFilesIncrement()).isEqualTo(committable.newFilesIncrement()); assertThat(newCommittable.indexIncrement()).isEqualTo(committable.indexIncrement()); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java index a9e093dab124..a1f7d69e2877 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java @@ -43,10 +43,10 @@ public class SplitGeneratorTest { public static DataFileMeta newFileFromSequence( - String name, int rowCount, long minSequence, long maxSequence) { + String name, int fileSize, long minSequence, long maxSequence) { return new DataFileMeta( name, - rowCount, + fileSize, 1, EMPTY_ROW, EMPTY_ROW, diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java index c64a12ffae2c..0219941a0ac0 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java @@ -49,6 +49,41 @@ /** Test for {@link DataSplit}. */ public class SplitTest { + @Test + public void testSplitMergedRowCount() { + // not rawConvertible + List dataFiles = + Arrays.asList(newDataFile(1000L), newDataFile(2000L), newDataFile(3000L)); + DataSplit split = newDataSplit(false, dataFiles, null); + assertThat(split.partialMergedRowCount()).isEqualTo(0L); + assertThat(split.mergedRowCountAvailable()).isEqualTo(false); + + // rawConvertible without deletion files + split = newDataSplit(true, dataFiles, null); + assertThat(split.partialMergedRowCount()).isEqualTo(6000L); + assertThat(split.mergedRowCountAvailable()).isEqualTo(true); + assertThat(split.mergedRowCount()).isEqualTo(6000L); + + // rawConvertible with deletion files without cardinality + ArrayList deletionFiles = new ArrayList<>(); + deletionFiles.add(null); + deletionFiles.add(new DeletionFile("p", 1, 2, null)); + deletionFiles.add(new DeletionFile("p", 1, 2, 100L)); + split = newDataSplit(true, dataFiles, deletionFiles); + assertThat(split.partialMergedRowCount()).isEqualTo(3900L); + assertThat(split.mergedRowCountAvailable()).isEqualTo(false); + + // rawConvertible with deletion files with cardinality + deletionFiles = new ArrayList<>(); + deletionFiles.add(null); + deletionFiles.add(new DeletionFile("p", 1, 2, 200L)); + deletionFiles.add(new DeletionFile("p", 1, 2, 100L)); + split = newDataSplit(true, dataFiles, deletionFiles); + assertThat(split.partialMergedRowCount()).isEqualTo(5700L); + assertThat(split.mergedRowCountAvailable()).isEqualTo(true); + assertThat(split.mergedRowCount()).isEqualTo(5700L); + } + @Test public void testSerializer() throws IOException { DataFileTestDataGenerator gen = DataFileTestDataGenerator.builder().build(); @@ -107,6 +142,9 @@ public void testSerializerNormal() throws Exception { Arrays.asList("field1", "field2", "field3")); List dataFiles = Collections.singletonList(dataFile); + DeletionFile deletionFile = new DeletionFile("deletion_file", 100, 22, 33L); + List deletionFiles = Collections.singletonList(deletionFile); + BinaryRow partition = new BinaryRow(1); BinaryRowWriter binaryRowWriter = new BinaryRowWriter(partition); binaryRowWriter.writeString(0, BinaryString.fromString("aaaaa")); @@ -118,6 +156,7 @@ public void testSerializerNormal() throws Exception { .withPartition(partition) .withBucket(20) .withDataFiles(dataFiles) + .withDataDeletionFiles(deletionFiles) .withBucketPath("my path") .build(); @@ -243,4 +282,100 @@ public void testSerializerCompatibleV2() throws Exception { InstantiationUtil.deserializeObject(v2Bytes, DataSplit.class.getClassLoader()); assertThat(actual).isEqualTo(split); } + + @Test + public void testSerializerCompatibleV3() throws Exception { + 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 = + new DataFileMeta( + "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")); + List dataFiles = Collections.singletonList(dataFile); + + DeletionFile deletionFile = new DeletionFile("deletion_file", 100, 22, null); + List deletionFiles = Collections.singletonList(deletionFile); + + BinaryRow partition = new BinaryRow(1); + BinaryRowWriter binaryRowWriter = new BinaryRowWriter(partition); + binaryRowWriter.writeString(0, BinaryString.fromString("aaaaa")); + binaryRowWriter.complete(); + + DataSplit split = + DataSplit.builder() + .withSnapshot(18) + .withPartition(partition) + .withBucket(20) + .withDataFiles(dataFiles) + .withDataDeletionFiles(deletionFiles) + .withBucketPath("my path") + .build(); + + byte[] v2Bytes = + IOUtils.readFully( + SplitTest.class + .getClassLoader() + .getResourceAsStream("compatibility/datasplit-v3"), + true); + + DataSplit actual = + InstantiationUtil.deserializeObject(v2Bytes, DataSplit.class.getClassLoader()); + assertThat(actual).isEqualTo(split); + } + + private DataFileMeta newDataFile(long rowCount) { + return DataFileMeta.forAppend( + "my_data_file.parquet", + 1024 * 1024, + rowCount, + null, + 0L, + rowCount, + 1, + Collections.emptyList(), + null, + null, + null); + } + + private DataSplit newDataSplit( + boolean rawConvertible, + List dataFiles, + List deletionFiles) { + DataSplit.Builder builder = DataSplit.builder(); + builder.withSnapshot(1) + .withPartition(BinaryRow.EMPTY_ROW) + .withBucket(1) + .withBucketPath("my path") + .rawConvertible(rawConvertible) + .withDataFiles(dataFiles); + if (deletionFiles != null) { + builder.withDataDeletionFiles(deletionFiles); + } + return builder.build(); + } } diff --git a/paimon-core/src/test/resources/compatibility/datasplit-v3 b/paimon-core/src/test/resources/compatibility/datasplit-v3 new file mode 100644 index 000000000000..6b19fe2d958d Binary files /dev/null and b/paimon-core/src/test/resources/compatibility/datasplit-v3 differ diff --git a/paimon-core/src/test/resources/compatibility/manifest-committable-v4 b/paimon-core/src/test/resources/compatibility/manifest-committable-v4 new file mode 100644 index 000000000000..9c095669a34b Binary files /dev/null and b/paimon-core/src/test/resources/compatibility/manifest-committable-v4 differ diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/source/DataTableSource.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/source/DataTableSource.java index ee00d41832cd..f41f8da6c820 100644 --- a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/source/DataTableSource.java +++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/source/DataTableSource.java @@ -51,7 +51,7 @@ public DataTableSource( null, null, null, - false); + null); } public DataTableSource( @@ -64,7 +64,7 @@ public DataTableSource( @Nullable int[][] projectFields, @Nullable Long limit, @Nullable WatermarkStrategy watermarkStrategy, - boolean isBatchCountStar) { + @Nullable Long countPushed) { super( tableIdentifier, table, @@ -75,7 +75,7 @@ public DataTableSource( projectFields, limit, watermarkStrategy, - isBatchCountStar); + countPushed); } @Override @@ -90,7 +90,7 @@ public DataTableSource copy() { projectFields, limit, watermarkStrategy, - isBatchCountStar); + countPushed); } @Override diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/PurgeFilesProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/PurgeFilesProcedure.java new file mode 100644 index 000000000000..3053eae3c7cf --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/PurgeFilesProcedure.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.procedure; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.Table; + +import org.apache.flink.table.procedure.ProcedureContext; + +import java.io.IOException; +import java.util.Arrays; + +/** A procedure to purge files for a table. */ +public class PurgeFilesProcedure extends ProcedureBase { + public static final String IDENTIFIER = "purge_files"; + + public String[] call(ProcedureContext procedureContext, String tableId) + throws Catalog.TableNotExistException { + Table table = catalog.getTable(Identifier.fromString(tableId)); + FileStoreTable fileStoreTable = (FileStoreTable) table; + FileIO fileIO = fileStoreTable.fileIO(); + Path tablePath = fileStoreTable.snapshotManager().tablePath(); + try { + Arrays.stream(fileIO.listStatus(tablePath)) + .filter(f -> !f.getPath().getName().contains("schema")) + .forEach( + fileStatus -> { + try { + fileIO.delete(fileStatus.getPath(), true); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } catch (IOException e) { + throw new RuntimeException(e); + } + return new String[] { + String.format("Success purge files with table: %s.", fileStoreTable.name()) + }; + } + + @Override + public String identifier() { + return IDENTIFIER; + } +} diff --git a/paimon-flink/paimon-flink-1.20/pom.xml b/paimon-flink/paimon-flink-1.20/pom.xml index 7cf1d8e98df7..f15792d2bea9 100644 --- a/paimon-flink/paimon-flink-1.20/pom.xml +++ b/paimon-flink/paimon-flink-1.20/pom.xml @@ -55,6 +55,20 @@ under the License. + + + org.apache.flink + flink-streaming-java + ${flink.version} + provided + + + + org.apache.flink + flink-table-common + ${flink.version} + provided + diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java index 83891c90b8e1..6482a625f4c7 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java @@ -42,7 +42,7 @@ import static org.apache.paimon.flink.action.MultiTablesSinkMode.DIVIDED; import static org.apache.paimon.utils.Preconditions.checkArgument; import static org.apache.paimon.utils.Preconditions.checkState; -import static org.apache.paimon.utils.StringUtils.caseSensitiveConversion; +import static org.apache.paimon.utils.StringUtils.toLowerCaseIfNeed; /** Common utils for CDC Action. */ public class CdcActionCommonUtils { @@ -56,6 +56,8 @@ public class CdcActionCommonUtils { public static final String PULSAR_CONF = "pulsar_conf"; public static final String TABLE_PREFIX = "table_prefix"; public static final String TABLE_SUFFIX = "table_suffix"; + public static final String TABLE_PREFIX_DB = "table_prefix_db"; + public static final String TABLE_SUFFIX_DB = "table_suffix_db"; public static final String TABLE_MAPPING = "table_mapping"; public static final String INCLUDING_TABLES = "including_tables"; public static final String EXCLUDING_TABLES = "excluding_tables"; @@ -127,21 +129,21 @@ public static Schema buildPaimonSchema( List allFieldNames = new ArrayList<>(); for (DataField field : sourceSchema.fields()) { - String fieldName = caseSensitiveConversion(field.name(), caseSensitive); + String fieldName = toLowerCaseIfNeed(field.name(), caseSensitive); allFieldNames.add(fieldName); builder.column(fieldName, field.type(), field.description()); } for (ComputedColumn computedColumn : computedColumns) { String computedColumnName = - caseSensitiveConversion(computedColumn.columnName(), caseSensitive); + toLowerCaseIfNeed(computedColumn.columnName(), caseSensitive); allFieldNames.add(computedColumnName); builder.column(computedColumnName, computedColumn.columnType()); } for (CdcMetadataConverter metadataConverter : metadataConverters) { String metadataColumnName = - caseSensitiveConversion(metadataConverter.columnName(), caseSensitive); + toLowerCaseIfNeed(metadataConverter.columnName(), caseSensitive); allFieldNames.add(metadataColumnName); builder.column(metadataColumnName, metadataConverter.dataType()); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/Expression.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/Expression.java index 2e0a1319293f..3290ec18291f 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/Expression.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/Expression.java @@ -189,7 +189,7 @@ public static ReferencedField checkArgument( String[] literals = Arrays.stream(args).skip(1).map(String::trim).toArray(String[]::new); String referencedFieldCheckForm = - StringUtils.caseSensitiveConversion(referencedField, caseSensitive); + StringUtils.toLowerCaseIfNeed(referencedField, caseSensitive); DataType fieldType = checkNotNull( diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSyncTableActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSyncTableActionBase.java index 9c629b5a516f..3af0957ce6da 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSyncTableActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/MessageQueueSyncTableActionBase.java @@ -86,7 +86,7 @@ protected Schema buildPaimonSchema(Schema retrievedSchema) { tableConfig, retrievedSchema, metadataConverters, - allowUpperCase, + caseSensitive, true, false); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java index ac3483ac23bf..56334c1e7bff 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.action.cdc; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.flink.action.Action; import org.apache.paimon.flink.action.MultiTablesSinkMode; import org.apache.paimon.flink.sink.cdc.EventParser; @@ -53,6 +54,8 @@ public abstract class SyncDatabaseActionBase extends SynchronizationActionBase { protected String tablePrefix = ""; protected String tableSuffix = ""; protected Map tableMapping = new HashMap<>(); + protected Map dbPrefix = new HashMap<>(); + protected Map dbSuffix = new HashMap<>(); protected String includingTables = ".*"; protected List partitionKeys = new ArrayList<>(); protected List primaryKeys = new ArrayList<>(); @@ -98,6 +101,30 @@ public SyncDatabaseActionBase withTableSuffix(@Nullable String tableSuffix) { return this; } + public SyncDatabaseActionBase withDbPrefix(Map dbPrefix) { + if (dbPrefix != null) { + this.dbPrefix = + dbPrefix.entrySet().stream() + .collect( + HashMap::new, + (m, e) -> m.put(e.getKey().toLowerCase(), e.getValue()), + HashMap::putAll); + } + return this; + } + + public SyncDatabaseActionBase withDbSuffix(Map dbSuffix) { + if (dbSuffix != null) { + this.dbSuffix = + dbSuffix.entrySet().stream() + .collect( + HashMap::new, + (m, e) -> m.put(e.getKey().toLowerCase(), e.getValue()), + HashMap::putAll); + } + return this; + } + public SyncDatabaseActionBase withTableMapping(Map tableMapping) { if (tableMapping != null) { this.tableMapping = tableMapping; @@ -129,9 +156,9 @@ public SyncDatabaseActionBase withPrimaryKeys(String... primaryKeys) { @Override protected void validateCaseSensitivity() { - Catalog.validateCaseInsensitive(allowUpperCase, "Database", database); - Catalog.validateCaseInsensitive(allowUpperCase, "Table prefix", tablePrefix); - Catalog.validateCaseInsensitive(allowUpperCase, "Table suffix", tableSuffix); + CatalogUtils.validateCaseInsensitive(caseSensitive, "Database", database); + CatalogUtils.validateCaseInsensitive(caseSensitive, "Table prefix", tablePrefix); + CatalogUtils.validateCaseInsensitive(caseSensitive, "Table suffix", tableSuffix); } @Override @@ -153,7 +180,7 @@ protected EventParser.Factory buildEventParserFactory() NewTableSchemaBuilder schemaBuilder = new NewTableSchemaBuilder( tableConfig, - allowUpperCase, + caseSensitive, partitionKeys, primaryKeys, requirePrimaryKeys(), @@ -164,7 +191,13 @@ protected EventParser.Factory buildEventParserFactory() excludingTables == null ? null : Pattern.compile(excludingTables); TableNameConverter tableNameConverter = new TableNameConverter( - allowUpperCase, mergeShards, tablePrefix, tableSuffix, tableMapping); + caseSensitive, + mergeShards, + dbPrefix, + dbSuffix, + tablePrefix, + tableSuffix, + tableMapping); Set createdTables; try { createdTables = new HashSet<>(catalog.listTables(database)); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java index 2135f2a28112..d497b588c2af 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java @@ -31,7 +31,9 @@ import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.PRIMARY_KEYS; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TABLE_MAPPING; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TABLE_PREFIX; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TABLE_PREFIX_DB; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TABLE_SUFFIX; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TABLE_SUFFIX_DB; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TYPE_MAPPING; /** Base {@link ActionFactory} for synchronizing into database. */ @@ -52,6 +54,8 @@ public Optional create(MultipleParameterToolAdapter params) { protected void withParams(MultipleParameterToolAdapter params, T action) { action.withTablePrefix(params.get(TABLE_PREFIX)) .withTableSuffix(params.get(TABLE_SUFFIX)) + .withDbPrefix(optionalConfigMap(params, TABLE_PREFIX_DB)) + .withDbSuffix(optionalConfigMap(params, TABLE_SUFFIX_DB)) .withTableMapping(optionalConfigMap(params, TABLE_MAPPING)) .includingTables(params.get(INCLUDING_TABLES)) .excludingTables(params.get(EXCLUDING_TABLES)) diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java index 87efeb2a19cf..6fcdbd44bca2 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java @@ -20,6 +20,7 @@ import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.flink.action.Action; @@ -107,15 +108,15 @@ protected Schema buildPaimonSchema(Schema retrievedSchema) { tableConfig, retrievedSchema, metadataConverters, - allowUpperCase, + caseSensitive, true, true); } @Override protected void validateCaseSensitivity() { - Catalog.validateCaseInsensitive(allowUpperCase, "Database", database); - Catalog.validateCaseInsensitive(allowUpperCase, "Table", table); + CatalogUtils.validateCaseInsensitive(caseSensitive, "Database", database); + CatalogUtils.validateCaseInsensitive(caseSensitive, "Table", table); } @Override @@ -142,7 +143,7 @@ protected void beforeBuildingSourceSink() throws Exception { buildComputedColumns( computedColumnArgs, fileStoreTable.schema().fields(), - allowUpperCase); + caseSensitive); // check partition keys and primary keys in case that user specified them checkConstraints(); } @@ -162,7 +163,7 @@ protected FlatMapFunction recordParse() @Override protected EventParser.Factory buildEventParserFactory() { - boolean caseSensitive = this.allowUpperCase; + boolean caseSensitive = this.caseSensitive; return () -> new RichCdcMultiplexRecordEventParser(caseSensitive); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java index f103396389e5..d755b200a957 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java @@ -40,7 +40,6 @@ import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.functions.source.SourceFunction; import java.time.Duration; import java.util.HashMap; @@ -65,7 +64,7 @@ public abstract class SynchronizationActionBase extends ActionBase { protected final String database; protected final Configuration cdcSourceConfig; protected final SyncJobHandler syncJobHandler; - protected final boolean allowUpperCase; + protected final boolean caseSensitive; protected Map tableConfig = new HashMap<>(); protected TypeMapping typeMapping = TypeMapping.defaultMapping(); @@ -81,7 +80,7 @@ public SynchronizationActionBase( this.database = database; this.cdcSourceConfig = Configuration.fromMap(cdcSourceConfig); this.syncJobHandler = syncJobHandler; - this.allowUpperCase = catalog.allowUpperCase(); + this.caseSensitive = catalog.caseSensitive(); this.syncJobHandler.registerJdbcDriver(); } @@ -131,7 +130,7 @@ public void build() throws Exception { protected void beforeBuildingSourceSink() throws Exception {} - protected Object buildSource() { + protected Source buildSource() { return syncJobHandler.provideSource(); } @@ -147,41 +146,32 @@ protected void validateRuntimeExecutionMode() { "It's only support STREAMING mode for flink-cdc sync table action."); } - private DataStreamSource buildDataStreamSource(Object source) { - if (source instanceof Source) { - boolean isAutomaticWatermarkCreationEnabled = - tableConfig.containsKey(CoreOptions.TAG_AUTOMATIC_CREATION.key()) - && Objects.equals( - tableConfig.get(CoreOptions.TAG_AUTOMATIC_CREATION.key()), - WATERMARK.toString()); - - Options options = Options.fromMap(tableConfig); - Duration idleTimeout = options.get(SCAN_WATERMARK_IDLE_TIMEOUT); - String watermarkAlignGroup = options.get(SCAN_WATERMARK_ALIGNMENT_GROUP); - WatermarkStrategy watermarkStrategy = - isAutomaticWatermarkCreationEnabled - ? watermarkAlignGroup != null - ? new CdcWatermarkStrategy(createCdcTimestampExtractor()) - .withWatermarkAlignment( - watermarkAlignGroup, - options.get(SCAN_WATERMARK_ALIGNMENT_MAX_DRIFT), - options.get( - SCAN_WATERMARK_ALIGNMENT_UPDATE_INTERVAL)) - : new CdcWatermarkStrategy(createCdcTimestampExtractor()) - : WatermarkStrategy.noWatermarks(); - if (idleTimeout != null) { - watermarkStrategy = watermarkStrategy.withIdleness(idleTimeout); - } - return env.fromSource( - (Source) source, - watermarkStrategy, - syncJobHandler.provideSourceName()); + private DataStreamSource buildDataStreamSource( + Source source) { + boolean isAutomaticWatermarkCreationEnabled = + tableConfig.containsKey(CoreOptions.TAG_AUTOMATIC_CREATION.key()) + && Objects.equals( + tableConfig.get(CoreOptions.TAG_AUTOMATIC_CREATION.key()), + WATERMARK.toString()); + + Options options = Options.fromMap(tableConfig); + Duration idleTimeout = options.get(SCAN_WATERMARK_IDLE_TIMEOUT); + String watermarkAlignGroup = options.get(SCAN_WATERMARK_ALIGNMENT_GROUP); + WatermarkStrategy watermarkStrategy = + isAutomaticWatermarkCreationEnabled + ? watermarkAlignGroup != null + ? new CdcWatermarkStrategy(createCdcTimestampExtractor()) + .withWatermarkAlignment( + watermarkAlignGroup, + options.get(SCAN_WATERMARK_ALIGNMENT_MAX_DRIFT), + options.get( + SCAN_WATERMARK_ALIGNMENT_UPDATE_INTERVAL)) + : new CdcWatermarkStrategy(createCdcTimestampExtractor()) + : WatermarkStrategy.noWatermarks(); + if (idleTimeout != null) { + watermarkStrategy = watermarkStrategy.withIdleness(idleTimeout); } - if (source instanceof SourceFunction) { - return env.addSource( - (SourceFunction) source, syncJobHandler.provideSourceName()); - } - throw new UnsupportedOperationException("Unrecognized source type"); + return env.fromSource(source, watermarkStrategy, syncJobHandler.provideSourceName()); } protected abstract FlatMapFunction recordParse(); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/TableNameConverter.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/TableNameConverter.java index 4eca8b903ed1..7dd63ed2273e 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/TableNameConverter.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/TableNameConverter.java @@ -24,6 +24,8 @@ import java.util.HashMap; import java.util.Map; +import static org.apache.paimon.utils.StringUtils.toLowerCaseIfNeed; + /** Used to convert a MySQL source table name to corresponding Paimon table name. */ public class TableNameConverter implements Serializable { @@ -31,6 +33,8 @@ public class TableNameConverter implements Serializable { private final boolean caseSensitive; private final boolean mergeShards; + private final Map dbPrefix; + private final Map dbSuffix; private final String prefix; private final String suffix; private final Map tableMapping; @@ -45,21 +49,54 @@ public TableNameConverter( String prefix, String suffix, Map tableMapping) { + this( + caseSensitive, + mergeShards, + new HashMap<>(), + new HashMap<>(), + prefix, + suffix, + tableMapping); + } + + public TableNameConverter( + boolean caseSensitive, + boolean mergeShards, + Map dbPrefix, + Map dbSuffix, + String prefix, + String suffix, + Map tableMapping) { this.caseSensitive = caseSensitive; this.mergeShards = mergeShards; + this.dbPrefix = dbPrefix; + this.dbSuffix = dbSuffix; this.prefix = prefix; this.suffix = suffix; this.tableMapping = lowerMapKey(tableMapping); } - public String convert(String originName) { - if (tableMapping.containsKey(originName.toLowerCase())) { - String mappedName = tableMapping.get(originName.toLowerCase()); - return caseSensitive ? mappedName : mappedName.toLowerCase(); + public String convert(String originDbName, String originTblName) { + // top priority: table mapping + if (tableMapping.containsKey(originTblName.toLowerCase())) { + String mappedName = tableMapping.get(originTblName.toLowerCase()); + return toLowerCaseIfNeed(mappedName, caseSensitive); + } + + String tblPrefix = prefix; + String tblSuffix = suffix; + + // second priority: prefix and postfix specified by db + if (dbPrefix.containsKey(originDbName.toLowerCase())) { + tblPrefix = dbPrefix.get(originDbName.toLowerCase()); + } + if (dbSuffix.containsKey(originDbName.toLowerCase())) { + tblSuffix = dbSuffix.get(originDbName.toLowerCase()); } - String tableName = caseSensitive ? originName : originName.toLowerCase(); - return prefix + tableName + suffix; + // third priority: normal prefix and suffix + String tableName = toLowerCaseIfNeed(originTblName, caseSensitive); + return tblPrefix + tableName + tblSuffix; } public String convert(Identifier originIdentifier) { @@ -69,7 +106,7 @@ public String convert(Identifier originIdentifier) { : originIdentifier.getDatabaseName() + "_" + originIdentifier.getObjectName(); - return convert(rawName); + return convert(originIdentifier.getDatabaseName(), rawName); } private Map lowerMapKey(Map map) { diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java index 235b3f9a3235..0f452e2834be 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java @@ -139,11 +139,12 @@ protected void beforeBuildingSourceSink() throws Exception { TableNameConverter tableNameConverter = new TableNameConverter( - allowUpperCase, mergeShards, tablePrefix, tableSuffix, tableMapping); + caseSensitive, mergeShards, tablePrefix, tableSuffix, tableMapping); for (JdbcTableInfo tableInfo : jdbcTableInfos) { Identifier identifier = Identifier.create( - database, tableNameConverter.convert(tableInfo.toPaimonTableName())); + database, + tableNameConverter.convert("", tableInfo.toPaimonTableName())); FileStoreTable table; Schema fromMySql = CdcActionCommonUtils.buildPaimonSchema( @@ -154,7 +155,7 @@ protected void beforeBuildingSourceSink() throws Exception { tableConfig, tableInfo.schema(), metadataConverters, - allowUpperCase, + caseSensitive, false, true); try { diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CaseSensitiveUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CaseSensitiveUtils.java index 4892aee03024..e80692ed22f5 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CaseSensitiveUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CaseSensitiveUtils.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.functions.ProcessFunction; @@ -28,8 +29,8 @@ public class CaseSensitiveUtils { public static DataStream cdcRecordConvert( - Catalog.Loader catalogLoader, DataStream input) { - if (allowUpperCase(catalogLoader)) { + CatalogLoader catalogLoader, DataStream input) { + if (caseSensitive(catalogLoader)) { return input; } @@ -46,8 +47,8 @@ public void processElement( } public static DataStream cdcMultiplexRecordConvert( - Catalog.Loader catalogLoader, DataStream input) { - if (allowUpperCase(catalogLoader)) { + CatalogLoader catalogLoader, DataStream input) { + if (caseSensitive(catalogLoader)) { return input; } @@ -65,9 +66,9 @@ public void processElement( .name("Case-insensitive Convert"); } - private static boolean allowUpperCase(Catalog.Loader catalogLoader) { + private static boolean caseSensitive(CatalogLoader catalogLoader) { try (Catalog catalog = catalogLoader.load()) { - return catalog.allowUpperCase(); + return catalog.caseSensitive(); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java index 886e33e2046a..4efcf1207e9f 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.types.DataField; @@ -62,13 +63,13 @@ public class CdcDynamicTableParsingProcessFunction extends ProcessFunction parserFactory; private final String database; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; private transient EventParser parser; private transient Catalog catalog; public CdcDynamicTableParsingProcessFunction( - String database, Catalog.Loader catalogLoader, EventParser.Factory parserFactory) { + String database, CatalogLoader catalogLoader, EventParser.Factory parserFactory) { // for now, only support single database this.database = database; this.catalogLoader = catalogLoader; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputer.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputer.java index fdad3a921d63..2858b2d4eb6b 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputer.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputer.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; @@ -38,13 +39,13 @@ public class CdcMultiplexRecordChannelComputer implements ChannelComputer channelComputers; - public CdcMultiplexRecordChannelComputer(Catalog.Loader catalogLoader) { + public CdcMultiplexRecordChannelComputer(CatalogLoader catalogLoader) { this.catalogLoader = catalogLoader; } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java index 5db111a30047..9387a8293874 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java @@ -20,6 +20,7 @@ import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.GenericRow; import org.apache.paimon.flink.sink.MultiTableCommittable; @@ -67,7 +68,7 @@ public class CdcRecordStoreMultiWriteOperator private final StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider; private final String initialCommitUser; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; private MemoryPoolFactory memoryPoolFactory; private Catalog catalog; @@ -79,7 +80,7 @@ public class CdcRecordStoreMultiWriteOperator private CdcRecordStoreMultiWriteOperator( StreamOperatorParameters parameters, - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider, String initialCommitUser, Options options) { @@ -264,10 +265,10 @@ public static class Factory extends PrepareCommitOperator.Factory { private final StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider; private final String initialCommitUser; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; public Factory( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider, String initialCommitUser, Options options) { diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcSinkBuilder.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcSinkBuilder.java index 28b68fedc3e6..5c27db6ddf1b 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcSinkBuilder.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcSinkBuilder.java @@ -19,7 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.annotation.Experimental; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.utils.SingleOutputStreamOperatorUtils; import org.apache.paimon.schema.SchemaManager; @@ -48,7 +48,7 @@ public class CdcSinkBuilder { private EventParser.Factory parserFactory = null; private Table table = null; private Identifier identifier = null; - private Catalog.Loader catalogLoader = null; + private CatalogLoader catalogLoader = null; @Nullable private Integer parallelism; @@ -77,7 +77,7 @@ public CdcSinkBuilder withIdentifier(Identifier identifier) { return this; } - public CdcSinkBuilder withCatalogLoader(Catalog.Loader catalogLoader) { + public CdcSinkBuilder withCatalogLoader(CatalogLoader catalogLoader) { this.catalogLoader = catalogLoader; return this; } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java index 1688d4deb088..4cd9235cb58a 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.sink.cdc; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.flink.sink.CommittableStateManager; import org.apache.paimon.flink.sink.Committer; import org.apache.paimon.flink.sink.CommitterOperatorFactory; @@ -60,13 +60,13 @@ public class FlinkCdcMultiTableSink implements Serializable { private static final String GLOBAL_COMMITTER_NAME = "Multiplex Global Committer"; private final boolean isOverwrite = false; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; private final double commitCpuCores; @Nullable private final MemorySize commitHeapMemory; private final String commitUser; public FlinkCdcMultiTableSink( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, double commitCpuCores, @Nullable MemorySize commitHeapMemory, String commitUser) { diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java index a9ad66847b4b..bd18c7e7ad82 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.sink.cdc; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.flink.action.MultiTablesSinkMode; @@ -72,7 +72,7 @@ public class FlinkCdcSyncDatabaseSinkBuilder { // it will check newly added tables and create the corresponding // Paimon tables. 2) in multiplex sink where it is used to // initialize different writers to multiple tables. - private Catalog.Loader catalogLoader; + private CatalogLoader catalogLoader; // database to sync, currently only support single database private String database; private MultiTablesSinkMode mode; @@ -111,7 +111,7 @@ public FlinkCdcSyncDatabaseSinkBuilder withDatabase(String database) { return this; } - public FlinkCdcSyncDatabaseSinkBuilder withCatalogLoader(Catalog.Loader catalogLoader) { + public FlinkCdcSyncDatabaseSinkBuilder withCatalogLoader(CatalogLoader catalogLoader) { this.catalogLoader = catalogLoader; return this; } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java index 0ad412e47d34..dd612a52c2eb 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.SchemaManager; @@ -51,7 +52,7 @@ public class MultiTableUpdatedDataFieldsProcessFunction private final Map schemaManagers = new HashMap<>(); - public MultiTableUpdatedDataFieldsProcessFunction(Catalog.Loader catalogLoader) { + public MultiTableUpdatedDataFieldsProcessFunction(CatalogLoader catalogLoader) { super(catalogLoader); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcSinkBuilder.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcSinkBuilder.java index 610856d3af54..43f63854bb22 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcSinkBuilder.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcSinkBuilder.java @@ -19,7 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.annotation.Public; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.table.Table; @@ -39,7 +39,7 @@ public class RichCdcSinkBuilder { private DataStream input = null; private Table table = null; private Identifier identifier = null; - private Catalog.Loader catalogLoader = null; + private CatalogLoader catalogLoader = null; @Nullable private Integer parallelism; @@ -62,7 +62,7 @@ public RichCdcSinkBuilder parallelism(@Nullable Integer parallelism) { return this; } - public RichCdcSinkBuilder catalogLoader(Catalog.Loader catalogLoader) { + public RichCdcSinkBuilder catalogLoader(CatalogLoader catalogLoader) { this.catalogLoader = catalogLoader; return this; } @@ -114,7 +114,7 @@ public RichCdcSinkBuilder withIdentifier(Identifier identifier) { /** @deprecated Use {@link #catalogLoader}. */ @Deprecated - public RichCdcSinkBuilder withCatalogLoader(Catalog.Loader catalogLoader) { + public RichCdcSinkBuilder withCatalogLoader(CatalogLoader catalogLoader) { this.catalogLoader = catalogLoader; return this; } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java index 64f00d96b0f5..504f63105801 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.sink.cdc; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.SchemaManager; @@ -53,7 +53,7 @@ public class UpdatedDataFieldsProcessFunction private Set latestFields; public UpdatedDataFieldsProcessFunction( - SchemaManager schemaManager, Identifier identifier, Catalog.Loader catalogLoader) { + SchemaManager schemaManager, Identifier identifier, CatalogLoader catalogLoader) { super(catalogLoader); this.schemaManager = schemaManager; this.identifier = identifier; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java index 4f02b784c2ba..d50df23742aa 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.SchemaManager; @@ -48,9 +49,9 @@ public abstract class UpdatedDataFieldsProcessFunctionBase extends Process private static final Logger LOG = LoggerFactory.getLogger(UpdatedDataFieldsProcessFunctionBase.class); - protected final Catalog.Loader catalogLoader; + protected final CatalogLoader catalogLoader; protected Catalog catalog; - private boolean allowUpperCase; + private boolean caseSensitive; private static final List STRING_TYPES = Arrays.asList(DataTypeRoot.CHAR, DataTypeRoot.VARCHAR); @@ -70,7 +71,7 @@ public abstract class UpdatedDataFieldsProcessFunctionBase extends Process private static final List TIMESTAMP_TYPES = Arrays.asList(DataTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE); - protected UpdatedDataFieldsProcessFunctionBase(Catalog.Loader catalogLoader) { + protected UpdatedDataFieldsProcessFunctionBase(CatalogLoader catalogLoader) { this.catalogLoader = catalogLoader; } @@ -86,7 +87,7 @@ public void open(OpenContext openContext) throws Exception { */ public void open(Configuration parameters) { this.catalog = catalogLoader.load(); - this.allowUpperCase = this.catalog.allowUpperCase(); + this.caseSensitive = this.catalog.caseSensitive(); } protected void applySchemaChange( @@ -215,8 +216,7 @@ protected List extractSchemaChanges( List result = new ArrayList<>(); for (DataField newField : updatedDataFields) { - String newFieldName = - StringUtils.caseSensitiveConversion(newField.name(), allowUpperCase); + String newFieldName = StringUtils.toLowerCaseIfNeed(newField.name(), caseSensitive); if (oldFields.containsKey(newFieldName)) { DataField oldField = oldFields.get(newFieldName); // we compare by ignoring nullable, because partition keys and primary keys might be diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java index 9ba18376867f..46c8e98fb639 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java @@ -19,7 +19,7 @@ package org.apache.paimon.flink.action.cdc; import org.apache.paimon.CoreOptions; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkCatalogFactory; import org.apache.paimon.flink.sink.cdc.UpdatedDataFieldsProcessFunction; @@ -202,7 +202,7 @@ public void testSchemaEvolution() throws Exception { DataStream> upDataFieldStream = env.fromCollection(prepareData()); Options options = new Options(); options.set("warehouse", tempPath.toString()); - final Catalog.Loader catalogLoader = () -> FlinkCatalogFactory.createPaimonCatalog(options); + final CatalogLoader catalogLoader = () -> FlinkCatalogFactory.createPaimonCatalog(options); Identifier identifier = Identifier.create(database, tableName); DataStream schemaChangeProcessFunction = upDataFieldStream diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/TableNameConverterTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/TableNameConverterTest.java index dfbe32e3d398..89bbadfeb8c8 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/TableNameConverterTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/TableNameConverterTest.java @@ -33,13 +33,47 @@ public void testConvertTableName() { tableMapping.put("mapped_src", "mapped_TGT"); TableNameConverter caseConverter = new TableNameConverter(true, true, "pre_", "_pos", tableMapping); - Assert.assertEquals(caseConverter.convert("mapped_SRC"), "mapped_TGT"); + Assert.assertEquals(caseConverter.convert("", "mapped_SRC"), "mapped_TGT"); - Assert.assertEquals(caseConverter.convert("unmapped_src"), "pre_unmapped_src_pos"); + Assert.assertEquals(caseConverter.convert("", "unmapped_src"), "pre_unmapped_src_pos"); TableNameConverter noCaseConverter = new TableNameConverter(false, true, "pre_", "_pos", tableMapping); - Assert.assertEquals(noCaseConverter.convert("mapped_src"), "mapped_tgt"); - Assert.assertEquals(noCaseConverter.convert("unmapped_src"), "pre_unmapped_src_pos"); + Assert.assertEquals(noCaseConverter.convert("", "mapped_src"), "mapped_tgt"); + Assert.assertEquals(noCaseConverter.convert("", "unmapped_src"), "pre_unmapped_src_pos"); + } + + @Test + public void testConvertTableNameByDBPrefix_Suffix() { + Map dbPrefix = new HashMap<>(2); + dbPrefix.put("db_with_prefix", "db_pref_"); + dbPrefix.put("db_with_prefix_suffix", "db_pref_"); + + Map dbSuffix = new HashMap<>(2); + dbSuffix.put("db_with_suffix", "_db_suff"); + dbSuffix.put("db_with_prefix_suffix", "_db_suff"); + + TableNameConverter tblNameConverter = + new TableNameConverter(false, true, dbPrefix, dbSuffix, "pre_", "_suf", null); + + // Tables in the specified db should have the specified prefix and suffix. + + // db prefix + normal suffix + Assert.assertEquals( + "db_pref_table_name_suf", tblNameConverter.convert("db_with_prefix", "table_name")); + + // normal prefix + db suffix + Assert.assertEquals( + "pre_table_name_db_suff", tblNameConverter.convert("db_with_suffix", "table_name")); + + // db prefix + db suffix + Assert.assertEquals( + "db_pref_table_name_db_suff", + tblNameConverter.convert("db_with_prefix_suffix", "table_name")); + + // only normal prefix and suffix + Assert.assertEquals( + "pre_table_name_suf", + tblNameConverter.convert("db_without_prefix_suffix", "table_name")); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java index 0e1f4e72ea8c..6e37c589ac92 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.action.cdc.kafka; -import org.apache.paimon.catalog.FileSystemCatalogOptions; +import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -535,7 +535,7 @@ public void testCaseInsensitive() throws Exception { .withTableConfig(getBasicTableConfig()) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .build(); runActionWithDefaultEnv(action); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java index 8a4dc2f3035b..ed1885f5d774 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncTableActionITCase.java @@ -19,7 +19,7 @@ package org.apache.paimon.flink.action.cdc.kafka; import org.apache.paimon.CoreOptions; -import org.apache.paimon.catalog.FileSystemCatalogOptions; +import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -1121,7 +1121,7 @@ public void testComputedColumnWithCaseInsensitive(boolean triggerSchemaRetrieval .withTableConfig(getBasicTableConfig()) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .withComputedColumnArgs("_YEAR=year(_DATE)") .build(); runActionWithDefaultEnv(action); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java index de189bc20536..606c46e90e4a 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.action.cdc.kafka; -import org.apache.paimon.catalog.FileSystemCatalogOptions; +import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -475,7 +475,7 @@ protected void testCaseInsensitive(String format) throws Exception { .withTableConfig(getBasicTableConfig()) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .build(); runActionWithDefaultEnv(action); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseActionITCase.java index ae0b0b412ab4..92c2a7243a7c 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseActionITCase.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.action.cdc.mongodb; -import org.apache.paimon.catalog.FileSystemCatalogOptions; +import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -196,7 +196,7 @@ public void testDynamicTableCreationInMongoDB() throws Exception { .withTableConfig(getBasicTableConfig()) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .build(); runActionWithDefaultEnv(action); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java index b4f31f2d6d3d..2d8489dc23df 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.action.cdc.mongodb; -import org.apache.paimon.catalog.FileSystemCatalogOptions; +import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -382,7 +382,7 @@ public void testComputedColumnWithCaseInsensitive() throws Exception { .withTableConfig(getBasicTableConfig()) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .withComputedColumnArgs("_YEAR=year(_DATE)") .build(); runActionWithDefaultEnv(action); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java index b48b898d66e3..10ee548125bc 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java @@ -19,7 +19,6 @@ package org.apache.paimon.flink.action.cdc.mysql; import org.apache.paimon.CoreOptions; -import org.apache.paimon.catalog.FileSystemCatalogOptions; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.action.MultiTablesSinkMode; import org.apache.paimon.options.CatalogOptions; @@ -475,7 +474,7 @@ public void testIgnoreCaseDivided() throws Exception { syncDatabaseActionBuilder(mySqlConfig) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .withTableConfig(getBasicTableConfig()) .build(); runActionWithDefaultEnv(action); @@ -496,7 +495,7 @@ public void testIgnoreCaseCombined() throws Exception { syncDatabaseActionBuilder(mySqlConfig) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .withMode(COMBINED.configString()) .withTableConfig(getBasicTableConfig()) .build(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java index febbe4e1deaa..749d87eb0636 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java @@ -19,7 +19,6 @@ package org.apache.paimon.flink.action.cdc.mysql; import org.apache.paimon.CoreOptions; -import org.apache.paimon.catalog.FileSystemCatalogOptions; import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.SchemaManager; @@ -1327,7 +1326,7 @@ public void testComputedColumnWithCaseInsensitive() throws Exception { syncTableActionBuilder(mySqlConfig) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .withComputedColumnArgs("SUBSTRING=substring(UPPERCASE_STRING,2)") .build(); runActionWithDefaultEnv(action); @@ -1363,7 +1362,7 @@ public void testSpecifyKeysWithCaseInsensitive() throws Exception { syncTableActionBuilder(mySqlConfig) .withCatalogConfig( Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + CatalogOptions.CASE_SENSITIVE.key(), "false")) .withPrimaryKeys("ID1", "PART") .withPartitionKeys("PART") .build(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputerTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputerTest.java index 867cbdbae002..43b7d2ba6399 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputerTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputerTest.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.fs.Path; import org.apache.paimon.options.CatalogOptions; @@ -54,7 +55,7 @@ public class CdcMultiplexRecordChannelComputerTest { @TempDir java.nio.file.Path tempDir; - private Catalog.Loader catalogLoader; + private CatalogLoader catalogLoader; private Path warehouse; private String databaseName; private Identifier tableWithPartition; diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java index 9f35b25026bb..4436aa392d42 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.sink.MultiTableCommittable; import org.apache.paimon.flink.sink.MultiTableCommittableTypeInfo; @@ -82,7 +83,7 @@ public class CdcRecordStoreMultiWriteOperatorTest { private Identifier firstTable; private Catalog catalog; private Identifier secondTable; - private Catalog.Loader catalogLoader; + private CatalogLoader catalogLoader; private Schema firstTableSchema; @BeforeEach @@ -340,7 +341,7 @@ public void testSingleTableAddColumn() throws Exception { harness.close(); } - private Catalog.Loader createCatalogLoader() { + private CatalogLoader createCatalogLoader() { Options catalogOptions = createCatalogOptions(warehouse); return () -> CatalogFactory.createCatalog(CatalogContext.create(catalogOptions)); } @@ -688,7 +689,7 @@ public void testUsingTheSameCompactExecutor() throws Exception { } private OneInputStreamOperatorTestHarness - createTestHarness(Catalog.Loader catalogLoader) throws Exception { + createTestHarness(CatalogLoader catalogLoader) throws Exception { CdcRecordStoreMultiWriteOperator.Factory operatorFactory = new CdcRecordStoreMultiWriteOperator.Factory( catalogLoader, diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java index 723f57a30e3f..ab81e37c7d04 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java @@ -26,7 +26,6 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.junit.jupiter.api.Test; @@ -45,14 +44,7 @@ public void testTransformationParallelism() { env.setParallelism(8); int inputParallelism = ThreadLocalRandom.current().nextInt(8) + 1; DataStreamSource input = - env.addSource( - new ParallelSourceFunction() { - @Override - public void run(SourceContext ctx) {} - - @Override - public void cancel() {} - }) + env.fromData(CdcMultiplexRecord.class, new CdcMultiplexRecord("", "", null)) .setParallelism(inputParallelism); FlinkCdcMultiTableSink sink = diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java index a7c6b2cb6323..35286e3a88d4 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java @@ -19,7 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.CoreOptions; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.data.InternalRow; import org.apache.paimon.flink.FlinkCatalogFactory; @@ -42,6 +42,7 @@ import org.apache.paimon.utils.FailingFileIO; import org.apache.paimon.utils.TraceableFileIO; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.Test; @@ -154,14 +155,14 @@ private void innerTestRandomCdcEvents(Supplier bucket, boolean unawareB .allowRestart(enableFailure) .build(); - TestCdcSourceFunction sourceFunction = new TestCdcSourceFunction(events); - DataStreamSource source = env.addSource(sourceFunction); + TestCdcSource testCdcSource = new TestCdcSource(events); + DataStreamSource source = + env.fromSource(testCdcSource, WatermarkStrategy.noWatermarks(), "TestCdcSource"); source.setParallelism(2); Options catalogOptions = new Options(); catalogOptions.set("warehouse", tempDir.toString()); - Catalog.Loader catalogLoader = - () -> FlinkCatalogFactory.createPaimonCatalog(catalogOptions); + CatalogLoader catalogLoader = () -> FlinkCatalogFactory.createPaimonCatalog(catalogOptions); new FlinkCdcSyncDatabaseSinkBuilder() .withInput(source) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java index 081bd7d073d7..9fccaac99228 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java @@ -19,7 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.CoreOptions; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.InternalRow; @@ -43,6 +43,7 @@ import org.apache.paimon.utils.FailingFileIO; import org.apache.paimon.utils.TraceableFileIO; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.Disabled; @@ -151,14 +152,14 @@ private void innerTestRandomCdcEvents( .allowRestart(enableFailure) .build(); - TestCdcSourceFunction sourceFunction = new TestCdcSourceFunction(testTable.events()); - DataStreamSource source = env.addSource(sourceFunction); + TestCdcSource testCdcSource = new TestCdcSource(testTable.events()); + DataStreamSource source = + env.fromSource(testCdcSource, WatermarkStrategy.noWatermarks(), "TestCdcSource"); source.setParallelism(2); Options catalogOptions = new Options(); catalogOptions.set("warehouse", tempDir.toString()); - Catalog.Loader catalogLoader = - () -> FlinkCatalogFactory.createPaimonCatalog(catalogOptions); + CatalogLoader catalogLoader = () -> FlinkCatalogFactory.createPaimonCatalog(catalogOptions); new CdcSinkBuilder() .withInput(source) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSource.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSource.java new file mode 100644 index 000000000000..b45983000a23 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSource.java @@ -0,0 +1,126 @@ +/* + * 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.cdc; + +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; +import org.apache.paimon.flink.source.SplitListState; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; + +import java.util.Collection; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Testing parallel {@link org.apache.flink.api.connector.source.Source} to produce {@link + * TestCdcEvent}. {@link TestCdcEvent}s with the same key will be produced by the same parallelism. + */ +public class TestCdcSource extends AbstractNonCoordinatedSource { + + private static final long serialVersionUID = 1L; + private final LinkedList events; + + public TestCdcSource(Collection events) { + this.events = new LinkedList<>(events); + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SourceReader createReader(SourceReaderContext context) { + return new Reader( + context.getIndexOfSubtask(), + context.currentParallelism(), + new LinkedList<>(events)); + } + + private static class Reader extends AbstractNonCoordinatedSourceReader { + private final int subtaskId; + private final int totalSubtasks; + + private final LinkedList events; + private final SplitListState remainingEventsCount = + new SplitListState<>("events", x -> Integer.toString(x), Integer::parseInt); + + private final int numRecordsPerCheckpoint; + private final AtomicInteger recordsThisCheckpoint; + + private Reader(int subtaskId, int totalSubtasks, LinkedList events) { + this.subtaskId = subtaskId; + this.totalSubtasks = totalSubtasks; + this.events = events; + numRecordsPerCheckpoint = + events.size() / ThreadLocalRandom.current().nextInt(10, 20) + 1; + recordsThisCheckpoint = new AtomicInteger(0); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + if (events.isEmpty()) { + return InputStatus.END_OF_INPUT; + } + + if (recordsThisCheckpoint.get() >= numRecordsPerCheckpoint) { + Thread.sleep(10); + return InputStatus.MORE_AVAILABLE; + } + + TestCdcEvent event = events.poll(); + if (event.records() != null) { + if (Math.abs(event.hashCode()) % totalSubtasks != subtaskId) { + return InputStatus.MORE_AVAILABLE; + } + } + readerOutput.collect(event); + recordsThisCheckpoint.incrementAndGet(); + return InputStatus.MORE_AVAILABLE; + } + + @Override + public List snapshotState(long l) { + recordsThisCheckpoint.set(0); + remainingEventsCount.clear(); + remainingEventsCount.add(events.size()); + return remainingEventsCount.snapshotState(); + } + + @Override + public void addSplits(List list) { + remainingEventsCount.restoreState(list); + int count = 0; + for (int c : remainingEventsCount.get()) { + count += c; + } + while (events.size() > count) { + events.poll(); + } + } + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSourceFunction.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSourceFunction.java deleted file mode 100644 index 4e03256a5253..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSourceFunction.java +++ /dev/null @@ -1,107 +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.cdc; - -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; -import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; -import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; - -import java.util.Collection; -import java.util.LinkedList; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * Testing {@link RichParallelSourceFunction} to produce {@link TestCdcEvent}. {@link TestCdcEvent}s - * with the same key will be produced by the same parallelism. - */ -public class TestCdcSourceFunction extends RichParallelSourceFunction - implements CheckpointedFunction { - - private static final long serialVersionUID = 1L; - - private final LinkedList events; - - private volatile boolean isRunning = true; - private transient int numRecordsPerCheckpoint; - private transient AtomicInteger recordsThisCheckpoint; - private transient ListState remainingEventsCount; - - public TestCdcSourceFunction(Collection events) { - this.events = new LinkedList<>(events); - } - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - numRecordsPerCheckpoint = events.size() / ThreadLocalRandom.current().nextInt(10, 20) + 1; - recordsThisCheckpoint = new AtomicInteger(0); - - remainingEventsCount = - context.getOperatorStateStore() - .getListState(new ListStateDescriptor<>("count", Integer.class)); - - if (context.isRestored()) { - int count = 0; - for (int c : remainingEventsCount.get()) { - count += c; - } - while (events.size() > count) { - events.poll(); - } - } - } - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - recordsThisCheckpoint.set(0); - remainingEventsCount.clear(); - remainingEventsCount.add(events.size()); - } - - @Override - public void run(SourceContext ctx) throws Exception { - while (isRunning && !events.isEmpty()) { - if (recordsThisCheckpoint.get() >= numRecordsPerCheckpoint) { - Thread.sleep(10); - continue; - } - - synchronized (ctx.getCheckpointLock()) { - TestCdcEvent event = events.poll(); - if (event.records() != null) { - int subtaskId = getRuntimeContext().getIndexOfThisSubtask(); - int totalSubtasks = getRuntimeContext().getNumberOfParallelSubtasks(); - if (Math.abs(event.hashCode()) % totalSubtasks != subtaskId) { - continue; - } - } - ctx.collect(event); - recordsThisCheckpoint.incrementAndGet(); - } - } - } - - @Override - public void cancel() { - isRunning = false; - } -} 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 2d57b2ef59ca..510d7c59a8b7 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,13 +22,13 @@ 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; import org.apache.paimon.flink.procedure.ProcedureUtil; import org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil; import org.apache.paimon.flink.utils.FlinkDescriptorProperties; -import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.operation.FileStoreCommit; import org.apache.paimon.options.Options; @@ -523,25 +523,11 @@ protected Schema buildPaimonSchema( // 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. - Catalog.tableDefaultOptions(catalog.options()).forEach(options::putIfAbsent); + CatalogUtils.tableDefaultOptions(catalog.options()).forEach(options::putIfAbsent); options.put(REGISTER_TIMEOUT.key(), logStoreAutoRegisterTimeout.toString()); registerLogSystem(catalog, identifier, options, classLoader); } - // remove table path - String path = options.remove(PATH.key()); - if (path != null) { - Path expectedPath = catalog.getTableLocation(identifier); - if (!new Path(path).equals(expectedPath)) { - throw new CatalogException( - String.format( - "You specified the Path when creating the table, " - + "but the Path '%s' is different from where it should be '%s'. " - + "Please remove the Path.", - path, expectedPath)); - } - } - 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/action/ActionBase.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java index 30e32d62efec..4490023e7b03 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java @@ -20,6 +20,7 @@ import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.flink.FlinkCatalog; import org.apache.paimon.flink.FlinkCatalogFactory; import org.apache.paimon.flink.LogicalTypeConversion; @@ -99,7 +100,7 @@ protected void execute(String defaultName) throws Exception { env.execute(name); } - protected Catalog.Loader catalogLoader() { + protected CatalogLoader catalogLoader() { // to make the action workflow serializable Options catalogOptions = this.catalogOptions; return () -> FlinkCatalogFactory.createPaimonCatalog(catalogOptions); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java index d3554242994b..8bcaa2a2071f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java @@ -18,12 +18,14 @@ package org.apache.paimon.flink.clone; +import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkCatalogFactory; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; +import org.apache.paimon.table.Table; import org.apache.paimon.utils.IOUtils; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; @@ -32,6 +34,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashMap; import java.util.Map; /** A Operator to copy files. */ @@ -43,8 +46,11 @@ public class CopyFileOperator extends AbstractStreamOperator private final Map sourceCatalogConfig; private final Map targetCatalogConfig; - private Catalog sourceCatalog; - private Catalog targetCatalog; + private transient Catalog sourceCatalog; + private transient Catalog targetCatalog; + + private transient Map srcLocations; + private transient Map targetLocations; public CopyFileOperator( Map sourceCatalogConfig, Map targetCatalogConfig) { @@ -58,6 +64,8 @@ public void open() throws Exception { FlinkCatalogFactory.createPaimonCatalog(Options.fromMap(sourceCatalogConfig)); targetCatalog = FlinkCatalogFactory.createPaimonCatalog(Options.fromMap(targetCatalogConfig)); + srcLocations = new HashMap<>(); + targetLocations = new HashMap<>(); } @Override @@ -66,12 +74,29 @@ public void processElement(StreamRecord streamRecord) throws Exce FileIO sourceTableFileIO = sourceCatalog.fileIO(); FileIO targetTableFileIO = targetCatalog.fileIO(); + Path sourceTableRootPath = - sourceCatalog.getTableLocation( - Identifier.fromString(cloneFileInfo.getSourceIdentifier())); + srcLocations.computeIfAbsent( + cloneFileInfo.getSourceIdentifier(), + key -> { + try { + return pathOfTable( + sourceCatalog.getTable(Identifier.fromString(key))); + } catch (Catalog.TableNotExistException e) { + throw new RuntimeException(e); + } + }); Path targetTableRootPath = - targetCatalog.getTableLocation( - Identifier.fromString(cloneFileInfo.getTargetIdentifier())); + targetLocations.computeIfAbsent( + cloneFileInfo.getTargetIdentifier(), + key -> { + try { + return pathOfTable( + targetCatalog.getTable(Identifier.fromString(key))); + } catch (Catalog.TableNotExistException e) { + throw new RuntimeException(e); + } + }); String filePathExcludeTableRoot = cloneFileInfo.getFilePathExcludeTableRoot(); Path sourcePath = new Path(sourceTableRootPath + filePathExcludeTableRoot); @@ -110,6 +135,10 @@ public void processElement(StreamRecord streamRecord) throws Exce output.collect(streamRecord); } + private Path pathOfTable(Table table) { + return new Path(table.options().get(CoreOptions.PATH.key())); + } + @Override public void close() throws Exception { if (sourceCatalog != null) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java index 747995d20d67..e2fd5a9d318e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink.compact; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; @@ -32,7 +32,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -48,19 +47,12 @@ public class MultiAwareBucketTableScan extends MultiTableScanBase scansMap; public MultiAwareBucketTableScan( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, - boolean isStreaming, - AtomicBoolean isRunning) { - super( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); + boolean isStreaming) { + super(catalogLoader, includingPattern, excludingPattern, databasePattern, isStreaming); tablesMap = new HashMap<>(); scansMap = new HashMap<>(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java index bd4ffe83a4ca..805e8da0a417 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java @@ -20,18 +20,18 @@ import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.EndOfScanException; import org.apache.paimon.table.source.Split; -import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.api.connector.source.ReaderOutput; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; import static org.apache.paimon.flink.utils.MultiTablesCompactorUtil.shouldCompactTable; @@ -57,22 +57,19 @@ public abstract class MultiTableScanBase implements AutoCloseable { protected transient Catalog catalog; - protected AtomicBoolean isRunning; protected boolean isStreaming; public MultiTableScanBase( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, - boolean isStreaming, - AtomicBoolean isRunning) { + boolean isStreaming) { catalog = catalogLoader.load(); this.includingPattern = includingPattern; this.excludingPattern = excludingPattern; this.databasePattern = databasePattern; - this.isRunning = isRunning; this.isStreaming = isStreaming; } @@ -104,13 +101,9 @@ protected void updateTableMap() } } - public ScanResult scanTable(SourceFunction.SourceContext ctx) + public ScanResult scanTable(ReaderOutput ctx) throws Catalog.TableNotExistException, Catalog.DatabaseNotExistException { try { - if (!isRunning.get()) { - return ScanResult.FINISHED; - } - updateTableMap(); List tasks = doScan(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java index 56bf971240e7..2ad2642b6248 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java @@ -20,7 +20,7 @@ import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; import org.apache.paimon.append.UnawareAppendTableCompactionCoordinator; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; @@ -29,7 +29,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; /** @@ -42,19 +41,12 @@ public class MultiUnawareBucketTableScan protected transient Map tablesMap; public MultiUnawareBucketTableScan( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, - boolean isStreaming, - AtomicBoolean isRunning) { - super( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); + boolean isStreaming) { + super(catalogLoader, includingPattern, excludingPattern, databasePattern, isStreaming); tablesMap = new HashMap<>(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java index 8c6ed4c9f59e..a572354e8984 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java @@ -126,7 +126,7 @@ private DataStreamSource buildSource() { new BucketUnawareCompactSource( table, isContinuous, scanInterval, partitionPredicate); - return BucketUnawareCompactSource.buildSource(env, source, isContinuous, tableIdentifier); + return BucketUnawareCompactSource.buildSource(env, source, tableIdentifier); } private void sinkFromSource(DataStreamSource input) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskSerializer.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskSerializer.java index 34f9d035d8d2..e21220b26db5 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskSerializer.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskSerializer.java @@ -26,10 +26,10 @@ import org.apache.paimon.io.DataOutputViewStreamWrapper; import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.util.CollectionUtil; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -102,8 +102,7 @@ private void serializeMap(Map> map, DataOutputView v private Map> deserializeMap(DataInputView view) throws IOException { final int size = view.readInt(); - final Map> map = - CollectionUtil.newHashMapWithExpectedSize(size); + final Map> map = new HashMap<>(size); for (int i = 0; i < size; i++) { map.put(view.readInt(), dataFileSerializer.deserializeList(view)); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/PurgeFilesProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/PurgeFilesProcedure.java new file mode 100644 index 000000000000..7ee2a3610402 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/PurgeFilesProcedure.java @@ -0,0 +1,78 @@ +/* + * 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.procedure; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.Table; + +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.annotation.DataTypeHint; +import org.apache.flink.table.annotation.ProcedureHint; +import org.apache.flink.table.procedure.ProcedureContext; + +import java.io.IOException; +import java.util.Arrays; + +/** + * A procedure to purge files for a table. Usage: + * + *

    
    + *  -- rollback to the snapshot which earlier or equal than watermark.
    + *  CALL sys.purge_files(`table` => 'tableId')
    + * 
    + */ +public class PurgeFilesProcedure extends ProcedureBase { + + public static final String IDENTIFIER = "purge_files"; + + @ProcedureHint(argument = {@ArgumentHint(name = "table", type = @DataTypeHint("STRING"))}) + public String[] call(ProcedureContext procedureContext, String tableId) + throws Catalog.TableNotExistException { + Table table = catalog.getTable(Identifier.fromString(tableId)); + FileStoreTable fileStoreTable = (FileStoreTable) table; + FileIO fileIO = fileStoreTable.fileIO(); + Path tablePath = fileStoreTable.snapshotManager().tablePath(); + try { + Arrays.stream(fileIO.listStatus(tablePath)) + .filter(f -> !f.getPath().getName().contains("schema")) + .forEach( + fileStatus -> { + try { + fileIO.delete(fileStatus.getPath(), true); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } catch (IOException e) { + throw new RuntimeException(e); + } + return new String[] { + String.format("Success purge files with table: %s.", fileStoreTable.name()) + }; + } + + @Override + public String identifier() { + return IDENTIFIER; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java index b9776786fa57..6688503778a0 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java @@ -21,6 +21,9 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.flink.utils.InternalTypeInfo; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; @@ -31,11 +34,14 @@ import org.apache.paimon.table.source.TableRead; import org.apache.paimon.table.system.FileMonitorTable; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import java.util.ArrayList; import java.util.List; @@ -50,19 +56,13 @@ *
  • Assigning them to downstream tasks for further processing. * */ -public class QueryFileMonitor extends RichSourceFunction { +public class QueryFileMonitor extends AbstractNonCoordinatedSource { private static final long serialVersionUID = 1L; private final Table table; private final long monitorInterval; - private transient SourceContext ctx; - private transient StreamTableScan scan; - private transient TableRead read; - - private volatile boolean isRunning = true; - public QueryFileMonitor(Table table) { this.table = table; this.monitorInterval = @@ -71,65 +71,54 @@ public QueryFileMonitor(Table table) { .toMillis(); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; } - /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. - */ - public void open(Configuration parameters) throws Exception { - FileMonitorTable monitorTable = new FileMonitorTable((FileStoreTable) table); - ReadBuilder readBuilder = monitorTable.newReadBuilder().dropStats(); - this.scan = readBuilder.newStreamScan(); - this.read = readBuilder.newRead(); + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - @Override - public void run(SourceContext ctx) throws Exception { - this.ctx = ctx; - while (isRunning) { - boolean isEmpty; - synchronized (ctx.getCheckpointLock()) { - if (!isRunning) { - return; - } - isEmpty = doScan(); - } + private class Reader extends AbstractNonCoordinatedSourceReader { + private transient StreamTableScan scan; + private transient TableRead read; + + @Override + public void start() { + FileMonitorTable monitorTable = new FileMonitorTable((FileStoreTable) table); + ReadBuilder readBuilder = monitorTable.newReadBuilder().dropStats(); + this.scan = readBuilder.newStreamScan(); + this.read = readBuilder.newRead(); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + boolean isEmpty = doScan(readerOutput); if (isEmpty) { Thread.sleep(monitorInterval); } + return InputStatus.MORE_AVAILABLE; } - } - private boolean doScan() throws Exception { - List records = new ArrayList<>(); - read.createReader(scan.plan()).forEachRemaining(records::add); - records.forEach(ctx::collect); - return records.isEmpty(); - } - - @Override - public void cancel() { - // this is to cover the case where cancel() is called before the run() - if (ctx != null) { - synchronized (ctx.getCheckpointLock()) { - isRunning = false; - } - } else { - isRunning = false; + private boolean doScan(ReaderOutput readerOutput) throws Exception { + List records = new ArrayList<>(); + read.createReader(scan.plan()).forEachRemaining(records::add); + records.forEach(readerOutput::collect); + return records.isEmpty(); } } public static DataStream build(StreamExecutionEnvironment env, Table table) { - return env.addSource( - new QueryFileMonitor(table), - "FileMonitor-" + table.name(), - InternalTypeInfo.fromRowType(FileMonitorTable.getRowType())); + return env.fromSource( + new QueryFileMonitor(table), + WatermarkStrategy.noWatermarks(), + "FileMonitor-" + table.name(), + InternalTypeInfo.fromRowType(FileMonitorTable.getRowType())) + .setParallelism(1); } public static ChannelComputer createChannelComputer() { 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 83d51f302e51..07ec7d165e3a 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 @@ -21,6 +21,7 @@ import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.compact.UnawareBucketCompactor; import org.apache.paimon.options.Options; @@ -56,7 +57,7 @@ public class AppendOnlyMultiTableCompactionWorkerOperator LoggerFactory.getLogger(AppendOnlyMultiTableCompactionWorkerOperator.class); private final String commitUser; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; // support multi table compaction private transient Map compactorContainer; @@ -67,7 +68,7 @@ public class AppendOnlyMultiTableCompactionWorkerOperator private AppendOnlyMultiTableCompactionWorkerOperator( StreamOperatorParameters parameters, - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, String commitUser, Options options) { super(parameters, options); @@ -188,9 +189,9 @@ public static class Factory MultiTableUnawareAppendCompactionTask, MultiTableCommittable> { private final String commitUser; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; - public Factory(Catalog.Loader catalogLoader, String commitUser, Options options) { + public Factory(CatalogLoader catalogLoader, String commitUser, Options options) { super(options); this.commitUser = commitUser; this.catalogLoader = catalogLoader; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java index 25f76ce97683..53f1bf165d98 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java @@ -20,7 +20,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.manifest.WrappedManifestCommittable; import org.apache.paimon.options.Options; @@ -55,14 +55,14 @@ public class CombinedTableCompactorSink implements Serializable { private static final String WRITER_NAME = "Writer"; private static final String GLOBAL_COMMITTER_NAME = "Global Committer"; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; private final boolean ignorePreviousFiles; private final boolean fullCompaction; private final Options options; public CombinedTableCompactorSink( - Catalog.Loader catalogLoader, Options options, boolean fullCompaction) { + CatalogLoader catalogLoader, Options options, boolean fullCompaction) { this.catalogLoader = catalogLoader; this.ignorePreviousFiles = false; this.fullCompaction = fullCompaction; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java index 58f6a3834096..02a7e6c1b3c8 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java @@ -20,6 +20,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.flink.utils.RuntimeContextUtils; @@ -72,7 +73,7 @@ public class MultiTablesStoreCompactOperator private transient StoreSinkWriteState state; private transient DataFileMetaSerializer dataFileMetaSerializer; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; protected Catalog catalog; protected Map tables; @@ -81,7 +82,7 @@ public class MultiTablesStoreCompactOperator private MultiTablesStoreCompactOperator( StreamOperatorParameters parameters, - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, String initialCommitUser, CheckpointConfig checkpointConfig, boolean isStreaming, @@ -324,7 +325,7 @@ private StoreSinkWrite.Provider createWriteProvider( /** {@link StreamOperatorFactory} of {@link MultiTablesStoreCompactOperator}. */ public static class Factory extends PrepareCommitOperator.Factory { - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; private final CheckpointConfig checkpointConfig; private final boolean isStreaming; private final boolean ignorePreviousFiles; @@ -332,7 +333,7 @@ public static class Factory private final String initialCommitUser; public Factory( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, String initialCommitUser, CheckpointConfig checkpointConfig, boolean isStreaming, 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 537a98f97fb0..01acddb9ad99 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 @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.manifest.WrappedManifestCommittable; @@ -56,12 +57,12 @@ public class StoreMultiCommitter private final boolean ignoreEmptyCommit; private final Map dynamicOptions; - public StoreMultiCommitter(Catalog.Loader catalogLoader, Context context) { + public StoreMultiCommitter(CatalogLoader catalogLoader, Context context) { this(catalogLoader, context, false, Collections.emptyMap()); } public StoreMultiCommitter( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Context context, boolean ignoreEmptyCommit, Map dynamicOptions) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionListeners.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionListeners.java index dbdf77601480..d190b9ccf39e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionListeners.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionListeners.java @@ -58,8 +58,11 @@ public static PartitionListeners create(Committer.Context context, FileStoreTabl throws Exception { List listeners = new ArrayList<>(); - ReportHmsListener.create(context.isRestored(), context.stateStore(), table) + // partition statistics reporter + ReportPartStatsListener.create(context.isRestored(), context.stateStore(), table) .ifPresent(listeners::add); + + // partition mark done PartitionMarkDone.create( context.streamingCheckpointEnabled(), context.isRestored(), diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/HmsReporter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporter.java similarity index 85% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/HmsReporter.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporter.java index eb965aa3a318..b75889d567ee 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/HmsReporter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporter.java @@ -40,22 +40,24 @@ import java.util.List; import java.util.Map; -import static org.apache.paimon.catalog.Catalog.HIVE_LAST_UPDATE_TIME_PROP; +import static org.apache.paimon.catalog.Catalog.LAST_UPDATE_TIME_PROP; import static org.apache.paimon.catalog.Catalog.NUM_FILES_PROP; 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.utils.PartitionPathUtils.extractPartitionSpecFromPath; /** Action to report the table statistic from the latest snapshot to HMS. */ -public class HmsReporter implements Closeable { +public class PartitionStatisticsReporter implements Closeable { - private static final Logger LOG = LoggerFactory.getLogger(HmsReporter.class); + private static final Logger LOG = LoggerFactory.getLogger(PartitionStatisticsReporter.class); + + private static final String HIVE_LAST_UPDATE_TIME_PROP = "transient_lastDdlTime"; private final MetastoreClient metastoreClient; private final SnapshotReader snapshotReader; private final SnapshotManager snapshotManager; - public HmsReporter(FileStoreTable table, MetastoreClient client) { + public PartitionStatisticsReporter(FileStoreTable table, MetastoreClient client) { this.metastoreClient = Preconditions.checkNotNull(client, "the metastore client factory is null"); this.snapshotReader = table.newSnapshotReader(); @@ -90,10 +92,15 @@ public void report(String partition, long modifyTime) throws Exception { statistic.put(NUM_FILES_PROP, String.valueOf(fileCount)); statistic.put(TOTAL_SIZE_PROP, String.valueOf(totalSize)); statistic.put(NUM_ROWS_PROP, String.valueOf(rowCount)); - statistic.put(HIVE_LAST_UPDATE_TIME_PROP, String.valueOf(modifyTime / 1000)); + + String modifyTimeSeconds = String.valueOf(modifyTime / 1000); + statistic.put(LAST_UPDATE_TIME_PROP, modifyTimeSeconds); + + // just for being compatible with hive metastore + statistic.put(HIVE_LAST_UPDATE_TIME_PROP, modifyTimeSeconds); LOG.info("alter partition {} with statistic {}.", partitionSpec, statistic); - metastoreClient.alterPartition(partitionSpec, statistic, modifyTime); + metastoreClient.alterPartition(partitionSpec, statistic, modifyTime, true); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/ReportHmsListener.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/ReportPartStatsListener.java similarity index 91% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/ReportHmsListener.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/ReportPartStatsListener.java index 842dd012e88e..ca51c3df5b1a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/ReportHmsListener.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/ReportPartStatsListener.java @@ -49,7 +49,7 @@ * This listener will collect data from the newly touched partition and then decide when to trigger * a report based on the partition's idle time. */ -public class ReportHmsListener implements PartitionListener { +public class ReportPartStatsListener implements PartitionListener { @SuppressWarnings("unchecked") private static final ListStateDescriptor> PENDING_REPORT_STATE_DESC = @@ -58,20 +58,20 @@ public class ReportHmsListener implements PartitionListener { new MapSerializer<>(StringSerializer.INSTANCE, LongSerializer.INSTANCE)); private final InternalRowPartitionComputer partitionComputer; - private final HmsReporter hmsReporter; + private final PartitionStatisticsReporter partitionStatisticsReporter; private final ListState> pendingPartitionsState; private final Map pendingPartitions; private final long idleTime; - private ReportHmsListener( + private ReportPartStatsListener( InternalRowPartitionComputer partitionComputer, - HmsReporter hmsReporter, + PartitionStatisticsReporter partitionStatisticsReporter, OperatorStateStore store, boolean isRestored, long idleTime) throws Exception { this.partitionComputer = partitionComputer; - this.hmsReporter = hmsReporter; + this.partitionStatisticsReporter = partitionStatisticsReporter; this.pendingPartitionsState = store.getListState(PENDING_REPORT_STATE_DESC); this.pendingPartitions = new HashMap<>(); if (isRestored) { @@ -108,7 +108,7 @@ public void notifyCommittable(List committables) { try { Map partitions = reportPartition(endInput); for (Map.Entry entry : partitions.entrySet()) { - hmsReporter.report(entry.getKey(), entry.getValue()); + partitionStatisticsReporter.report(entry.getKey(), entry.getValue()); } } catch (Exception e) { throw new RuntimeException(e); @@ -138,7 +138,7 @@ public void snapshotState() throws Exception { pendingPartitionsState.update(Collections.singletonList(pendingPartitions)); } - public static Optional create( + public static Optional create( boolean isRestored, OperatorStateStore stateStore, FileStoreTable table) throws Exception { @@ -169,9 +169,9 @@ public static Optional create( coreOptions.legacyPartitionName()); return Optional.of( - new ReportHmsListener( + new ReportPartStatsListener( partitionComputer, - new HmsReporter( + new PartitionStatisticsReporter( table, table.catalogEnvironment().metastoreClientFactory().create()), stateStore, @@ -182,8 +182,8 @@ public static Optional create( @Override public void close() throws IOException { - if (hmsReporter != null) { - hmsReporter.close(); + if (partitionStatisticsReporter != null) { + partitionStatisticsReporter.close(); } } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSource.java new file mode 100644 index 000000000000..a9a389e837a2 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSource.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.source; + +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.core.io.SimpleVersionedSerializer; + +/** {@link Source} that does not require coordination between JobManager and TaskManagers. */ +public abstract class AbstractNonCoordinatedSource + implements Source { + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + return new NoOpEnumerator<>(); + } + + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, NoOpEnumState checkpoint) { + return new NoOpEnumerator<>(); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new SimpleSourceSplitSerializer(); + } + + @Override + public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { + return new NoOpEnumStateSerializer(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSourceReader.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSourceReader.java new file mode 100644 index 000000000000..18c278868ffa --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSourceReader.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.source; + +import org.apache.flink.api.connector.source.SourceReader; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** Abstract {@link SourceReader} for {@link AbstractNonCoordinatedSource}. */ +public abstract class AbstractNonCoordinatedSourceReader + implements SourceReader { + @Override + public void start() {} + + @Override + public List snapshotState(long l) { + return Collections.emptyList(); + } + + @Override + public CompletableFuture isAvailable() { + return CompletableFuture.completedFuture(null); + } + + @Override + public void addSplits(List list) {} + + @Override + public void notifyNoMoreSplits() {} + + @Override + public void close() throws Exception {} +} 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 5dbbdcedd82a..a94d799773bc 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 @@ -28,7 +28,6 @@ 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.manifest.PartitionEntry; import org.apache.paimon.options.ConfigOption; import org.apache.paimon.options.Options; import org.apache.paimon.predicate.Predicate; @@ -36,7 +35,8 @@ import org.apache.paimon.table.DataTable; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; -import org.apache.paimon.table.source.TableScan; +import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.table.source.Split; import org.apache.paimon.utils.Projection; import org.apache.flink.api.common.eventtime.WatermarkStrategy; @@ -74,6 +74,7 @@ import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_ALIGNMENT_UPDATE_INTERVAL; import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_EMIT_STRATEGY; import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_IDLE_TIMEOUT; +import static org.apache.paimon.utils.Preconditions.checkNotNull; /** * Table source to create {@link StaticFileStoreSource} or {@link ContinuousFileStoreSource} under @@ -98,7 +99,7 @@ public abstract class BaseDataTableSource extends FlinkTableSource @Nullable protected final LogStoreTableFactory logStoreTableFactory; @Nullable protected WatermarkStrategy watermarkStrategy; - protected boolean isBatchCountStar; + @Nullable protected Long countPushed; public BaseDataTableSource( ObjectIdentifier tableIdentifier, @@ -110,7 +111,7 @@ public BaseDataTableSource( @Nullable int[][] projectFields, @Nullable Long limit, @Nullable WatermarkStrategy watermarkStrategy, - boolean isBatchCountStar) { + @Nullable Long countPushed) { super(table, predicate, projectFields, limit); this.tableIdentifier = tableIdentifier; this.streaming = streaming; @@ -120,7 +121,7 @@ public BaseDataTableSource( this.projectFields = projectFields; this.limit = limit; this.watermarkStrategy = watermarkStrategy; - this.isBatchCountStar = isBatchCountStar; + this.countPushed = countPushed; } @Override @@ -159,7 +160,7 @@ public ChangelogMode getChangelogMode() { @Override public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { - if (isBatchCountStar) { + if (countPushed != null) { return createCountStarScan(); } @@ -212,10 +213,8 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { } private ScanRuntimeProvider createCountStarScan() { - TableScan scan = table.newReadBuilder().withFilter(predicate).newScan(); - List partitionEntries = scan.listPartitionEntries(); - long rowCount = partitionEntries.stream().mapToLong(PartitionEntry::recordCount).sum(); - NumberSequenceRowSource source = new NumberSequenceRowSource(rowCount, rowCount); + checkNotNull(countPushed); + NumberSequenceRowSource source = new NumberSequenceRowSource(countPushed, countPushed); return new SourceProvider() { @Override public Source createSource() { @@ -303,15 +302,6 @@ public boolean applyAggregates( return false; } - if (!table.primaryKeys().isEmpty()) { - return false; - } - - CoreOptions options = ((DataTable) table).coreOptions(); - if (options.deletionVectorsEnabled()) { - return false; - } - if (groupingSets.size() != 1) { return false; } @@ -334,7 +324,22 @@ public boolean applyAggregates( return false; } - isBatchCountStar = true; + List splits = + table.newReadBuilder().dropStats().withFilter(predicate).newScan().plan().splits(); + long countPushed = 0; + for (Split s : splits) { + if (!(s instanceof DataSplit)) { + return false; + } + DataSplit split = (DataSplit) s; + if (!split.mergedRowCountAvailable()) { + return false; + } + + countPushed += split.mergedRowCount(); + } + + this.countPushed = countPushed; return true; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java index 79ee827fe6e4..7954aad2df0a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java @@ -21,19 +21,19 @@ import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.append.UnawareAppendTableCompactionCoordinator; import org.apache.paimon.flink.sink.CompactionTaskTypeInfo; -import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.source.EndOfScanException; -import org.apache.paimon.utils.Preconditions; -import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; -import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,15 +42,16 @@ import java.util.List; /** - * Source Function for unaware-bucket Compaction. + * Source for unaware-bucket Compaction. * - *

    Note: The function is the source function of unaware-bucket compactor coordinator. It will - * read the latest snapshot continuously by compactionCoordinator, and generate new compaction - * tasks. The source function is used in unaware-bucket compaction job (both stand-alone and - * write-combined). Besides, we don't need to save state in this function, it will invoke a full - * scan when starting up, and scan continuously for the following snapshot. + *

    Note: The function is the source of unaware-bucket compactor coordinator. It will read the + * latest snapshot continuously by compactionCoordinator, and generate new compaction tasks. The + * source is used in unaware-bucket compaction job (both stand-alone and write-combined). Besides, + * we don't need to save state in this source, it will invoke a full scan when starting up, and scan + * continuously for the following snapshot. */ -public class BucketUnawareCompactSource extends RichSourceFunction { +public class BucketUnawareCompactSource + extends AbstractNonCoordinatedSource { private static final Logger LOG = LoggerFactory.getLogger(BucketUnawareCompactSource.class); private static final String COMPACTION_COORDINATOR_NAME = "Compaction Coordinator"; @@ -59,9 +60,6 @@ public class BucketUnawareCompactSource extends RichSourceFunction ctx; - private volatile boolean isRunning = true; public BucketUnawareCompactSource( FileStoreTable table, @@ -74,76 +72,64 @@ public BucketUnawareCompactSource( this.filter = filter; } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + @Override + public Boundedness getBoundedness() { + return streaming ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; } - /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. - */ - public void open(Configuration parameters) throws Exception { - compactionCoordinator = - new UnawareAppendTableCompactionCoordinator(table, streaming, filter); + @Override + public SourceReader createReader( + SourceReaderContext readerContext) throws Exception { Preconditions.checkArgument( - RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()) == 1, + readerContext.currentParallelism() == 1, "Compaction Operator parallelism in paimon MUST be one."); + return new BucketUnawareCompactSourceReader(table, streaming, filter, scanInterval); } - @Override - public void run(SourceContext sourceContext) throws Exception { - this.ctx = sourceContext; - while (isRunning) { + /** BucketUnawareCompactSourceReader. */ + public static class BucketUnawareCompactSourceReader + extends AbstractNonCoordinatedSourceReader { + private final UnawareAppendTableCompactionCoordinator compactionCoordinator; + private final long scanInterval; + + public BucketUnawareCompactSourceReader( + FileStoreTable table, boolean streaming, Predicate filter, long scanInterval) { + this.scanInterval = scanInterval; + compactionCoordinator = + new UnawareAppendTableCompactionCoordinator(table, streaming, filter); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) + throws Exception { boolean isEmpty; - synchronized (ctx.getCheckpointLock()) { - if (!isRunning) { - return; - } - try { - // do scan and plan action, emit append-only compaction tasks. - List tasks = compactionCoordinator.run(); - isEmpty = tasks.isEmpty(); - tasks.forEach(ctx::collect); - } catch (EndOfScanException esf) { - LOG.info("Catching EndOfStreamException, the stream is finished."); - return; - } + try { + // do scan and plan action, emit append-only compaction tasks. + List tasks = compactionCoordinator.run(); + isEmpty = tasks.isEmpty(); + tasks.forEach(readerOutput::collect); + } catch (EndOfScanException esf) { + LOG.info("Catching EndOfStreamException, the stream is finished."); + return InputStatus.END_OF_INPUT; } if (isEmpty) { Thread.sleep(scanInterval); } - } - } - - @Override - public void cancel() { - if (ctx != null) { - synchronized (ctx.getCheckpointLock()) { - isRunning = false; - } - } else { - isRunning = false; + return InputStatus.MORE_AVAILABLE; } } public static DataStreamSource buildSource( StreamExecutionEnvironment env, BucketUnawareCompactSource source, - boolean streaming, String tableIdentifier) { - final StreamSource sourceOperator = - new StreamSource<>(source); return (DataStreamSource) - new DataStreamSource<>( - env, - new CompactionTaskTypeInfo(), - sourceOperator, - false, + env.fromSource( + source, + WatermarkStrategy.noWatermarks(), COMPACTION_COORDINATOR_NAME + " : " + tableIdentifier, - streaming ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED) + new CompactionTaskTypeInfo()) .setParallelism(1) .setMaxParallelism(1); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java index e5cbbe845ceb..ac6af6a14f6a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java @@ -19,12 +19,12 @@ package org.apache.paimon.flink.source; import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.flink.LogicalTypeConversion; -import org.apache.paimon.flink.source.operator.CombinedAwareBatchSourceFunction; -import org.apache.paimon.flink.source.operator.CombinedAwareStreamingSourceFunction; -import org.apache.paimon.flink.source.operator.CombinedUnawareBatchSourceFunction; -import org.apache.paimon.flink.source.operator.CombinedUnawareStreamingSourceFunction; +import org.apache.paimon.flink.source.operator.CombinedAwareBatchSource; +import org.apache.paimon.flink.source.operator.CombinedAwareStreamingSource; +import org.apache.paimon.flink.source.operator.CombinedUnawareBatchSource; +import org.apache.paimon.flink.source.operator.CombinedUnawareStreamingSource; import org.apache.paimon.table.system.CompactBucketsTable; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.Preconditions; @@ -44,7 +44,8 @@ * compactor jobs in combined mode. */ public class CombinedTableCompactorSourceBuilder { - private final Catalog.Loader catalogLoader; + + private final CatalogLoader catalogLoader; private final Pattern includingPattern; private final Pattern excludingPattern; private final Pattern databasePattern; @@ -55,7 +56,7 @@ public class CombinedTableCompactorSourceBuilder { @Nullable private Duration partitionIdleTime = null; public CombinedTableCompactorSourceBuilder( - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern databasePattern, Pattern includingPattern, Pattern excludingPattern, @@ -87,7 +88,7 @@ public DataStream buildAwareBucketTableSource() { Preconditions.checkArgument(env != null, "StreamExecutionEnvironment should not be null."); RowType produceType = CompactBucketsTable.getRowType(); if (isContinuous) { - return CombinedAwareStreamingSourceFunction.buildSource( + return CombinedAwareStreamingSource.buildSource( env, "Combine-MultiBucketTables--StreamingCompactorSource", InternalTypeInfo.of(LogicalTypeConversion.toLogicalType(produceType)), @@ -97,7 +98,7 @@ public DataStream buildAwareBucketTableSource() { databasePattern, monitorInterval); } else { - return CombinedAwareBatchSourceFunction.buildSource( + return CombinedAwareBatchSource.buildSource( env, "Combine-MultiBucketTables-BatchCompactorSource", InternalTypeInfo.of(LogicalTypeConversion.toLogicalType(produceType)), @@ -112,7 +113,7 @@ public DataStream buildAwareBucketTableSource() { public DataStream buildForUnawareBucketsTableSource() { Preconditions.checkArgument(env != null, "StreamExecutionEnvironment should not be null."); if (isContinuous) { - return CombinedUnawareStreamingSourceFunction.buildSource( + return CombinedUnawareStreamingSource.buildSource( env, "Combined-UnawareBucketTables-StreamingCompactorSource", catalogLoader, @@ -121,7 +122,7 @@ public DataStream buildForUnawareBucketsT databasePattern, monitorInterval); } else { - return CombinedUnawareBatchSourceFunction.buildSource( + return CombinedUnawareBatchSource.buildSource( env, "Combined-UnawareBucketTables-BatchCompactorSource", catalogLoader, 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 53a1b5f63083..2b470cb4383a 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 @@ -70,7 +70,7 @@ public DataTableSource( null, null, null, - false); + null); } public DataTableSource( @@ -84,7 +84,7 @@ public DataTableSource( @Nullable Long limit, @Nullable WatermarkStrategy watermarkStrategy, @Nullable List dynamicPartitionFilteringFields, - boolean isBatchCountStar) { + @Nullable Long countPushed) { super( tableIdentifier, table, @@ -95,7 +95,7 @@ public DataTableSource( projectFields, limit, watermarkStrategy, - isBatchCountStar); + countPushed); this.dynamicPartitionFilteringFields = dynamicPartitionFilteringFields; } @@ -112,7 +112,7 @@ public DataTableSource copy() { limit, watermarkStrategy, dynamicPartitionFilteringFields, - isBatchCountStar); + countPushed); } @Override 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 e864ec050045..b85d5274b241 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 @@ -26,7 +26,7 @@ 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.MonitorFunction; +import org.apache.paimon.flink.source.operator.MonitorSource; import org.apache.paimon.flink.utils.TableScanUtils; import org.apache.paimon.options.Options; import org.apache.paimon.predicate.Predicate; @@ -306,7 +306,7 @@ private DataStream buildContinuousStreamOperator() { "Cannot limit streaming source, please use batch execution mode."); } dataStream = - MonitorFunction.buildSource( + MonitorSource.buildSource( env, sourceName, produceTypeInfo(), diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumState.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumState.java new file mode 100644 index 000000000000..f07317c155aa --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumState.java @@ -0,0 +1,22 @@ +/* + * 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; + +/** The enumerator state class for {@link NoOpEnumerator}. */ +public class NoOpEnumState {} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumStateSerializer.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumStateSerializer.java new file mode 100644 index 000000000000..89c0ad6ac1f1 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumStateSerializer.java @@ -0,0 +1,41 @@ +/* + * 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.flink.core.io.SimpleVersionedSerializer; + +import java.io.IOException; + +/** {@link SimpleVersionedSerializer} for {@link NoOpEnumState}. */ +public class NoOpEnumStateSerializer implements SimpleVersionedSerializer { + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(NoOpEnumState obj) throws IOException { + return new byte[0]; + } + + @Override + public NoOpEnumState deserialize(int version, byte[] serialized) throws IOException { + return new NoOpEnumState(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumerator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumerator.java new file mode 100644 index 000000000000..f29c6d6db76d --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumerator.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.source; + +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.api.connector.source.SplitEnumerator; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.List; + +/** + * A {@link SplitEnumerator} that provides no functionality. It is basically used for sources that + * does not require a coordinator. + */ +public class NoOpEnumerator + implements SplitEnumerator { + @Override + public void start() {} + + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {} + + @Override + public void addSplitsBack(List splits, int subtaskId) {} + + @Override + public void addReader(int subtaskId) {} + + @Override + public NoOpEnumState snapshotState(long checkpointId) throws Exception { + return new NoOpEnumState(); + } + + @Override + public void close() throws IOException {} +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplit.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplit.java new file mode 100644 index 000000000000..2db0868f8e34 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplit.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.source; + +import org.apache.flink.api.connector.source.SourceSplit; + +import java.util.UUID; + +/** A {@link SourceSplit} that provides basic information through splitId. */ +public class SimpleSourceSplit implements SourceSplit { + private final String splitId; + private final String value; + + public SimpleSourceSplit() { + this(""); + } + + public SimpleSourceSplit(String value) { + this(UUID.randomUUID().toString(), value); + } + + public SimpleSourceSplit(String splitId, String value) { + this.splitId = splitId; + this.value = value; + } + + @Override + public String splitId() { + return splitId; + } + + public String value() { + return value; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplitSerializer.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplitSerializer.java new file mode 100644 index 000000000000..3387afed1c2a --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplitSerializer.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.source; + +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +/** {@link SimpleVersionedSerializer} for {@link SimpleSourceSplit}. */ +public class SimpleSourceSplitSerializer implements SimpleVersionedSerializer { + + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(SimpleSourceSplit split) throws IOException { + if (split.splitId() == null) { + return new byte[0]; + } + + try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final DataOutputStream out = new DataOutputStream(baos)) { + writeString(out, split.splitId()); + writeString(out, split.value()); + return baos.toByteArray(); + } + } + + @Override + public SimpleSourceSplit deserialize(int version, byte[] serialized) throws IOException { + if (serialized.length == 0) { + return new SimpleSourceSplit(); + } + + try (final ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + final DataInputStream in = new DataInputStream(bais)) { + String splitId = readString(in); + String value = readString(in); + return new SimpleSourceSplit(splitId, value); + } + } + + private void writeString(DataOutputStream out, String str) throws IOException { + byte[] bytes = str.getBytes(); + out.writeInt(bytes.length); + out.write(str.getBytes()); + } + + private String readString(DataInputStream in) throws IOException { + int length = in.readInt(); + byte[] bytes = new byte[length]; + in.readFully(bytes); + return new String(bytes); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SplitListState.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SplitListState.java new file mode 100644 index 000000000000..0049bdf284e3 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SplitListState.java @@ -0,0 +1,92 @@ +/* + * 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.utils.Preconditions; + +import org.apache.flink.api.common.state.ListState; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Utility class to provide {@link ListState}-like experience for sources that use {@link + * SimpleSourceSplit}. + */ +public class SplitListState implements ListState { + private final String splitPrefix; + private final List values; + private final Function serializer; + private final Function deserializer; + + public SplitListState( + String identifier, Function serializer, Function deserializer) { + Preconditions.checkArgument( + !Character.isDigit(identifier.charAt(0)), + String.format("Identifier %s should not start with digits.", identifier)); + this.splitPrefix = identifier.length() + identifier; + this.serializer = serializer; + this.deserializer = deserializer; + this.values = new ArrayList<>(); + } + + @Override + public void add(T value) { + values.add(value); + } + + @Override + public List get() { + return new ArrayList<>(values); + } + + @Override + public void update(List values) { + this.values.clear(); + this.values.addAll(values); + } + + @Override + public void addAll(List values) throws Exception { + this.values.addAll(values); + } + + @Override + public void clear() { + values.clear(); + } + + public List snapshotState() { + return values.stream() + .map(x -> new SimpleSourceSplit(splitPrefix + serializer.apply(x))) + .collect(Collectors.toList()); + } + + public void restoreState(List splits) { + values.clear(); + splits.stream() + .map(SimpleSourceSplit::value) + .filter(x -> x.startsWith(splitPrefix)) + .map(x -> x.substring(splitPrefix.length())) + .map(this.deserializer) + .forEach(values::add); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java similarity index 64% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java index 2157be51aee4..2f7a82c95184 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java @@ -18,24 +18,26 @@ package org.apache.paimon.flink.source.operator; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.flink.compact.MultiAwareBucketTableScan; import org.apache.paimon.flink.compact.MultiTableScanBase; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.flink.utils.JavaTypeInfo; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.Split; -import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.table.data.RowData; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,50 +49,45 @@ import static org.apache.paimon.flink.compact.MultiTableScanBase.ScanResult.IS_EMPTY; /** It is responsible for monitoring compactor source of aware bucket table in batch mode. */ -public class CombinedAwareBatchSourceFunction - extends CombinedCompactorSourceFunction> { +public class CombinedAwareBatchSource extends CombinedCompactorSource> { - private static final Logger LOGGER = - LoggerFactory.getLogger(CombinedAwareBatchSourceFunction.class); + private static final Logger LOGGER = LoggerFactory.getLogger(CombinedAwareBatchSource.class); - private MultiTableScanBase> tableScan; - - public CombinedAwareBatchSourceFunction( - Catalog.Loader catalogLoader, + public CombinedAwareBatchSource( + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern) { super(catalogLoader, includingPattern, excludingPattern, databasePattern, false); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + @Override + public SourceReader, SimpleSourceSplit> createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. - */ - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiAwareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); - } + private class Reader extends AbstractNonCoordinatedSourceReader> { + private MultiTableScanBase> tableScan; - @Override - void scanTable() throws Exception { - if (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + @Override + public void start() { + super.start(); + tableScan = + new MultiAwareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } + + @Override + public InputStatus pollNext(ReaderOutput> readerOutput) + throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { // Currently, in the combined mode, there are two scan tasks for the table of two @@ -99,6 +96,15 @@ void scanTable() throws Exception { // should not be thrown exception here. LOGGER.info("No file were collected for the table of aware-bucket"); } + return InputStatus.END_OF_INPUT; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } @@ -106,20 +112,19 @@ public static DataStream buildSource( StreamExecutionEnvironment env, String name, TypeInformation typeInfo, - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, Duration partitionIdleTime) { - CombinedAwareBatchSourceFunction function = - new CombinedAwareBatchSourceFunction( + CombinedAwareBatchSource source = + new CombinedAwareBatchSource( catalogLoader, includingPattern, excludingPattern, databasePattern); - StreamSource, ?> sourceOperator = new StreamSource<>(function); TupleTypeInfo> tupleTypeInfo = new TupleTypeInfo<>( new JavaTypeInfo<>(Split.class), BasicTypeInfo.STRING_TYPE_INFO); - return new DataStreamSource<>( - env, tupleTypeInfo, sourceOperator, false, name, Boundedness.BOUNDED) + + return env.fromSource(source, WatermarkStrategy.noWatermarks(), name, tupleTypeInfo) .forceNonParallel() .partitionCustom( (key, numPartitions) -> key % numPartitions, @@ -129,12 +134,4 @@ public static DataStream buildSource( typeInfo, new MultiTablesReadOperator(catalogLoader, false, partitionIdleTime)); } - - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java similarity index 60% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java index 01e0127e9fda..a23a3b41a441 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java @@ -18,24 +18,26 @@ package org.apache.paimon.flink.source.operator; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.flink.compact.MultiAwareBucketTableScan; import org.apache.paimon.flink.compact.MultiTableScanBase; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.flink.utils.JavaTypeInfo; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.Split; -import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.table.data.RowData; import java.util.regex.Pattern; @@ -44,14 +46,12 @@ import static org.apache.paimon.flink.compact.MultiTableScanBase.ScanResult.IS_EMPTY; /** It is responsible for monitoring compactor source of multi bucket table in stream mode. */ -public class CombinedAwareStreamingSourceFunction - extends CombinedCompactorSourceFunction> { +public class CombinedAwareStreamingSource extends CombinedCompactorSource> { private final long monitorInterval; - private transient MultiTableScanBase> tableScan; - public CombinedAwareStreamingSourceFunction( - Catalog.Loader catalogLoader, + public CombinedAwareStreamingSource( + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, @@ -60,39 +60,46 @@ public CombinedAwareStreamingSourceFunction( this.monitorInterval = monitorInterval; } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + @Override + public SourceReader, SimpleSourceSplit> createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. - */ - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiAwareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); - } + private class Reader extends AbstractNonCoordinatedSourceReader> { + private transient MultiTableScanBase> tableScan; - @SuppressWarnings("BusyWait") - @Override - void scanTable() throws Exception { - while (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + @Override + public void start() { + super.start(); + tableScan = + new MultiAwareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } + + @Override + public InputStatus pollNext(ReaderOutput> readerOutput) + throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { Thread.sleep(monitorInterval); } + return InputStatus.MORE_AVAILABLE; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } @@ -100,43 +107,28 @@ public static DataStream buildSource( StreamExecutionEnvironment env, String name, TypeInformation typeInfo, - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, long monitorInterval) { - CombinedAwareStreamingSourceFunction function = - new CombinedAwareStreamingSourceFunction( + CombinedAwareStreamingSource source = + new CombinedAwareStreamingSource( catalogLoader, includingPattern, excludingPattern, databasePattern, monitorInterval); - StreamSource, ?> sourceOperator = new StreamSource<>(function); - boolean isParallel = false; TupleTypeInfo> tupleTypeInfo = new TupleTypeInfo<>( new JavaTypeInfo<>(Split.class), BasicTypeInfo.STRING_TYPE_INFO); - return new DataStreamSource<>( - env, - tupleTypeInfo, - sourceOperator, - isParallel, - name, - Boundedness.CONTINUOUS_UNBOUNDED) + + return env.fromSource(source, WatermarkStrategy.noWatermarks(), name, tupleTypeInfo) .forceNonParallel() .partitionCustom( (key, numPartitions) -> key % numPartitions, split -> ((DataSplit) split.f0).bucket()) .transform(name, typeInfo, new MultiTablesReadOperator(catalogLoader, true)); } - - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java similarity index 58% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java index 02bb8786505d..e292d2441ccd 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java @@ -19,14 +19,12 @@ package org.apache.paimon.flink.source.operator; import org.apache.paimon.append.UnawareAppendCompactionTask; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; import org.apache.paimon.table.source.Split; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.api.connector.source.Boundedness; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; /** @@ -45,21 +43,18 @@ *

    Currently, only dedicated compaction job for multi-tables rely on this monitor. This is the * single (non-parallel) monitoring task, it is responsible for the new Paimon table. */ -public abstract class CombinedCompactorSourceFunction extends RichSourceFunction { +public abstract class CombinedCompactorSource extends AbstractNonCoordinatedSource { - private static final long serialVersionUID = 2L; + private static final long serialVersionUID = 3L; - protected final Catalog.Loader catalogLoader; + protected final CatalogLoader catalogLoader; protected final Pattern includingPattern; protected final Pattern excludingPattern; protected final Pattern databasePattern; protected final boolean isStreaming; - protected transient AtomicBoolean isRunning; - protected transient SourceContext ctx; - - public CombinedCompactorSourceFunction( - Catalog.Loader catalogLoader, + public CombinedCompactorSource( + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, @@ -71,37 +66,8 @@ public CombinedCompactorSourceFunction( this.isStreaming = isStreaming; } - /** - * 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 parameters) throws Exception { - isRunning = new AtomicBoolean(true); - } - @Override - public void run(SourceContext sourceContext) throws Exception { - this.ctx = sourceContext; - scanTable(); + public Boundedness getBoundedness() { + return isStreaming ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; } - - @Override - public void cancel() { - // this is to cover the case where cancel() is called before the run() - if (ctx != null) { - synchronized (ctx.getCheckpointLock()) { - isRunning.set(false); - } - } else { - isRunning.set(false); - } - } - - abstract void scanTable() throws Exception; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSource.java similarity index 70% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSource.java index 6a40f10ada61..5c0d9c42dd29 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSource.java @@ -20,23 +20,26 @@ import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.flink.compact.MultiTableScanBase; import org.apache.paimon.flink.compact.MultiUnawareBucketTableScan; import org.apache.paimon.flink.sink.MultiTableCompactionTaskTypeInfo; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.StreamSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,49 +58,47 @@ * It is responsible for the batch compactor source of the table with unaware bucket in combined * mode. */ -public class CombinedUnawareBatchSourceFunction - extends CombinedCompactorSourceFunction { +public class CombinedUnawareBatchSource + extends CombinedCompactorSource { - private static final Logger LOGGER = - LoggerFactory.getLogger(CombinedUnawareBatchSourceFunction.class); - private transient MultiTableScanBase tableScan; + private static final Logger LOGGER = LoggerFactory.getLogger(CombinedUnawareBatchSource.class); - public CombinedUnawareBatchSourceFunction( - Catalog.Loader catalogLoader, + public CombinedUnawareBatchSource( + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern) { super(catalogLoader, includingPattern, excludingPattern, databasePattern, false); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. - */ - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiUnawareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); - } + private class Reader + extends AbstractNonCoordinatedSourceReader { + private transient MultiTableScanBase tableScan; + + @Override + public void start() { + super.start(); + tableScan = + new MultiUnawareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } - @Override - void scanTable() throws Exception { - if (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + @Override + public InputStatus pollNext( + ReaderOutput readerOutput) throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { // Currently, in the combined mode, there are two scan tasks for the table of two @@ -106,33 +107,38 @@ void scanTable() throws Exception { // should not be thrown exception here. LOGGER.info("No file were collected for the table of unaware-bucket"); } + return InputStatus.END_OF_INPUT; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } public static DataStream buildSource( StreamExecutionEnvironment env, String name, - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, @Nullable Duration partitionIdleTime) { - CombinedUnawareBatchSourceFunction function = - new CombinedUnawareBatchSourceFunction( + CombinedUnawareBatchSource combinedUnawareBatchSource = + new CombinedUnawareBatchSource( catalogLoader, includingPattern, excludingPattern, databasePattern); - StreamSource - sourceOperator = new StreamSource<>(function); MultiTableCompactionTaskTypeInfo compactionTaskTypeInfo = new MultiTableCompactionTaskTypeInfo(); SingleOutputStreamOperator source = - new DataStreamSource<>( - env, - compactionTaskTypeInfo, - sourceOperator, - false, + env.fromSource( + combinedUnawareBatchSource, + WatermarkStrategy.noWatermarks(), name, - Boundedness.BOUNDED) + compactionTaskTypeInfo) .forceNonParallel(); if (partitionIdleTime != null) { @@ -177,12 +183,4 @@ private static Long getPartitionInfo( } return partitionInfo.get(partition); } - - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSource.java similarity index 55% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSource.java index b64518a7ef60..2e38d538a999 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSource.java @@ -19,18 +19,20 @@ package org.apache.paimon.flink.source.operator; import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; -import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.flink.compact.MultiTableScanBase; import org.apache.paimon.flink.compact.MultiUnawareBucketTableScan; import org.apache.paimon.flink.sink.MultiTableCompactionTaskTypeInfo; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.StreamSource; import java.util.regex.Pattern; @@ -40,14 +42,13 @@ /** * It is responsible for monitoring compactor source in stream mode for the table of unaware bucket. */ -public class CombinedUnawareStreamingSourceFunction - extends CombinedCompactorSourceFunction { +public class CombinedUnawareStreamingSource + extends CombinedCompactorSource { private final long monitorInterval; - private MultiTableScanBase tableScan; - public CombinedUnawareStreamingSourceFunction( - Catalog.Loader catalogLoader, + public CombinedUnawareStreamingSource( + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, @@ -56,78 +57,71 @@ public CombinedUnawareStreamingSourceFunction( this.monitorInterval = monitorInterval; } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. - */ - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiUnawareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); - } + private class Reader + extends AbstractNonCoordinatedSourceReader { + private MultiTableScanBase tableScan; - @SuppressWarnings("BusyWait") - @Override - void scanTable() throws Exception { - while (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + @Override + public void start() { + super.start(); + tableScan = + new MultiUnawareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } + + @Override + public InputStatus pollNext( + ReaderOutput readerOutput) throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { Thread.sleep(monitorInterval); } + return InputStatus.MORE_AVAILABLE; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } public static DataStream buildSource( StreamExecutionEnvironment env, String name, - Catalog.Loader catalogLoader, + CatalogLoader catalogLoader, Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, long monitorInterval) { - CombinedUnawareStreamingSourceFunction function = - new CombinedUnawareStreamingSourceFunction( + CombinedUnawareStreamingSource source = + new CombinedUnawareStreamingSource( catalogLoader, includingPattern, excludingPattern, databasePattern, monitorInterval); - StreamSource - sourceOperator = new StreamSource<>(function); - boolean isParallel = false; MultiTableCompactionTaskTypeInfo compactionTaskTypeInfo = new MultiTableCompactionTaskTypeInfo(); - return new DataStreamSource<>( - env, - compactionTaskTypeInfo, - sourceOperator, - isParallel, - name, - Boundedness.CONTINUOUS_UNBOUNDED) - .forceNonParallel(); - } - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } + return env.fromSource( + source, WatermarkStrategy.noWatermarks(), name, compactionTaskTypeInfo) + .forceNonParallel(); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java similarity index 53% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java index 3805f6f8c536..4ec0a4f99d9f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java @@ -18,6 +18,10 @@ package org.apache.paimon.flink.source.operator; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; +import org.apache.paimon.flink.source.SplitListState; import org.apache.paimon.flink.utils.JavaTypeInfo; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.sink.ChannelComputer; @@ -27,22 +31,18 @@ import org.apache.paimon.table.source.Split; import org.apache.paimon.table.source.StreamTableScan; -import org.apache.flink.api.common.state.CheckpointListener; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; -import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; -import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.table.data.RowData; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -71,33 +71,23 @@ *

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

      - *
    1. Consumer-id: rely on this function to do aligned snapshot consumption, and ensure that all + *
    2. Consumer-id: rely on this source to do aligned snapshot consumption, and ensure that all * data in a snapshot is consumed within each checkpoint. *
    3. Snapshot-watermark: when there is no watermark definition, the default Paimon table will * pass the watermark recorded in the snapshot. *
    */ -public class MonitorFunction extends RichSourceFunction - implements CheckpointedFunction, CheckpointListener { +public class MonitorSource extends AbstractNonCoordinatedSource { private static final long serialVersionUID = 1L; - private static final Logger LOG = LoggerFactory.getLogger(MonitorFunction.class); + private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); private final ReadBuilder readBuilder; private final long monitorInterval; private final boolean emitSnapshotWatermark; - private volatile boolean isRunning = true; - - private transient StreamTableScan scan; - private transient SourceContext ctx; - - private transient ListState checkpointState; - private transient ListState> nextSnapshotState; - private transient TreeMap nextSnapshotPerCheckpoint; - - public MonitorFunction( + public MonitorSource( ReadBuilder readBuilder, long monitorInterval, boolean emitSnapshotWatermark) { this.readBuilder = readBuilder; this.monitorInterval = monitorInterval; @@ -105,40 +95,74 @@ public MonitorFunction( } @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - this.scan = readBuilder.newStreamScan(); - - this.checkpointState = - context.getOperatorStateStore() - .getListState( - new ListStateDescriptor<>( - "next-snapshot", LongSerializer.INSTANCE)); - - @SuppressWarnings("unchecked") - final Class> typedTuple = - (Class>) (Class) Tuple2.class; - this.nextSnapshotState = - context.getOperatorStateStore() - .getListState( - new ListStateDescriptor<>( - "next-snapshot-per-checkpoint", - new TupleSerializer<>( - typedTuple, - new TypeSerializer[] { - LongSerializer.INSTANCE, LongSerializer.INSTANCE - }))); - - this.nextSnapshotPerCheckpoint = new TreeMap<>(); - - if (context.isRestored()) { - LOG.info("Restoring state for the {}.", getClass().getSimpleName()); + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); + } + + private class Reader extends AbstractNonCoordinatedSourceReader { + private static final String CHECKPOINT_STATE = "CS"; + private static final String NEXT_SNAPSHOT_STATE = "NSS"; + + private final StreamTableScan scan = readBuilder.newStreamScan(); + private final SplitListState checkpointState = + new SplitListState<>(CHECKPOINT_STATE, x -> Long.toString(x), Long::parseLong); + private final SplitListState> nextSnapshotState = + new SplitListState<>( + NEXT_SNAPSHOT_STATE, + x -> x.f0 + ":" + x.f1, + x -> + Tuple2.of( + Long.parseLong(x.split(":")[0]), + Long.parseLong(x.split(":")[1]))); + private final TreeMap nextSnapshotPerCheckpoint = new TreeMap<>(); + + @Override + public void notifyCheckpointComplete(long checkpointId) { + NavigableMap nextSnapshots = + nextSnapshotPerCheckpoint.headMap(checkpointId, true); + OptionalLong max = nextSnapshots.values().stream().mapToLong(Long::longValue).max(); + max.ifPresent(scan::notifyCheckpointComplete); + nextSnapshots.clear(); + } - List retrievedStates = new ArrayList<>(); - for (Long entry : this.checkpointState.get()) { - retrievedStates.add(entry); + @Override + public List snapshotState(long checkpointId) { + this.checkpointState.clear(); + Long nextSnapshot = this.scan.checkpoint(); + if (nextSnapshot != null) { + this.checkpointState.add(nextSnapshot); + this.nextSnapshotPerCheckpoint.put(checkpointId, nextSnapshot); } - // given that the parallelism of the function is 1, we can only have 1 retrieved items. + List> nextSnapshots = new ArrayList<>(); + this.nextSnapshotPerCheckpoint.forEach((k, v) -> nextSnapshots.add(new Tuple2<>(k, v))); + this.nextSnapshotState.update(nextSnapshots); + + if (LOG.isDebugEnabled()) { + LOG.debug("{} checkpoint {}.", getClass().getSimpleName(), nextSnapshot); + } + + List results = new ArrayList<>(); + results.addAll(checkpointState.snapshotState()); + results.addAll(nextSnapshotState.snapshotState()); + return results; + } + + @Override + public void addSplits(List list) { + LOG.info("Restoring state for the {}.", getClass().getSimpleName()); + checkpointState.restoreState(list); + nextSnapshotState.restoreState(list); + + List retrievedStates = checkpointState.get(); + + // given that the parallelism of the source is 1, we can only have 1 retrieved items. Preconditions.checkArgument( retrievedStates.size() <= 1, getClass().getSimpleName() + " retrieved invalid state."); @@ -150,80 +174,31 @@ public void initializeState(FunctionInitializationContext context) throws Except for (Tuple2 tuple2 : nextSnapshotState.get()) { nextSnapshotPerCheckpoint.put(tuple2.f0, tuple2.f1); } - } else { - LOG.info("No state to restore for the {}.", getClass().getSimpleName()); } - } - - @Override - public void snapshotState(FunctionSnapshotContext ctx) throws Exception { - this.checkpointState.clear(); - Long nextSnapshot = this.scan.checkpoint(); - if (nextSnapshot != null) { - this.checkpointState.add(nextSnapshot); - this.nextSnapshotPerCheckpoint.put(ctx.getCheckpointId(), nextSnapshot); - } - - List> nextSnapshots = new ArrayList<>(); - this.nextSnapshotPerCheckpoint.forEach((k, v) -> nextSnapshots.add(new Tuple2<>(k, v))); - this.nextSnapshotState.update(nextSnapshots); - if (LOG.isDebugEnabled()) { - LOG.debug("{} checkpoint {}.", getClass().getSimpleName(), nextSnapshot); - } - } - - @SuppressWarnings("BusyWait") - @Override - public void run(SourceContext ctx) throws Exception { - this.ctx = ctx; - while (isRunning) { + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { boolean isEmpty; - synchronized (ctx.getCheckpointLock()) { - if (!isRunning) { - return; - } - try { - List splits = scan.plan().splits(); - isEmpty = splits.isEmpty(); - splits.forEach(ctx::collect); - - if (emitSnapshotWatermark) { - Long watermark = scan.watermark(); - if (watermark != null) { - ctx.emitWatermark(new Watermark(watermark)); - } + try { + List splits = scan.plan().splits(); + isEmpty = splits.isEmpty(); + splits.forEach(readerOutput::collect); + + if (emitSnapshotWatermark) { + Long watermark = scan.watermark(); + if (watermark != null) { + readerOutput.emitWatermark(new Watermark(watermark)); } - } catch (EndOfScanException esf) { - LOG.info("Catching EndOfStreamException, the stream is finished."); - return; } + } catch (EndOfScanException esf) { + LOG.info("Catching EndOfStreamException, the stream is finished."); + return InputStatus.END_OF_INPUT; } if (isEmpty) { Thread.sleep(monitorInterval); } - } - } - - @Override - public void notifyCheckpointComplete(long checkpointId) { - NavigableMap nextSnapshots = - nextSnapshotPerCheckpoint.headMap(checkpointId, true); - OptionalLong max = nextSnapshots.values().stream().mapToLong(Long::longValue).max(); - max.ifPresent(scan::notifyCheckpointComplete); - nextSnapshots.clear(); - } - - @Override - public void cancel() { - // this is to cover the case where cancel() is called before the run() - if (ctx != null) { - synchronized (ctx.getCheckpointLock()) { - isRunning = false; - } - } else { - isRunning = false; + return InputStatus.MORE_AVAILABLE; } } @@ -237,9 +212,10 @@ public static DataStream buildSource( boolean shuffleBucketWithPartition, BucketMode bucketMode) { SingleOutputStreamOperator singleOutputStreamOperator = - env.addSource( - new MonitorFunction( + env.fromSource( + new MonitorSource( readBuilder, monitorInterval, emitSnapshotWatermark), + WatermarkStrategy.noWatermarks(), name + "-Monitor", new JavaTypeInfo<>(Split.class)) .forceNonParallel(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java index 73d46ae1e3f1..ae3099ec0628 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.source.operator; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; @@ -52,27 +53,26 @@ /** * The operator that reads the Tuple2<{@link Split}, String> received from the preceding {@link - * CombinedAwareBatchSourceFunction} or {@link CombinedAwareStreamingSourceFunction}. Contrary to - * the {@link CombinedCompactorSourceFunction} which has a parallelism of 1, this operator can have - * DOP > 1. + * CombinedAwareBatchSource} or {@link CombinedAwareStreamingSource}. Contrary to the {@link + * CombinedCompactorSource} which has a parallelism of 1, this operator can have DOP > 1. */ public class MultiTablesReadOperator extends AbstractStreamOperator implements OneInputStreamOperator, RowData> { private static final long serialVersionUID = 1L; - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; private final boolean isStreaming; private Duration partitionIdleTime = null; - public MultiTablesReadOperator(Catalog.Loader catalogLoader, boolean isStreaming) { + public MultiTablesReadOperator(CatalogLoader catalogLoader, boolean isStreaming) { this.catalogLoader = catalogLoader; this.isStreaming = isStreaming; } public MultiTablesReadOperator( - Catalog.Loader catalogLoader, boolean isStreaming, Duration partitionIdleTime) { + CatalogLoader catalogLoader, boolean isStreaming, Duration partitionIdleTime) { this.catalogLoader = catalogLoader; this.isStreaming = isStreaming; this.partitionIdleTime = partitionIdleTime; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java index c501c2519b41..15fde93755fd 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java @@ -20,6 +20,7 @@ import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.manifest.PartitionEntry; @@ -44,7 +45,7 @@ /** * The operator is used for historical partition compaction. It reads {@link * MultiTableUnawareAppendCompactionTask} received from the preceding {@link - * CombinedUnawareBatchSourceFunction} and filter partitions which is not historical. + * CombinedUnawareBatchSource} and filter partitions which is not historical. */ public class MultiUnawareTablesReadOperator extends AbstractStreamOperator @@ -54,12 +55,11 @@ public class MultiUnawareTablesReadOperator private static final Logger LOG = LoggerFactory.getLogger(MultiUnawareTablesReadOperator.class); - private final Catalog.Loader catalogLoader; + private final CatalogLoader catalogLoader; private final Duration partitionIdleTime; - public MultiUnawareTablesReadOperator( - Catalog.Loader catalogLoader, Duration partitionIdleTime) { + public MultiUnawareTablesReadOperator(CatalogLoader catalogLoader, Duration partitionIdleTime) { this.catalogLoader = catalogLoader; this.partitionIdleTime = partitionIdleTime; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java index d884724c6749..ccc66194560e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java @@ -38,8 +38,8 @@ /** * The operator that reads the {@link Split splits} received from the preceding {@link - * MonitorFunction}. Contrary to the {@link MonitorFunction} which has a parallelism of 1, this - * operator can have DOP > 1. + * MonitorSource}. Contrary to the {@link MonitorSource} which has a parallelism of 1, this operator + * can have DOP > 1. */ public class ReadOperator extends AbstractStreamOperator implements OneInputStreamOperator { diff --git a/paimon-flink/paimon-flink-common/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-flink/paimon-flink-common/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index 6c3b0e7664c0..6251189560f6 100644 --- a/paimon-flink/paimon-flink-common/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-flink/paimon-flink-common/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -69,6 +69,7 @@ org.apache.paimon.flink.procedure.RemoveOrphanFilesProcedure org.apache.paimon.flink.procedure.QueryServiceProcedure org.apache.paimon.flink.procedure.ExpireSnapshotsProcedure org.apache.paimon.flink.procedure.ExpirePartitionsProcedure +org.apache.paimon.flink.procedure.PurgeFilesProcedure org.apache.paimon.flink.procedure.privilege.InitFileBasedPrivilegeProcedure org.apache.paimon.flink.procedure.privilege.CreatePrivilegedUserProcedure org.apache.paimon.flink.procedure.privilege.DropPrivilegedUserProcedure diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java index cdc114b048a1..d48b6e771236 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java @@ -561,19 +561,35 @@ public void testCountStarAppendWithDv() { String sql = "SELECT COUNT(*) FROM count_append_dv"; assertThat(sql(sql)).containsOnly(Row.of(2L)); - validateCount1NotPushDown(sql); + validateCount1PushDown(sql); } @Test public void testCountStarPK() { - sql("CREATE TABLE count_pk (f0 INT PRIMARY KEY NOT ENFORCED, f1 STRING)"); - sql("INSERT INTO count_pk VALUES (1, 'a'), (2, 'b')"); + sql( + "CREATE TABLE count_pk (f0 INT PRIMARY KEY NOT ENFORCED, f1 STRING) WITH ('file.format' = 'avro')"); + sql("INSERT INTO count_pk VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')"); + sql("INSERT INTO count_pk VALUES (1, 'e')"); String sql = "SELECT COUNT(*) FROM count_pk"; - assertThat(sql(sql)).containsOnly(Row.of(2L)); + assertThat(sql(sql)).containsOnly(Row.of(4L)); validateCount1NotPushDown(sql); } + @Test + public void testCountStarPKDv() { + sql( + "CREATE TABLE count_pk_dv (f0 INT PRIMARY KEY NOT ENFORCED, f1 STRING) WITH (" + + "'file.format' = 'avro', " + + "'deletion-vectors.enabled' = 'true')"); + sql("INSERT INTO count_pk_dv VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')"); + sql("INSERT INTO count_pk_dv VALUES (1, 'e')"); + + String sql = "SELECT COUNT(*) FROM count_pk_dv"; + assertThat(sql(sql)).containsOnly(Row.of(4L)); + validateCount1PushDown(sql); + } + @Test public void testParquetRowDecimalAndTimestamp() { sql( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java index 6a2c7b071d2d..5245114e80ee 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java @@ -36,6 +36,7 @@ import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.FailingFileIO; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.dag.Transformation; @@ -450,7 +451,12 @@ private void sinkAndValidate( throw new UnsupportedOperationException(); } DataStreamSource source = - env.addSource(new FiniteTestSource<>(src, true), InternalTypeInfo.of(TABLE_TYPE)); + env.fromSource( + new FiniteTestSource<>(src, true), + WatermarkStrategy.noWatermarks(), + "FiniteTestSource", + InternalTypeInfo.of(TABLE_TYPE)); + source.forceNonParallel(); new FlinkSinkBuilder(table).forRowData(source).build(); env.execute(); assertThat(iterator.collect(expected.length)).containsExactlyInAnyOrder(expected); @@ -521,9 +527,13 @@ public static DataStreamSource buildTestSource( StreamExecutionEnvironment env, boolean isBatch) { return isBatch ? env.fromCollection(SOURCE_DATA, InternalTypeInfo.of(TABLE_TYPE)) - : env.addSource( - new FiniteTestSource<>(SOURCE_DATA, false), - InternalTypeInfo.of(TABLE_TYPE)); + : (DataStreamSource) + env.fromSource( + new FiniteTestSource<>(SOURCE_DATA, false), + WatermarkStrategy.noWatermarks(), + "FiniteTestSource", + InternalTypeInfo.of(TABLE_TYPE)) + .forceNonParallel(); } public static List executeAndCollect(DataStream source) throws Exception { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java index 9c5254d6283b..6691b9c09514 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java @@ -18,16 +18,18 @@ package org.apache.paimon.flink; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; +import org.apache.paimon.flink.source.SplitListState; import org.apache.paimon.utils.Preconditions; -import org.apache.flink.api.common.state.CheckpointListener; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeutils.base.IntSerializer; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; -import org.apache.flink.streaming.api.functions.source.SourceFunction; import java.util.ArrayList; import java.util.List; @@ -39,8 +41,7 @@ * *

    The reason this class is rewritten is to support {@link CheckpointedFunction}. */ -public class FiniteTestSource - implements SourceFunction, CheckpointedFunction, CheckpointListener { +public class FiniteTestSource extends AbstractNonCoordinatedSource { private static final long serialVersionUID = 1L; @@ -48,27 +49,78 @@ public class FiniteTestSource private final boolean emitOnce; - private volatile boolean running = true; - - private transient int numCheckpointsComplete; - - private transient ListState checkpointedState; - - private volatile int numTimesEmitted; - public FiniteTestSource(List elements, boolean emitOnce) { this.elements = elements; this.emitOnce = emitOnce; } @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - this.checkpointedState = - context.getOperatorStateStore() - .getListState( - new ListStateDescriptor<>("emit-times", IntSerializer.INSTANCE)); + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public SourceReader createReader(SourceReaderContext sourceReaderContext) + throws Exception { + return new Reader<>(elements, emitOnce); + } + + private static class Reader extends AbstractNonCoordinatedSourceReader { + + private final List elements; + + private final boolean emitOnce; + + private final SplitListState checkpointedState = + new SplitListState<>("emit-times", x -> Integer.toString(x), Integer::parseInt); + + private int numTimesEmitted = 0; + + private int numCheckpointsComplete; + + private Integer checkpointToAwait; + + private Reader(List elements, boolean emitOnce) { + this.elements = elements; + this.emitOnce = emitOnce; + this.numCheckpointsComplete = 0; + } + + @Override + public synchronized InputStatus pollNext(ReaderOutput readerOutput) { + if (checkpointToAwait == null) { + checkpointToAwait = numCheckpointsComplete + 2; + } + switch (numTimesEmitted) { + case 0: + emitElements(readerOutput, false); + if (numCheckpointsComplete < checkpointToAwait) { + return InputStatus.MORE_AVAILABLE; + } + emitElements(readerOutput, true); + if (numCheckpointsComplete < checkpointToAwait + 2) { + return InputStatus.MORE_AVAILABLE; + } + break; + case 1: + emitElements(readerOutput, true); + if (numCheckpointsComplete < checkpointToAwait) { + return InputStatus.MORE_AVAILABLE; + } + break; + case 2: + // Maybe missed notifyCheckpointComplete, wait next notifyCheckpointComplete + if (numCheckpointsComplete < checkpointToAwait) { + return InputStatus.MORE_AVAILABLE; + } + break; + } + return InputStatus.END_OF_INPUT; + } - if (context.isRestored()) { + @Override + public void addSplits(List list) { + checkpointedState.restoreState(list); List retrievedStates = new ArrayList<>(); for (Integer entry : this.checkpointedState.get()) { retrievedStates.add(entry); @@ -85,76 +137,27 @@ public void initializeState(FunctionInitializationContext context) throws Except getClass().getSimpleName() + " retrieved invalid numTimesEmitted: " + numTimesEmitted); - } else { - this.numTimesEmitted = 0; } - } - @Override - public void run(SourceContext ctx) throws Exception { - switch (numTimesEmitted) { - case 0: - emitElementsAndWaitForCheckpoints(ctx, false); - emitElementsAndWaitForCheckpoints(ctx, true); - break; - case 1: - emitElementsAndWaitForCheckpoints(ctx, true); - break; - case 2: - // Maybe missed notifyCheckpointComplete, wait next notifyCheckpointComplete - final Object lock = ctx.getCheckpointLock(); - synchronized (lock) { - int checkpointToAwait = numCheckpointsComplete + 2; - while (running && numCheckpointsComplete < checkpointToAwait) { - lock.wait(1); - } - } - break; + @Override + public List snapshotState(long l) { + this.checkpointedState.clear(); + this.checkpointedState.add(this.numTimesEmitted); + return this.checkpointedState.snapshotState(); } - } - private void emitElementsAndWaitForCheckpoints(SourceContext ctx, boolean isSecond) - throws InterruptedException { - final Object lock = ctx.getCheckpointLock(); + @Override + public void notifyCheckpointComplete(long checkpointId) { + numCheckpointsComplete++; + } - final int checkpointToAwait; - synchronized (lock) { - checkpointToAwait = numCheckpointsComplete + 2; + private void emitElements(ReaderOutput readerOutput, boolean isSecond) { if (!isSecond || !emitOnce) { for (T t : elements) { - ctx.collect(t); + readerOutput.collect(t); } } numTimesEmitted++; } - - synchronized (lock) { - while (running && numCheckpointsComplete < checkpointToAwait) { - lock.wait(1); - } - } - } - - @Override - public void cancel() { - running = false; - } - - @Override - public void notifyCheckpointComplete(long checkpointId) { - numCheckpointsComplete++; - } - - @Override - public void notifyCheckpointAborted(long checkpointId) {} - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - Preconditions.checkState( - this.checkpointedState != null, - "The " + getClass().getSimpleName() + " has not been properly initialized."); - - this.checkpointedState.clear(); - this.checkpointedState.add(this.numTimesEmitted); } } 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 fb36f7ad77b1..0b68d9ecf6ff 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 @@ -109,6 +109,7 @@ /** Test for {@link FlinkCatalog}. */ public class FlinkCatalogTest { + private static final String TESTING_LOG_STORE = "testing"; private final ObjectPath path1 = new ObjectPath("db1", "t1"); @@ -355,12 +356,7 @@ public void testCreateFlinkTableWithPath() throws Exception { CatalogTable table1 = createTable(options); assertThatThrownBy(() -> catalog.createTable(this.path1, table1, false)) .hasMessageContaining( - "You specified the Path when creating the table, " - + "but the Path '/unknown/path' is different from where it should be"); - - options.put(PATH.key(), warehouse + "/db1.db/t1"); - CatalogTable table2 = createTable(options); - catalog.createTable(this.path1, table2, false); + "The current catalog FileSystemCatalog does not support specifying the table path when creating a table."); } @ParameterizedTest 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 10de1ae4839f..732e96454236 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 @@ -801,6 +801,43 @@ public void testStreamingReadOverwriteWithoutPartitionedRecords() throws Excepti streamingItr.close(); } + @Test + public void testStreamingReadOverwriteWithDeleteRecords() throws Exception { + String table = + createTable( + Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), + Collections.singletonList("currency"), + Collections.emptyList(), + Collections.emptyList(), + streamingReadOverwrite); + + insertInto( + table, + "('US Dollar', 102, '2022-01-01')", + "('Yen', 1, '2022-01-02')", + "('Euro', 119, '2022-01-02')"); + + bEnv.executeSql(String.format("DELETE FROM %s WHERE currency = 'Euro'", table)).await(); + + checkFileStorePath(table, Collections.emptyList()); + + // test projection and filter + BlockingIterator streamingItr = + testStreamingRead( + buildQuery(table, "currency, rate", "WHERE dt = '2022-01-02'"), + Collections.singletonList(changelogRow("+I", "Yen", 1L))); + + insertOverwrite(table, "('US Dollar', 100, '2022-01-02')", "('Yen', 10, '2022-01-01')"); + + validateStreamingReadResult( + streamingItr, + Arrays.asList( + changelogRow("-D", "Yen", 1L), changelogRow("+I", "US Dollar", 100L))); + assertNoMoreRecords(streamingItr); + + streamingItr.close(); + } + @Test public void testUnsupportStreamingReadOverwriteWithoutPk() { assertThatThrownBy( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java index 594affc124eb..75b96cbe02eb 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java @@ -47,8 +47,10 @@ public SerializableRowData(RowData row, TypeSerializer serializer) { this.serializer = serializer; } - private void writeObject(ObjectOutputStream out) throws IOException { + private synchronized void writeObject(ObjectOutputStream out) throws IOException { out.defaultWriteObject(); + // This following invocation needs to be synchronized to avoid racing problems when the + // serializer is reused across multiple subtasks. serializer.serialize(row, new DataOutputViewStreamWrapper(out)); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java index 6ca78b088fb7..fb8bee5d5962 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java @@ -20,7 +20,9 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.flink.utils.RuntimeContextUtils; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.reader.RecordReader; @@ -30,9 +32,14 @@ import org.apache.paimon.utils.FailingFileIO; import org.apache.paimon.utils.TimeUtils; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.planner.factories.TestValuesTableFactory; import org.apache.flink.types.Row; @@ -380,7 +387,12 @@ public void testStatelessWriter() throws Exception { .checkpointIntervalMs(500) .build(); DataStream source = - env.addSource(new TestStatelessWriterSource(table)).setParallelism(2).forward(); + env.fromSource( + new TestStatelessWriterSource(table), + WatermarkStrategy.noWatermarks(), + "TestStatelessWriterSource") + .setParallelism(2) + .forward(); StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); tEnv.registerCatalog("mycat", sEnv.getCatalog("PAIMON").get()); @@ -392,46 +404,59 @@ public void testStatelessWriter() throws Exception { .containsExactlyInAnyOrder(Row.of(1, "test"), Row.of(2, "test")); } - private static class TestStatelessWriterSource extends RichParallelSourceFunction { + private static class TestStatelessWriterSource extends AbstractNonCoordinatedSource { private final FileStoreTable table; - private volatile boolean isRunning = true; - private TestStatelessWriterSource(FileStoreTable table) { this.table = table; } @Override - public void run(SourceContext sourceContext) throws Exception { - int taskId = RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()); - // wait some time in parallelism #2, - // so that it does not commit in the same checkpoint with parallelism #1 - int waitCount = (taskId == 0 ? 0 : 10); - - while (isRunning) { - synchronized (sourceContext.getCheckpointLock()) { - if (taskId == 0) { + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(sourceReaderContext.getIndexOfSubtask()); + } + + private class Reader extends AbstractNonCoordinatedSourceReader { + private final int taskId; + private int waitCount; + + private Reader(int taskId) { + this.taskId = taskId; + this.waitCount = (taskId == 0 ? 0 : 10); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + if (taskId == 0) { + if (waitCount == 0) { + readerOutput.collect(1); + } else if (countNumRecords() >= 1) { + // wait for the record to commit before exiting + Thread.sleep(1000); + return InputStatus.END_OF_INPUT; + } + } else { + int numRecords = countNumRecords(); + if (numRecords >= 1) { if (waitCount == 0) { - sourceContext.collect(1); - } else if (countNumRecords() >= 1) { - // wait for the record to commit before exiting - break; - } - } else { - int numRecords = countNumRecords(); - if (numRecords >= 1) { - if (waitCount == 0) { - sourceContext.collect(2); - } else if (countNumRecords() >= 2) { - // make sure the next checkpoint is successful - break; - } + readerOutput.collect(2); + } else if (countNumRecords() >= 2) { + // make sure the next checkpoint is successful + Thread.sleep(1000); + return InputStatus.END_OF_INPUT; } } - waitCount--; } + waitCount--; Thread.sleep(1000); + return InputStatus.MORE_AVAILABLE; } } @@ -447,11 +472,6 @@ private int countNumRecords() throws Exception { } return ret; } - - @Override - public void cancel() { - isRunning = false; - } } @Override diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/PurgeFilesProcedureITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/PurgeFilesProcedureITCase.java new file mode 100644 index 000000000000..9eb9aad2969e --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/PurgeFilesProcedureITCase.java @@ -0,0 +1,48 @@ +/* + * 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.procedure; + +import org.apache.paimon.flink.CatalogITCaseBase; + +import org.apache.flink.types.Row; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** IT Case for {@link PurgeFilesProcedure}. */ +public class PurgeFilesProcedureITCase extends CatalogITCaseBase { + + @Test + public void testPurgeFiles() throws Exception { + sql( + "CREATE TABLE T (id INT, name STRING," + + " PRIMARY KEY (id) NOT ENFORCED)" + + " WITH ('bucket'='1')"); + + sql("INSERT INTO T VALUES (1, 'a')"); + assertThat(sql("select * from `T`")).containsExactly(Row.of(1, "a")); + + sql("INSERT INTO T VALUES (1, 'a')"); + sql("CALL sys.purge_files(`table` => 'default.T')"); + assertThat(sql("select * from `T`")).containsExactly(); + + sql("INSERT INTO T VALUES (2, 'a')"); + assertThat(sql("select * from `T`")).containsExactly(Row.of(2, "a")); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java index d487d75925eb..0e85f559d9de 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java @@ -20,6 +20,7 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.BinaryRowWriter; @@ -273,7 +274,7 @@ protected StoreCompactOperator.Factory createCompactOperator(FileStoreTable tabl } protected MultiTablesStoreCompactOperator.Factory createMultiTablesCompactOperator( - Catalog.Loader catalogLoader) throws Exception { + CatalogLoader catalogLoader) throws Exception { return new MultiTablesStoreCompactOperator.Factory( catalogLoader, commitUser, 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 752679fb5903..53e3a6dcb79c 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 @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.BinaryString; @@ -78,7 +79,7 @@ class StoreMultiCommitterTest { private String initialCommitUser; private Path warehouse; - private Catalog.Loader catalogLoader; + private CatalogLoader catalogLoader; private Catalog catalog; private Identifier firstTable; private Identifier secondTable; @@ -691,7 +692,7 @@ public void snapshotState( return harness; } - private Catalog.Loader createCatalogLoader() { + private CatalogLoader createCatalogLoader() { Options catalogOptions = createCatalogOptions(warehouse); return () -> CatalogFactory.createCatalog(CatalogContext.create(catalogOptions)); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java index 19c22d137c7f..fca5dcf0ed69 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java @@ -66,7 +66,8 @@ public void markDone(LinkedHashMap partitionSpec) public void alterPartition( LinkedHashMap partitionSpec, Map parameters, - long modifyTime) + long modifyTime, + boolean ignoreIfNotExist) throws Exception { throw new UnsupportedOperationException(); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java index f0f4596c61bb..9e5fe7ff9ff7 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java @@ -83,7 +83,7 @@ private void innerTest(boolean deletionVectors) throws Exception { .build(); catalog.createTable(identifier, schema, true); FileStoreTable table = (FileStoreTable) catalog.getTable(identifier); - Path location = catalog.getTableLocation(identifier); + Path location = table.location(); Path successFile = new Path(location, "a=0/_SUCCESS"); PartitionMarkDone markDone = PartitionMarkDone.create(false, false, new MockOperatorStateStore(), table).get(); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/HmsReporterTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporterTest.java similarity index 93% rename from paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/HmsReporterTest.java rename to paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporterTest.java index 0050f3083a8e..142a0c32f781 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/HmsReporterTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionStatisticsReporterTest.java @@ -49,8 +49,8 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; -/** Test for {@link HmsReporter}. */ -public class HmsReporterTest { +/** Test for {@link PartitionStatisticsReporter}. */ +public class PartitionStatisticsReporterTest { @TempDir java.nio.file.Path tempDir; @@ -117,7 +117,8 @@ public void markDone(LinkedHashMap partitionSpec) public void alterPartition( LinkedHashMap partitionSpec, Map parameters, - long modifyTime) + long modifyTime, + boolean ignoreIfNotExist) throws Exception { partitionParams.put( PartitionPathUtils.generatePartitionPath(partitionSpec), @@ -130,7 +131,7 @@ public void close() throws Exception { } }; - HmsReporter action = new HmsReporter(table, client); + PartitionStatisticsReporter action = new PartitionStatisticsReporter(table, client); long time = 1729598544974L; action.report("c1=a/", time); Assertions.assertThat(partitionParams).containsKey("c1=a/"); @@ -143,6 +144,8 @@ public void close() throws Exception { "591", "numRows", "1", + "lastUpdateTime", + String.valueOf(time / 1000), "transient_lastDdlTime", String.valueOf(time / 1000))); action.close(); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/MultiTablesCompactorSourceBuilderITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/MultiTablesCompactorSourceBuilderITCase.java index fba5f33807de..3b41f39431bd 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/MultiTablesCompactorSourceBuilderITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/MultiTablesCompactorSourceBuilderITCase.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.Snapshot; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLoader; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.BinaryRowWriter; @@ -662,7 +663,7 @@ private BinaryRow binaryRow(String dt, int hh) { return b; } - private Catalog.Loader catalogLoader() { + private CatalogLoader catalogLoader() { // to make the action workflow serializable catalogOptions.set(CatalogOptions.WAREHOUSE, warehouse); return () -> FlinkCatalogFactory.createPaimonCatalog(catalogOptions); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java index 0bce8c8901ea..0cd969707cfa 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java @@ -33,12 +33,17 @@ import org.apache.paimon.table.source.TableRead; import org.apache.paimon.types.DataTypes; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.SourceOperator; import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.data.GenericRowData; @@ -46,6 +51,7 @@ import org.apache.flink.table.runtime.typeutils.InternalSerializers; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.function.SupplierWithException; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -58,11 +64,13 @@ import java.util.List; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import static org.apache.paimon.CoreOptions.CONSUMER_ID; import static org.assertj.core.api.Assertions.assertThat; -/** Test for {@link MonitorFunction} and {@link ReadOperator}. */ +/** Test for {@link MonitorSource} and {@link ReadOperator}. */ public class OperatorSourceTest { @TempDir Path tempDir; @@ -114,28 +122,39 @@ private List> readSplit(Split split) throws IOException { } @Test - public void testMonitorFunction() throws Exception { + public void testMonitorSource() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); // 1. run first OperatorSubtaskState snapshot; { - MonitorFunction function = new MonitorFunction(table.newReadBuilder(), 10, false); - StreamSource src = new StreamSource<>(function); + MonitorSource source = new MonitorSource(table.newReadBuilder(), 10, false); + TestingSourceOperator operator = + (TestingSourceOperator) + TestingSourceOperator.createTestOperator( + source.createReader(null), + WatermarkStrategy.noWatermarks(), + false); AbstractStreamOperatorTestHarness testHarness = - new AbstractStreamOperatorTestHarness<>(src, 1, 1, 0); + new AbstractStreamOperatorTestHarness<>(operator, 1, 1, 0); testHarness.open(); - snapshot = testReadSplit(function, () -> testHarness.snapshot(0, 0), 1, 1, 1); + snapshot = testReadSplit(operator, () -> testHarness.snapshot(0, 0), 1, 1, 1); } // 2. restore from state { - MonitorFunction functionCopy1 = new MonitorFunction(table.newReadBuilder(), 10, false); - StreamSource srcCopy1 = new StreamSource<>(functionCopy1); + MonitorSource sourceCopy1 = new MonitorSource(table.newReadBuilder(), 10, false); + TestingSourceOperator operatorCopy1 = + (TestingSourceOperator) + TestingSourceOperator.createTestOperator( + sourceCopy1.createReader(null), + WatermarkStrategy.noWatermarks(), + false); AbstractStreamOperatorTestHarness testHarnessCopy1 = - new AbstractStreamOperatorTestHarness<>(srcCopy1, 1, 1, 0); + new AbstractStreamOperatorTestHarness<>(operatorCopy1, 1, 1, 0); testHarnessCopy1.initializeState(snapshot); testHarnessCopy1.open(); testReadSplit( - functionCopy1, + operatorCopy1, () -> { testHarnessCopy1.snapshot(1, 1); testHarnessCopy1.notifyOfCompletedCheckpoint(1); @@ -148,12 +167,17 @@ public void testMonitorFunction() throws Exception { // 3. restore from consumer id { - MonitorFunction functionCopy2 = new MonitorFunction(table.newReadBuilder(), 10, false); - StreamSource srcCopy2 = new StreamSource<>(functionCopy2); + MonitorSource sourceCopy2 = new MonitorSource(table.newReadBuilder(), 10, false); + TestingSourceOperator operatorCopy2 = + (TestingSourceOperator) + TestingSourceOperator.createTestOperator( + sourceCopy2.createReader(null), + WatermarkStrategy.noWatermarks(), + false); AbstractStreamOperatorTestHarness testHarnessCopy2 = - new AbstractStreamOperatorTestHarness<>(srcCopy2, 1, 1, 0); + new AbstractStreamOperatorTestHarness<>(operatorCopy2, 1, 1, 0); testHarnessCopy2.open(); - testReadSplit(functionCopy2, () -> null, 3, 3, 3); + testReadSplit(operatorCopy2, () -> null, 3, 3, 3); } } @@ -247,7 +271,7 @@ public void testReadOperatorMetricsRegisterAndUpdate() throws Exception { } private T testReadSplit( - MonitorFunction function, + SourceOperator operator, SupplierWithException beforeClose, int a, int b, @@ -255,20 +279,36 @@ private T testReadSplit( throws Exception { Throwable[] error = new Throwable[1]; ArrayBlockingQueue queue = new ArrayBlockingQueue<>(10); + AtomicReference> iteratorRef = new AtomicReference<>(); - DummySourceContext sourceContext = - new DummySourceContext() { + PushingAsyncDataInput.DataOutput output = + new PushingAsyncDataInput.DataOutput() { @Override - public void collect(Split element) { - queue.add(element); + public void emitRecord(StreamRecord streamRecord) { + queue.add(streamRecord.getValue()); } + + @Override + public void emitWatermark(Watermark watermark) {} + + @Override + public void emitWatermarkStatus(WatermarkStatus watermarkStatus) {} + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) {} + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) {} }; + AtomicBoolean isRunning = new AtomicBoolean(true); Thread runner = new Thread( () -> { try { - function.run(sourceContext); + while (isRunning.get()) { + operator.emitNext(output); + } } catch (Throwable t) { t.printStackTrace(); error[0] = t; @@ -282,34 +322,15 @@ public void collect(Split element) { assertThat(readSplit(split)).containsExactlyInAnyOrder(Arrays.asList(a, b, c)); T t = beforeClose.get(); - function.cancel(); + CloseableIterator iterator = iteratorRef.get(); + if (iterator != null) { + iterator.close(); + } + isRunning.set(false); runner.join(); assertThat(error[0]).isNull(); return t; } - - private abstract static class DummySourceContext - implements SourceFunction.SourceContext { - - private final Object lock = new Object(); - - @Override - public void collectWithTimestamp(Split element, long timestamp) {} - - @Override - public void emitWatermark(Watermark mark) {} - - @Override - public void markAsTemporarilyIdle() {} - - @Override - public Object getCheckpointLock() { - return lock; - } - - @Override - public void close() {} - } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/TestingSourceOperator.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/TestingSourceOperator.java new file mode 100644 index 000000000000..77b44d5b0e5c --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/TestingSourceOperator.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.source.operator; + +import org.apache.paimon.flink.source.SimpleSourceSplit; +import org.apache.paimon.flink.source.SimpleSourceSplitSerializer; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.OperatorStateBackendParametersImpl; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateInitializationContextImpl; +import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; +import org.apache.flink.streaming.api.operators.SourceOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; +import org.apache.flink.streaming.util.MockStreamingRuntimeContext; + +import java.util.ArrayList; +import java.util.Collections; + +/** + * A SourceOperator extension to simplify test setup. + * + *

    This class is implemented in reference to {@link + * org.apache.flink.streaming.api.operators.source.TestingSourceOperator}. + * + *

    See Flink + * PR that introduced this class + */ +public class TestingSourceOperator extends SourceOperator { + + private static final long serialVersionUID = 1L; + + private final int subtaskIndex; + private final int parallelism; + + public TestingSourceOperator( + StreamOperatorParameters parameters, + SourceReader reader, + WatermarkStrategy watermarkStrategy, + ProcessingTimeService timeService, + boolean emitProgressiveWatermarks) { + + this( + parameters, + reader, + watermarkStrategy, + timeService, + new TestingOperatorEventGateway(), + 1, + 5, + emitProgressiveWatermarks); + } + + public TestingSourceOperator( + StreamOperatorParameters parameters, + SourceReader reader, + WatermarkStrategy watermarkStrategy, + ProcessingTimeService timeService, + OperatorEventGateway eventGateway, + int subtaskIndex, + int parallelism, + boolean emitProgressiveWatermarks) { + + super( + (context) -> reader, + eventGateway, + new SimpleSourceSplitSerializer(), + watermarkStrategy, + timeService, + new Configuration(), + "localhost", + emitProgressiveWatermarks, + () -> false); + + this.subtaskIndex = subtaskIndex; + this.parallelism = parallelism; + this.metrics = UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup(); + initSourceMetricGroup(); + + // unchecked wrapping is okay to keep tests simpler + try { + initReader(); + } catch (Exception e) { + throw new RuntimeException(e); + } + + setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + } + + @Override + public StreamingRuntimeContext getRuntimeContext() { + return new MockStreamingRuntimeContext(false, parallelism, subtaskIndex); + } + + // this is overridden to avoid complex mock injection through the "containingTask" + @Override + public ExecutionConfig getExecutionConfig() { + ExecutionConfig cfg = new ExecutionConfig(); + cfg.setAutoWatermarkInterval(100); + return cfg; + } + + public static SourceOperator createTestOperator( + SourceReader reader, + WatermarkStrategy watermarkStrategy, + boolean emitProgressiveWatermarks) + throws Exception { + + AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); + Environment env = new MockEnvironmentBuilder().build(); + CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); + final OperatorStateStore operatorStateStore = + abstractStateBackend.createOperatorStateBackend( + new OperatorStateBackendParametersImpl( + env, + "test-operator", + Collections.emptyList(), + cancelStreamRegistry)); + + final StateInitializationContext stateContext = + new StateInitializationContextImpl(null, operatorStateStore, null, null, null); + + TestProcessingTimeService timeService = new TestProcessingTimeService(); + timeService.setCurrentTime(Integer.MAX_VALUE); // start somewhere that is not zero + + final SourceOperator sourceOperator = + new TestingSourceOperator<>( + new StreamOperatorParameters<>( + new SourceOperatorStreamTask(new DummyEnvironment()), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>()), + null, + null, + null), + reader, + watermarkStrategy, + timeService, + emitProgressiveWatermarks); + sourceOperator.initializeState(stateContext); + sourceOperator.open(); + + return sourceOperator; + } + + private static class TestingOperatorEventGateway implements OperatorEventGateway { + @Override + public void sendEventToCoordinator(OperatorEvent event) {} + } +} 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 826bf28d1248..42a47ea1e298 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 @@ -153,7 +153,7 @@ public void testTableFilterPartitionStatistics() throws Exception { null, null, null, - false); + null); Assertions.assertThat(partitionFilterSource.reportStatistics().getRowCount()).isEqualTo(5L); Map> colStatsMap = new HashMap<>(); colStatsMap.put("pt", ColStats.newColStats(0, 1L, 1, 2, 0L, null, null)); @@ -232,7 +232,7 @@ public void testTableFilterKeyStatistics() throws Exception { null, null, null, - false); + null); Assertions.assertThat(keyFilterSource.reportStatistics().getRowCount()).isEqualTo(2L); Map> colStatsMap = new HashMap<>(); colStatsMap.put("pt", ColStats.newColStats(0, 1L, 2, 2, 0L, null, null)); @@ -311,7 +311,7 @@ public void testTableFilterValueStatistics() throws Exception { null, null, null, - false); + null); Assertions.assertThat(keyFilterSource.reportStatistics().getRowCount()).isEqualTo(4L); Map> colStatsMap = new HashMap<>(); colStatsMap.put("pt", ColStats.newColStats(0, 4L, 2, 2, 0L, 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 f5d4121672b0..ea47df2d9d72 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 @@ -52,7 +52,7 @@ public void testTableFilterValueStatistics() throws Exception { null, null, null, - false); + null); Assertions.assertThat(keyFilterSource.reportStatistics().getRowCount()).isEqualTo(9L); // TODO validate column statistics } diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java index c3521c6f1a37..9acea56ab393 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java @@ -74,9 +74,9 @@ public class OrcFileFormat extends FileFormat { public OrcFileFormat(FormatContext formatContext) { super(IDENTIFIER); this.orcProperties = getOrcProperties(formatContext.options(), formatContext); - this.readerConf = new org.apache.hadoop.conf.Configuration(); + this.readerConf = new org.apache.hadoop.conf.Configuration(false); this.orcProperties.forEach((k, v) -> readerConf.set(k.toString(), v.toString())); - this.writerConf = new org.apache.hadoop.conf.Configuration(); + this.writerConf = new org.apache.hadoop.conf.Configuration(false); this.orcProperties.forEach((k, v) -> writerConf.set(k.toString(), v.toString())); this.readBatchSize = formatContext.readBatchSize(); this.writeBatchSize = formatContext.writeBatchSize(); diff --git a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcFileFormatTest.java b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcFileFormatTest.java index 46bf6afe6613..9e5769595c32 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcFileFormatTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcFileFormatTest.java @@ -83,4 +83,68 @@ public void testSupportedDataTypes() { dataFields.add(new DataField(index++, "decimal_type", DataTypes.DECIMAL(10, 3))); orc.validateDataFields(new RowType(dataFields)); } + + @Test + public void testCreateCost() { + double createConfCost = createConfigCost(); + for (int i = 0; i < 1000; i++) { + create(); + } + int times = 10_000; + long start = System.nanoTime(); + for (int i = 0; i < times; i++) { + create(); + } + double cost = ((double) (System.nanoTime() - start)) / 1000_000 / times; + assertThat(cost * 500 < createConfCost).isTrue(); + } + + @Test + public void testCreateCostWithRandomConfig() { + double createConfCost = createConfigCost(); + for (int i = 0; i < 1000; i++) { + createRandomConfig(); + } + int times = 10_000; + long start = System.nanoTime(); + for (int i = 0; i < times; i++) { + createRandomConfig(); + } + double cost = ((double) (System.nanoTime() - start)) / 1000_000 / times; + assertThat(cost * 10 < createConfCost).isTrue(); + } + + private double createConfigCost() { + for (int i = 0; i < 1000; i++) { + createConfig(); + } + int times = 10_000; + long start = System.nanoTime(); + for (int i = 0; i < times; i++) { + createConfig(); + } + return ((double) (System.nanoTime() - start)) / 1000_000 / times; + } + + private void createConfig() { + org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration(); + conf.set("a", "a"); + } + + private void create() { + Options options = new Options(); + options.setString("haha", "1"); + options.setString("compress", "zlib"); + OrcFileFormat orcFileFormat = + new OrcFileFormatFactory().create(new FormatContext(options, 1024, 1024)); + } + + private void createRandomConfig() { + Options options = new Options(); + options.setString("haha", "1"); + options.setString("compress", "zlib"); + options.setString("a", Math.random() + ""); + OrcFileFormat orcFileFormat = + new OrcFileFormatFactory().create(new FormatContext(options, 1024, 1024)); + } } 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 f1e01209dfa4..237b59e43c4d 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 @@ -106,7 +106,7 @@ import static org.apache.paimon.hive.HiveCatalogOptions.IDENTIFIER; import static org.apache.paimon.hive.HiveCatalogOptions.LOCATION_IN_PROPERTIES; import static org.apache.paimon.hive.HiveTableUtils.convertToFormatTable; -import static org.apache.paimon.options.CatalogOptions.ALLOW_UPPER_CASE; +import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; import static org.apache.paimon.options.CatalogOptions.FORMAT_TABLE_ENABLED; import static org.apache.paimon.options.CatalogOptions.SYNC_ALL_PROPERTIES; import static org.apache.paimon.options.CatalogOptions.TABLE_TYPE; @@ -902,8 +902,8 @@ private void alterTableToHms(Table table, Identifier identifier, TableSchema new } @Override - public boolean allowUpperCase() { - return catalogOptions.getOptional(ALLOW_UPPER_CASE).orElse(false); + public boolean caseSensitive() { + return catalogOptions.getOptional(CASE_SENSITIVE).orElse(false); } @Override @@ -961,7 +961,6 @@ public void repairDatabase(String databaseName) { public void repairTable(Identifier identifier) throws TableNotExistException { checkNotBranch(identifier, "repairTable"); checkNotSystemTable(identifier, "repairTable"); - validateIdentifierNameCaseInsensitive(identifier); Path location = getTableLocation(identifier); TableSchema tableSchema = diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java index 3793c86f8269..885fa463e5a7 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java @@ -119,17 +119,28 @@ public void addPartitionsSpec(List> partitionSpecs public void alterPartition( LinkedHashMap partitionSpec, Map parameters, - long modifyTime) + long modifyTime, + boolean ignoreIfNotExist) throws Exception { List partitionValues = new ArrayList<>(partitionSpec.values()); int currentTime = (int) (modifyTime / 1000); - Partition hivePartition = - clients.run( - client -> - client.getPartition( - identifier.getDatabaseName(), - identifier.getObjectName(), - partitionValues)); + Partition hivePartition; + try { + hivePartition = + clients.run( + client -> + client.getPartition( + identifier.getDatabaseName(), + identifier.getObjectName(), + partitionValues)); + } catch (NoSuchObjectException e) { + if (ignoreIfNotExist) { + return; + } else { + throw e; + } + } + hivePartition.setValues(partitionValues); hivePartition.setLastAccessTime(currentTime); hivePartition.getParameters().putAll(parameters); diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java index fad3a383b1c8..12023cb84779 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java @@ -70,7 +70,7 @@ import static org.apache.paimon.CoreOptions.FILE_FORMAT; import static org.apache.paimon.CoreOptions.TYPE; import static org.apache.paimon.TableType.FORMAT_TABLE; -import static org.apache.paimon.options.CatalogOptions.ALLOW_UPPER_CASE; +import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; import static org.apache.paimon.spark.SparkCatalogOptions.DEFAULT_DATABASE; import static org.apache.paimon.spark.SparkTypeUtils.toPaimonType; import static org.apache.paimon.spark.util.OptionUtils.copyWithSQLConf; @@ -97,9 +97,9 @@ public void initialize(String name, CaseInsensitiveStringMap options) { CatalogContext catalogContext = CatalogContext.create(Options.fromMap(options), sessionState.newHadoopConf()); - // if spark is case-insensitive, set allow upper case to catalog + // if spark is case-insensitive, set case-sensitive to catalog if (!sessionState.conf().caseSensitiveAnalysis()) { - newOptions.put(ALLOW_UPPER_CASE.key(), "true"); + newOptions.put(CASE_SENSITIVE.key(), "true"); } options = new CaseInsensitiveStringMap(newOptions); diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java index 6b7b17b1b1a5..b57228fa44f0 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java @@ -62,7 +62,7 @@ import java.util.Map; import java.util.concurrent.Callable; -import static org.apache.paimon.options.CatalogOptions.ALLOW_UPPER_CASE; +import static org.apache.paimon.options.CatalogOptions.CASE_SENSITIVE; import static org.apache.paimon.options.CatalogOptions.METASTORE; import static org.apache.paimon.options.CatalogOptions.WAREHOUSE; import static org.apache.paimon.spark.SparkCatalogOptions.CREATE_UNDERLYING_SESSION_CATALOG; @@ -331,9 +331,9 @@ private void fillCommonConfigurations(Map options, SQLConf sqlCo options.put(DEFAULT_DATABASE.key(), sessionCatalogDefaultDatabase); } - // if spark is case-insensitive, set allow upper case to catalog + // if spark is case-insensitive, set case-sensitive to catalog if (!sqlConf.caseSensitiveAnalysis()) { - options.put(ALLOW_UPPER_CASE.key(), "true"); + options.put(CASE_SENSITIVE.key(), "true"); } } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkProcedures.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkProcedures.java index b2fa66a15090..f5052ea25f95 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkProcedures.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkProcedures.java @@ -35,6 +35,7 @@ import org.apache.paimon.spark.procedure.MigrateTableProcedure; import org.apache.paimon.spark.procedure.Procedure; import org.apache.paimon.spark.procedure.ProcedureBuilder; +import org.apache.paimon.spark.procedure.PurgeFilesProcedure; import org.apache.paimon.spark.procedure.RefreshObjectTableProcedure; import org.apache.paimon.spark.procedure.RemoveOrphanFilesProcedure; import org.apache.paimon.spark.procedure.RenameTagProcedure; @@ -74,6 +75,7 @@ private static Map> initProcedureBuilders() { procedureBuilders.put("rollback", RollbackProcedure::builder); procedureBuilders.put("rollback_to_timestamp", RollbackToTimestampProcedure::builder); procedureBuilders.put("rollback_to_watermark", RollbackToWatermarkProcedure::builder); + procedureBuilders.put("purge_files", PurgeFilesProcedure::builder); procedureBuilders.put("create_tag", CreateTagProcedure::builder); procedureBuilders.put("replace_tag", ReplaceTagProcedure::builder); procedureBuilders.put("rename_tag", RenameTagProcedure::builder); diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/PurgeFilesProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/PurgeFilesProcedure.java new file mode 100644 index 000000000000..8a7aec6e1410 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/PurgeFilesProcedure.java @@ -0,0 +1,112 @@ +/* + * 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.spark.procedure; + +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.table.FileStoreTable; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +import java.io.IOException; +import java.util.Arrays; + +import static org.apache.spark.sql.types.DataTypes.StringType; + +/** A procedure to purge files for a table. */ +public class PurgeFilesProcedure extends BaseProcedure { + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {ProcedureParameter.required("table", StringType)}; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("result", StringType, true, Metadata.empty()) + }); + + private PurgeFilesProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public StructType outputType() { + return OUTPUT_TYPE; + } + + @Override + public InternalRow[] call(InternalRow args) { + Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); + + return modifyPaimonTable( + tableIdent, + table -> { + FileStoreTable fileStoreTable = (FileStoreTable) table; + FileIO fileIO = fileStoreTable.fileIO(); + Path tablePath = fileStoreTable.snapshotManager().tablePath(); + try { + Arrays.stream(fileIO.listStatus(tablePath)) + .filter(f -> !f.getPath().getName().contains("schema")) + .forEach( + fileStatus -> { + try { + fileIO.delete(fileStatus.getPath(), true); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } catch (IOException e) { + throw new RuntimeException(e); + } + + InternalRow outputRow = + newInternalRow( + UTF8String.fromString( + String.format( + "Success purge files with table: %s.", + fileStoreTable.name()))); + return new InternalRow[] {outputRow}; + }); + } + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + public PurgeFilesProcedure doBuild() { + return new PurgeFilesProcedure(tableCatalog()); + } + }; + } + + @Override + public String description() { + return "PurgeFilesProcedure"; + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScanBuilder.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScanBuilder.scala index d8b66e1cd1e0..0393a1cd1578 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScanBuilder.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScanBuilder.scala @@ -21,6 +21,7 @@ package org.apache.paimon.spark import org.apache.paimon.predicate.PredicateBuilder import org.apache.paimon.spark.aggregate.LocalAggregator import org.apache.paimon.table.Table +import org.apache.paimon.table.source.DataSplit import org.apache.spark.sql.connector.expressions.aggregate.Aggregation import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownAggregates, SupportsPushDownLimit} @@ -36,12 +37,12 @@ class PaimonScanBuilder(table: Table) override def pushLimit(limit: Int): Boolean = { // It is safe, since we will do nothing if it is the primary table and the split is not `rawConvertible` pushDownLimit = Some(limit) - // just make a best effort to push down limit + // just make the best effort to push down limit false } override def supportCompletePushDown(aggregation: Aggregation): Boolean = { - // for now we only support complete push down, so there is no difference with `pushAggregation` + // for now, we only support complete push down, so there is no difference with `pushAggregation` pushAggregation(aggregation) } @@ -66,8 +67,11 @@ class PaimonScanBuilder(table: Table) val pushedPartitionPredicate = PredicateBuilder.and(pushedPredicates.map(_._2): _*) readBuilder.withFilter(pushedPartitionPredicate) } - val scan = readBuilder.newScan() - scan.listPartitionEntries.asScala.foreach(aggregator.update) + val dataSplits = readBuilder.newScan().plan().splits().asScala.map(_.asInstanceOf[DataSplit]) + if (!dataSplits.forall(_.mergedRowCountAvailable())) { + return false + } + dataSplits.foreach(aggregator.update) localScan = Some( PaimonLocalScan( aggregator.result(), diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala index 41e7fd3c3ce9..8988e7218d1f 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala @@ -19,10 +19,10 @@ package org.apache.paimon.spark.aggregate import org.apache.paimon.data.BinaryRow -import org.apache.paimon.manifest.PartitionEntry import org.apache.paimon.spark.SparkTypeUtils import org.apache.paimon.spark.data.SparkInternalRow import org.apache.paimon.table.{DataTable, Table} +import org.apache.paimon.table.source.DataSplit import org.apache.paimon.utils.{InternalRowUtils, ProjectedRow} import org.apache.spark.sql.catalyst.InternalRow @@ -78,13 +78,7 @@ class LocalAggregator(table: Table) { } def pushAggregation(aggregation: Aggregation): Boolean = { - if ( - !table.isInstanceOf[DataTable] || - !table.primaryKeys.isEmpty - ) { - return false - } - if (table.asInstanceOf[DataTable].coreOptions.deletionVectorsEnabled) { + if (!table.isInstanceOf[DataTable]) { return false } @@ -108,12 +102,12 @@ class LocalAggregator(table: Table) { SparkInternalRow.create(partitionType).replace(genericRow) } - def update(partitionEntry: PartitionEntry): Unit = { + def update(dataSplit: DataSplit): Unit = { assert(isInitialized) - val groupByRow = requiredGroupByRow(partitionEntry.partition()) + val groupByRow = requiredGroupByRow(dataSplit.partition()) val aggFuncEvaluator = groupByEvaluatorMap.getOrElseUpdate(groupByRow, aggFuncEvaluatorGetter()) - aggFuncEvaluator.foreach(_.update(partitionEntry)) + aggFuncEvaluator.foreach(_.update(dataSplit)) } def result(): Array[InternalRow] = { @@ -147,7 +141,7 @@ class LocalAggregator(table: Table) { } trait AggFuncEvaluator[T] { - def update(partitionEntry: PartitionEntry): Unit + def update(dataSplit: DataSplit): Unit def result(): T def resultType: DataType def prettyName: String @@ -156,8 +150,8 @@ trait AggFuncEvaluator[T] { class CountStarEvaluator extends AggFuncEvaluator[Long] { private var _result: Long = 0L - override def update(partitionEntry: PartitionEntry): Unit = { - _result += partitionEntry.recordCount() + override def update(dataSplit: DataSplit): Unit = { + _result += dataSplit.mergedRowCount() } override def result(): Long = _result diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonCommand.scala index 87583593e3fe..04118a438307 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonCommand.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonCommand.scala @@ -245,12 +245,12 @@ trait PaimonCommand extends WithFileStoreTable with ExpressionHelper with SQLCon val relativeFilePath = location.toUri.relativize(new URI(filePath)).toString val (partition, bucket) = dataFileToPartitionAndBucket.toMap.apply(relativeFilePath) val pathFactory = my_table.store().pathFactory() - val partitionAndBucket = pathFactory - .relativePartitionAndBucketPath(partition, bucket) + val relativeBucketPath = pathFactory + .relativeBucketPath(partition, bucket) .toString SparkDeletionVectors( - partitionAndBucket, + relativeBucketPath, SerializationUtils.serializeBinaryRow(partition), bucket, Seq((new Path(filePath).getName, dv.serializeToBytes())) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDataFileMeta.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDataFileMeta.scala index 9c377b47c4fd..569a84a74cf5 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDataFileMeta.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDataFileMeta.scala @@ -35,7 +35,7 @@ case class SparkDataFileMeta( def relativePath(fileStorePathFactory: FileStorePathFactory): String = { fileStorePathFactory - .relativePartitionAndBucketPath(partition, bucket) + .relativeBucketPath(partition, bucket) .toUri .toString + "/" + dataFileMeta.fileName() } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDeletionVectors.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDeletionVectors.scala index 9f687e6e3c92..1f908aeb908b 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDeletionVectors.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDeletionVectors.scala @@ -36,7 +36,7 @@ case class SparkDeletionVectors( ) { def relativePaths(fileStorePathFactory: FileStorePathFactory): Seq[String] = { val prefix = fileStorePathFactory - .relativePartitionAndBucketPath(SerializationUtils.deserializeBinaryRow(partition), bucket) + .relativeBucketPath(SerializationUtils.deserializeBinaryRow(partition), bucket) .toUri .toString + "/" dataFileAndDeletionVector.map(prefix + _._1) diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/PurgeFilesProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/PurgeFilesProcedureTest.scala new file mode 100644 index 000000000000..27eafe1c3d03 --- /dev/null +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/PurgeFilesProcedureTest.scala @@ -0,0 +1,44 @@ +/* + * 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.spark.procedure + +import org.apache.paimon.spark.PaimonSparkTestBase + +import org.apache.spark.sql.Row + +class PurgeFilesProcedureTest extends PaimonSparkTestBase { + + test("Paimon procedure: purge files test") { + spark.sql(s""" + |CREATE TABLE T (id STRING, name STRING) + |USING PAIMON + |""".stripMargin) + + spark.sql("insert into T select '1', 'aa'"); + checkAnswer(spark.sql("select * from test.T"), Row("1", "aa") :: Nil) + + spark.sql("CALL paimon.sys.purge_files(table => 'test.T')") + checkAnswer(spark.sql("select * from test.T"), Nil) + + spark.sql("refresh table test.T"); + spark.sql("insert into T select '2', 'aa'"); + checkAnswer(spark.sql("select * from test.T"), Row("2", "aa") :: Nil) + } + +} diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala index 1189f1f2906b..4ba079ea0bb2 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala @@ -434,6 +434,63 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { } } + test("Paimon DDL with hive catalog: create external table with schema evolution") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + spark.sql(s"USE $catalogName") + withTempDir { + tbLocation => + withDatabase("paimon_db") { + spark.sql(s"CREATE DATABASE IF NOT EXISTS paimon_db") + spark.sql(s"USE paimon_db") + withTable("t1", "t2") { + val expertTbLocation = tbLocation.getCanonicalPath + spark.sql( + s""" + |CREATE TABLE t1 (a INT, b INT, c STRUCT) USING paimon + |LOCATION '$expertTbLocation' + |""".stripMargin) + spark.sql("INSERT INTO t1 VALUES (1, 1, STRUCT(1, 1, 1))") + spark.sql("ALTER TABLE t1 DROP COLUMN b") + spark.sql("ALTER TABLE t1 ADD COLUMN b INT") + spark.sql("ALTER TABLE t1 DROP COLUMN c.f2") + spark.sql("ALTER TABLE t1 ADD COLUMN c.f2 INT") + spark.sql("INSERT INTO t1 VALUES (2, STRUCT(1, 1, 1), 1)") + checkAnswer( + spark.sql("SELECT * FROM t1 ORDER by a"), + Seq(Row(1, Row(1, 1, null), null), Row(2, Row(1, 1, 1), 1))) + + spark.sql( + s""" + |CREATE TABLE t2 (a INT, c STRUCT, b INT) USING paimon + |LOCATION '$expertTbLocation' + |""".stripMargin) + checkAnswer( + spark.sql("SELECT * FROM t2 ORDER by a"), + Seq(Row(1, Row(1, 1, null), null), Row(2, Row(1, 1, 1), 1))) + + // create table with wrong schema + intercept[Exception] { + spark.sql( + s""" + |CREATE TABLE t3 (a INT, b INT, c STRUCT) USING paimon + |LOCATION '$expertTbLocation' + |""".stripMargin) + } + + intercept[Exception] { + spark.sql( + s""" + |CREATE TABLE t4 (a INT, c STRUCT, b INT) USING paimon + |LOCATION '$expertTbLocation' + |""".stripMargin) + } + } + } + } + } + } + def getDatabaseProp(dbName: String, propertyName: String): String = { spark .sql(s"DESC DATABASE EXTENDED $dbName") diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala index ea8309e14ffe..ec5526f20e1d 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala @@ -631,6 +631,43 @@ class DeletionVectorTest extends PaimonSparkTestBase with AdaptiveSparkPlanHelpe ) } + test("Paimon deletionVector: get cardinality") { + sql(s""" + |CREATE TABLE T (id INT) + |TBLPROPERTIES ( + | 'deletion-vectors.enabled' = 'true', + | 'bucket-key' = 'id', + | 'bucket' = '1' + |) + |""".stripMargin) + + sql("INSERT INTO T SELECT /*+ REPARTITION(1) */ id FROM range (1, 50000)") + sql("DELETE FROM T WHERE id >= 111 and id <= 444") + + val fileStore = loadTable("T").store() + val indexManifest = fileStore.snapshotManager().latestSnapshot().indexManifest() + val entry = fileStore.newIndexFileHandler().readManifest(indexManifest).get(0) + val dvMeta = entry.indexFile().deletionVectorMetas().values().iterator().next() + + assert(dvMeta.cardinality() == 334) + } + + test("Paimon deletionVector: delete from non-pk table with data file path") { + sql(s""" + |CREATE TABLE T (id INT) + |TBLPROPERTIES ( + | 'deletion-vectors.enabled' = 'true', + | 'bucket-key' = 'id', + | 'bucket' = '1', + | 'data-file.path-directory' = 'data' + |) + |""".stripMargin) + + sql("INSERT INTO T SELECT /*+ REPARTITION(1) */ id FROM range (1, 50000)") + sql("DELETE FROM T WHERE id >= 111 and id <= 444") + checkAnswer(sql("SELECT count(*) FROM T"), Row(49665)) + } + private def getPathName(path: String): String = { new Path(path).getName } diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala index 8973ea93d8a0..bcd84fdc11da 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala @@ -113,6 +113,35 @@ abstract class MergeIntoTableTestBase extends PaimonSparkTestBase with PaimonTab } } + test(s"Paimon MergeInto: update + insert with data file path") { + withTable("source", "target") { + + Seq((1, 100, "c11"), (3, 300, "c33")).toDF("a", "b", "c").createOrReplaceTempView("source") + + createTable( + "target", + "a INT, b INT, c STRING", + Seq("a"), + Seq(), + Map("data-file.path-directory" -> "data")) + spark.sql("INSERT INTO target values (1, 10, 'c1'), (2, 20, 'c2')") + + spark.sql(s""" + |MERGE INTO target + |USING source + |ON target.a = source.a + |WHEN MATCHED THEN + |UPDATE SET a = source.a, b = source.b, c = source.c + |WHEN NOT MATCHED + |THEN INSERT (a, b, c) values (a, b, c) + |""".stripMargin) + + checkAnswer( + spark.sql("SELECT * FROM target ORDER BY a, b"), + Seq(Row(1, 100, "c11"), Row(2, 20, "c2"), Row(3, 300, "c33"))) + } + } + test(s"Paimon MergeInto: delete + insert") { withTable("source", "target") { diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala index ba314e3afa81..503f1c8e3e9d 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala @@ -18,7 +18,7 @@ package org.apache.paimon.spark.sql -import org.apache.paimon.spark.{PaimonBatch, PaimonInputPartition, PaimonScan, PaimonSparkTestBase, SparkTable} +import org.apache.paimon.spark.{PaimonScan, PaimonSparkTestBase, SparkTable} import org.apache.paimon.table.source.DataSplit import org.apache.spark.sql.Row @@ -29,8 +29,6 @@ import org.apache.spark.sql.connector.read.{ScanBuilder, SupportsPushDownLimit} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.junit.jupiter.api.Assertions -import scala.collection.JavaConverters._ - class PaimonPushDownTest extends PaimonSparkTestBase { import testImplicits._ @@ -64,7 +62,7 @@ class PaimonPushDownTest extends PaimonSparkTestBase { checkAnswer(spark.sql(q), Row(1, "a", "p1") :: Nil) // case 2 - // filter "id = '1' or pt = 'p1'" can't push down completely, it still need to be evaluated after scanning + // filter "id = '1' or pt = 'p1'" can't push down completely, it still needs to be evaluated after scanning q = "SELECT * FROM T WHERE id = '1' or pt = 'p1'" Assertions.assertTrue(checkEqualToFilterExists(q, "pt", Literal("p1"))) checkAnswer(spark.sql(q), Row(1, "a", "p1") :: Row(2, "b", "p1") :: Nil) @@ -121,7 +119,7 @@ class PaimonPushDownTest extends PaimonSparkTestBase { val dataSplitsWithoutLimit = scanBuilder.build().asInstanceOf[PaimonScan].getOriginSplits Assertions.assertTrue(dataSplitsWithoutLimit.length >= 2) - // It still return false even it can push down limit. + // It still returns false even it can push down limit. Assertions.assertFalse(scanBuilder.asInstanceOf[SupportsPushDownLimit].pushLimit(1)) val dataSplitsWithLimit = scanBuilder.build().asInstanceOf[PaimonScan].getOriginSplits Assertions.assertEquals(1, dataSplitsWithLimit.length) @@ -169,12 +167,7 @@ class PaimonPushDownTest extends PaimonSparkTestBase { // Now, we have 4 dataSplits, and 2 dataSplit is nonRawConvertible, 2 dataSplit is rawConvertible. Assertions.assertEquals( 2, - dataSplitsWithoutLimit2 - .filter( - split => { - split.asInstanceOf[DataSplit].rawConvertible() - }) - .length) + dataSplitsWithoutLimit2.count(split => { split.asInstanceOf[DataSplit].rawConvertible() })) // Return 2 dataSplits. Assertions.assertFalse(scanBuilder2.asInstanceOf[SupportsPushDownLimit].pushLimit(2)) @@ -206,7 +199,40 @@ class PaimonPushDownTest extends PaimonSparkTestBase { // Need to scan all dataSplits. Assertions.assertEquals(4, dataSplitsWithLimit3.length) Assertions.assertEquals(1, spark.sql("SELECT * FROM T LIMIT 1").count()) + } + test("Paimon pushDown: limit for table with deletion vector") { + Seq(true, false).foreach( + deletionVectorsEnabled => { + Seq(true, false).foreach( + primaryKeyTable => { + withTable("T") { + sql(s""" + |CREATE TABLE T (id INT) + |TBLPROPERTIES ( + | 'deletion-vectors.enabled' = $deletionVectorsEnabled, + | '${if (primaryKeyTable) "primary-key" else "bucket-key"}' = 'id', + | 'bucket' = '10' + |) + |""".stripMargin) + + sql("INSERT INTO T SELECT id FROM range (1, 50000)") + sql("DELETE FROM T WHERE id % 13 = 0") + + val withoutLimit = getScanBuilder().build().asInstanceOf[PaimonScan].getOriginSplits + assert(withoutLimit.length == 10) + + val scanBuilder = getScanBuilder().asInstanceOf[SupportsPushDownLimit] + scanBuilder.pushLimit(1) + val withLimit = scanBuilder.build().asInstanceOf[PaimonScan].getOriginSplits + if (deletionVectorsEnabled || !primaryKeyTable) { + assert(withLimit.length == 1) + } else { + assert(withLimit.length == 10) + } + } + }) + }) } test("Paimon pushDown: runtime filter") { @@ -250,8 +276,7 @@ class PaimonPushDownTest extends PaimonSparkTestBase { } private def getScanBuilder(tableName: String = "T"): ScanBuilder = { - new SparkTable(loadTable(tableName)) - .newScanBuilder(CaseInsensitiveStringMap.empty()) + SparkTable(loadTable(tableName)).newScanBuilder(CaseInsensitiveStringMap.empty()) } private def checkFilterExists(sql: String): Boolean = { @@ -272,5 +297,4 @@ class PaimonPushDownTest extends PaimonSparkTestBase { case _ => false } } - } diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala index 501e7bfb4a51..78c02644a7ce 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala @@ -117,22 +117,58 @@ class PushDownAggregatesTest extends PaimonSparkTestBase with AdaptiveSparkPlanH } } - test("Push down aggregate - primary table") { - withTable("T") { - spark.sql("CREATE TABLE T (c1 INT, c2 STRING) TBLPROPERTIES ('primary-key' = 'c1')") - runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(0) :: Nil, 2) - spark.sql("INSERT INTO T VALUES(1, 'x'), (2, 'x'), (3, 'x'), (3, 'x')") - runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(3) :: Nil, 2) - } + test("Push down aggregate - primary key table with deletion vector") { + Seq(true, false).foreach( + deletionVectorsEnabled => { + withTable("T") { + spark.sql(s""" + |CREATE TABLE T (c1 INT, c2 STRING) + |TBLPROPERTIES ( + |'primary-key' = 'c1', + |'deletion-vectors.enabled' = $deletionVectorsEnabled + |) + |""".stripMargin) + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(0) :: Nil, 0) + + spark.sql("INSERT INTO T VALUES(1, 'x'), (2, 'x'), (3, 'x'), (3, 'x')") + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(3) :: Nil, 0) + + spark.sql("INSERT INTO T VALUES(1, 'x_1')") + if (deletionVectorsEnabled) { + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(3) :: Nil, 0) + } else { + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(3) :: Nil, 2) + } + } + }) } - test("Push down aggregate - enable deletion vector") { - withTable("T") { - spark.sql( - "CREATE TABLE T (c1 INT, c2 STRING) TBLPROPERTIES('deletion-vectors.enabled' = 'true')") - runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(0) :: Nil, 2) - spark.sql("INSERT INTO T VALUES(1, 'x'), (2, 'x'), (3, 'x'), (3, 'x')") - runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(4) :: Nil, 2) - } + test("Push down aggregate - table with deletion vector") { + Seq(true, false).foreach( + deletionVectorsEnabled => { + Seq(true, false).foreach( + primaryKeyTable => { + withTable("T") { + sql(s""" + |CREATE TABLE T (id INT) + |TBLPROPERTIES ( + | 'deletion-vectors.enabled' = $deletionVectorsEnabled, + | '${if (primaryKeyTable) "primary-key" else "bucket-key"}' = 'id', + | 'bucket' = '1' + |) + |""".stripMargin) + + sql("INSERT INTO T SELECT id FROM range (0, 5000)") + runAndCheckAggregate("SELECT COUNT(*) FROM T", Seq(Row(5000)), 0) + + sql("DELETE FROM T WHERE id > 100 and id <= 400") + if (deletionVectorsEnabled || !primaryKeyTable) { + runAndCheckAggregate("SELECT COUNT(*) FROM T", Seq(Row(4700)), 0) + } else { + runAndCheckAggregate("SELECT COUNT(*) FROM T", Seq(Row(4700)), 2) + } + } + }) + }) } } diff --git a/pom.xml b/pom.xml index 904b1c73c741..dbef98af06b2 100644 --- a/pom.xml +++ b/pom.xml @@ -529,6 +529,7 @@ under the License. release/** paimon-common/src/main/antlr4/** + paimon-core/src/test/resources/compatibility/**