diff --git a/docs/content/concepts/specification.md b/docs/content/concepts/specification.md index aae8970b69ce..3116f2eed839 100644 --- a/docs/content/concepts/specification.md +++ b/docs/content/concepts/specification.md @@ -57,7 +57,7 @@ CREATE TABLE my_table ( f1 STRING ); -INSERT INTO t VALUES (1, 11, '111'); +INSERT INTO my_table VALUES (1, 11, '111'); ``` Take a look to the disk: diff --git a/docs/content/flink/procedures.md b/docs/content/flink/procedures.md index a1a8bfe37a42..cfa905628772 100644 --- a/docs/content/flink/procedures.md +++ b/docs/content/flink/procedures.md @@ -238,6 +238,24 @@ All available procedures are listed below. CALL sys.expire_snapshots(`table` => 'default.T', older_than => '2024-01-01 12:00:00', max_deletes => 10)

+ + expire_partitions + + CALL sys.expire_partitions(table, expiration_time, timestamp_formatter)

+ + + To expire partitions. Argument: +
  • table: the target table identifier. Cannot be empty.
  • +
  • expiration_time: the expiration interval of a partition. A partition will be expired if it‘s lifetime is over this value. Partition time is extracted from the partition value.
  • +
  • timestamp_formatter: the formatter to format timestamp from string.
  • + + + -- for Flink 1.18

    + CALL sys.expire_partitions('default.T', '1 d', 'yyyy-MM-dd')

    + -- for Flink 1.19 and later

    + CALL sys.expire_partitions(`table` => 'default.T', expiration_time => '1 d', timestamp_formatter => 'yyyy-MM-dd')

    + + repair diff --git a/docs/content/flink/sql-query.md b/docs/content/flink/sql-query.md index 565810944f2e..1a92915b46ca 100644 --- a/docs/content/flink/sql-query.md +++ b/docs/content/flink/sql-query.md @@ -51,6 +51,10 @@ SELECT * FROM t /*+ OPTIONS('scan.snapshot-id' = '1') */; -- read the snapshot from specified timestamp in unix milliseconds SELECT * FROM t /*+ OPTIONS('scan.timestamp-millis' = '1678883047356') */; +-- read the snapshot from specified timestamp string ,it will be automatically converted to timestamp in unix milliseconds +-- Supported formats include:yyyy-MM-dd, yyyy-MM-dd HH:mm:ss, yyyy-MM-dd HH:mm:ss.SSS, use default local time zone +SELECT * FROM t /*+ OPTIONS('scan.timestamp' = '2023-12-09 23:09:12') */; + -- read tag 'my-tag' SELECT * FROM t /*+ OPTIONS('scan.tag-name' = 'my-tag') */; diff --git a/docs/content/primary-key-table/deletion-vectors.md b/docs/content/primary-key-table/deletion-vectors.md index 23262bf1bdb2..3eb6f293f5a9 100644 --- a/docs/content/primary-key-table/deletion-vectors.md +++ b/docs/content/primary-key-table/deletion-vectors.md @@ -45,6 +45,5 @@ By specifying `'deletion-vectors.enabled' = 'true'`, the Deletion Vectors mode c ## Limitation - `changelog-producer` needs to be `none` or `lookup`. -- `changelog-producer.lookup-wait` can't be `false`. - `merge-engine` can't be `first-row`, because the read of first-row is already no merging, deletion vectors are not needed. - This mode will filter the data in level-0, so when using time travel to read `APPEND` snapshot, there will be data delay. diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index beb8a7dc9f10..862f614eedae 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -62,6 +62,12 @@

    Enum

    Whether to double write to a changelog file. This changelog file keeps the details of data changes, it can be read directly during stream reads. This can be applied to tables with primary keys.

    Possible values: + +
    changelog-producer.lookup-wait
    + true + Boolean + When changelog-producer is set to LOOKUP, commit will wait for changelog generation by lookup. +
    changelog-producer.row-deduplicate
    false @@ -581,6 +587,12 @@ String Optional tag name used in case of "from-snapshot" scan mode. + +
    scan.timestamp
    + (none) + String + Optional timestamp used in case of "from-timestamp" scan mode, it will be automatically converted to timestamp in unix milliseconds, use local time zone +
    scan.timestamp-millis
    (none) diff --git a/docs/layouts/shortcodes/generated/flink_connector_configuration.html b/docs/layouts/shortcodes/generated/flink_connector_configuration.html index 8a826acc1fff..3f8a9e523ddc 100644 --- a/docs/layouts/shortcodes/generated/flink_connector_configuration.html +++ b/docs/layouts/shortcodes/generated/flink_connector_configuration.html @@ -26,12 +26,6 @@ - -
    changelog-producer.lookup-wait
    - true - Boolean - When changelog-producer is set to LOOKUP, commit will wait for changelog generation by lookup. -
    end-input.watermark
    (none) 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 484a205172de..5e8ebb264cc1 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -33,6 +33,7 @@ import org.apache.paimon.options.description.DescribedEnum; import org.apache.paimon.options.description.Description; import org.apache.paimon.options.description.InlineElement; +import org.apache.paimon.utils.DateTimeUtils; import org.apache.paimon.utils.MathUtils; import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.StringUtils; @@ -51,6 +52,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.TimeZone; import java.util.UUID; import java.util.stream.Collectors; @@ -553,6 +555,13 @@ public class CoreOptions implements Serializable { .withDeprecatedKeys("log.scan") .withDescription("Specify the scanning behavior of the source."); + public static final ConfigOption SCAN_TIMESTAMP = + key("scan.timestamp") + .stringType() + .noDefaultValue() + .withDescription( + "Optional timestamp used in case of \"from-timestamp\" scan mode, it will be automatically converted to timestamp in unix milliseconds, use local time zone"); + public static final ConfigOption SCAN_TIMESTAMP_MILLIS = key("scan.timestamp-millis") .longType() @@ -1194,6 +1203,17 @@ public class CoreOptions implements Serializable { .noDefaultValue() .withDescription("Specifies the commit user prefix."); + public static final ConfigOption CHANGELOG_PRODUCER_LOOKUP_WAIT = + key("changelog-producer.lookup-wait") + .booleanType() + .defaultValue(true) + .withDescription( + "When " + + CoreOptions.CHANGELOG_PRODUCER.key() + + " is set to " + + ChangelogProducer.LOOKUP.name() + + ", commit will wait for changelog generation by lookup."); + private final Options options; public CoreOptions(Map options) { @@ -1609,7 +1629,8 @@ public StartupMode startupMode() { public static StartupMode startupMode(Options options) { StartupMode mode = options.get(SCAN_MODE); if (mode == StartupMode.DEFAULT) { - if (options.getOptional(SCAN_TIMESTAMP_MILLIS).isPresent()) { + if (options.getOptional(SCAN_TIMESTAMP_MILLIS).isPresent() + || options.getOptional(SCAN_TIMESTAMP).isPresent()) { return StartupMode.FROM_TIMESTAMP; } else if (options.getOptional(SCAN_SNAPSHOT_ID).isPresent() || options.getOptional(SCAN_TAG_NAME).isPresent() @@ -1632,7 +1653,17 @@ public static StartupMode startupMode(Options options) { } public Long scanTimestampMills() { - return options.get(SCAN_TIMESTAMP_MILLIS); + String timestampStr = scanTimestamp(); + Long timestampMillis = options.get(SCAN_TIMESTAMP_MILLIS); + if (timestampMillis == null && timestampStr != null) { + return DateTimeUtils.parseTimestampData(timestampStr, 3, TimeZone.getDefault()) + .getMillisecond(); + } + return timestampMillis; + } + + public String scanTimestamp() { + return options.get(SCAN_TIMESTAMP); } public Long scanWatermark() { @@ -1893,6 +1924,11 @@ public String recordLevelTimeField() { return options.get(RECORD_LEVEL_TIME_FIELD); } + public boolean prepareCommitWaitCompaction() { + return changelogProducer() == ChangelogProducer.LOOKUP + && options.get(CHANGELOG_PRODUCER_LOOKUP_WAIT); + } + /** Specifies the merge engine for table with primary key. */ public enum MergeEngine implements DescribedEnum { DEDUPLICATE("deduplicate", "De-duplicate and keep the last row."), @@ -2227,6 +2263,10 @@ public static void setDefaultValues(Options options) { options.set(SCAN_MODE, StartupMode.FROM_TIMESTAMP); } + if (options.contains(SCAN_TIMESTAMP) && !options.contains(SCAN_MODE)) { + options.set(SCAN_MODE, StartupMode.FROM_TIMESTAMP); + } + if (options.contains(SCAN_FILE_CREATION_TIME_MILLIS) && !options.contains(SCAN_MODE)) { options.set(SCAN_MODE, StartupMode.FROM_FILE_CREATION_TIME); } diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexFormat.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexFormat.java index 9d4a97b1a0cf..4541cc46d216 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexFormat.java +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexFormat.java @@ -309,7 +309,10 @@ private FileIndexReader getFileIndexReader( indexType, FileIndexCommon.getFieldType(fields, columnName), new Options()) - .createReader(getBytesWithStartAndLength(startAndLength)); + .createReader( + seekableInputStream, + startAndLength.getLeft(), + startAndLength.getRight()); } private byte[] getBytesWithStartAndLength(Pair startAndLength) { diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexPredicate.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexPredicate.java index 31672a8b74e9..4f243e690cd5 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexPredicate.java +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexPredicate.java @@ -30,6 +30,9 @@ import org.apache.paimon.predicate.PredicateVisitor; import org.apache.paimon.types.RowType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import javax.annotation.Nullable; import java.io.Closeable; @@ -37,19 +40,23 @@ import java.util.Collection; import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; + +import static org.apache.paimon.fileindex.FileIndexResult.REMAIN; /** Utils to check secondary index (e.g. bloom filter) predicate. */ public class FileIndexPredicate implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(FileIndexPredicate.class); + private final FileIndexFormat.Reader reader; - private final Map fieldPredicates = new HashMap<>(); + + @Nullable private Path path; public FileIndexPredicate(Path path, FileIO fileIO, RowType fileRowType) throws IOException { this(fileIO.newInputStream(path), fileRowType); + this.path = path; } public FileIndexPredicate(byte[] serializedBytes, RowType fileRowType) { @@ -67,22 +74,13 @@ public boolean testPredicate(@Nullable Predicate filePredicate) { Set requredFieldNames = getRequiredNames(filePredicate); - List testWorkers = - requredFieldNames.stream() - .map( - cname -> - fieldPredicates.computeIfAbsent( - cname, - k -> - new FileIndexFieldPredicate( - cname, - reader.readColumnIndex(cname)))) - .collect(Collectors.toList()); - - for (FileIndexFieldPredicate testWorker : testWorkers) { - if (!testWorker.test(filePredicate)) { - return false; - } + Map> indexReaders = new HashMap<>(); + requredFieldNames.forEach(name -> indexReaders.put(name, reader.readColumnIndex(name))); + if (!new FileIndexPredicateTest(indexReaders).test(filePredicate).remain()) { + LOG.debug( + "One file has been filtered: " + + (path == null ? "in scan stage" : path.toString())); + return false; } return true; } @@ -114,55 +112,62 @@ public void close() throws IOException { } /** Predicate test worker. */ - private static class FileIndexFieldPredicate implements PredicateVisitor { + private static class FileIndexPredicateTest implements PredicateVisitor { - private final String columnName; - private final Collection fileIndexReaders; + private final Map> columnIndexReaders; - public FileIndexFieldPredicate( - String columnName, Collection fileIndexReaders) { - this.columnName = columnName; - this.fileIndexReaders = fileIndexReaders; + public FileIndexPredicateTest(Map> fileIndexReaders) { + this.columnIndexReaders = fileIndexReaders; } - public Boolean test(Predicate predicate) { + public FileIndexResult test(Predicate predicate) { return predicate.visit(this); } @Override - public Boolean visit(LeafPredicate predicate) { - if (columnName.equals(predicate.fieldName())) { - FieldRef fieldRef = - new FieldRef(predicate.index(), predicate.fieldName(), predicate.type()); - for (FileIndexReader fileIndexReader : fileIndexReaders) { - if (!predicate - .function() - .visit(fileIndexReader, fieldRef, predicate.literals())) { - return false; - } + public FileIndexResult visit(LeafPredicate predicate) { + FileIndexResult compoundResult = REMAIN; + FieldRef fieldRef = + new FieldRef(predicate.index(), predicate.fieldName(), predicate.type()); + for (FileIndexReader fileIndexReader : columnIndexReaders.get(predicate.fieldName())) { + compoundResult = + compoundResult.and( + predicate + .function() + .visit(fileIndexReader, fieldRef, predicate.literals())); + + if (!compoundResult.remain()) { + return compoundResult; } } - return true; + return compoundResult; } @Override - public Boolean visit(CompoundPredicate predicate) { - + public FileIndexResult visit(CompoundPredicate predicate) { if (predicate.function() instanceof Or) { + FileIndexResult compoundResult = null; for (Predicate predicate1 : predicate.children()) { - if (predicate1.visit(this)) { - return true; - } + compoundResult = + compoundResult == null + ? predicate1.visit(this) + : compoundResult.or(predicate1.visit(this)); } - return false; + return compoundResult == null ? REMAIN : compoundResult; } else { + FileIndexResult compoundResult = null; for (Predicate predicate1 : predicate.children()) { - if (!predicate1.visit(this)) { - return false; + compoundResult = + compoundResult == null + ? predicate1.visit(this) + : compoundResult.and(predicate1.visit(this)); + // if not remain, no need to test anymore + if (!compoundResult.remain()) { + return compoundResult; } } - return true; + return compoundResult == null ? REMAIN : compoundResult; } } } diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexReader.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexReader.java index f4708c2eebd1..0726c061e05e 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexReader.java +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexReader.java @@ -23,84 +23,90 @@ import java.util.List; +import static org.apache.paimon.fileindex.FileIndexResult.REMAIN; + /** * Read file index from serialized bytes. Return true, means we need to search this file, else means * needn't. */ -public abstract class FileIndexReader implements FunctionVisitor { +public abstract class FileIndexReader implements FunctionVisitor { @Override - public Boolean visitIsNotNull(FieldRef fieldRef) { - return true; + public FileIndexResult visitIsNotNull(FieldRef fieldRef) { + return REMAIN; } @Override - public Boolean visitIsNull(FieldRef fieldRef) { - return true; + public FileIndexResult visitIsNull(FieldRef fieldRef) { + return REMAIN; } @Override - public Boolean visitStartsWith(FieldRef fieldRef, Object literal) { - return true; + public FileIndexResult visitStartsWith(FieldRef fieldRef, Object literal) { + return REMAIN; } @Override - public Boolean visitLessThan(FieldRef fieldRef, Object literal) { - return true; + public FileIndexResult visitLessThan(FieldRef fieldRef, Object literal) { + return REMAIN; } @Override - public Boolean visitGreaterOrEqual(FieldRef fieldRef, Object literal) { - return true; + public FileIndexResult visitGreaterOrEqual(FieldRef fieldRef, Object literal) { + return REMAIN; } @Override - public Boolean visitNotEqual(FieldRef fieldRef, Object literal) { - return true; + public FileIndexResult visitNotEqual(FieldRef fieldRef, Object literal) { + return REMAIN; } @Override - public Boolean visitLessOrEqual(FieldRef fieldRef, Object literal) { - return true; + public FileIndexResult visitLessOrEqual(FieldRef fieldRef, Object literal) { + return REMAIN; } @Override - public Boolean visitEqual(FieldRef fieldRef, Object literal) { - return true; + public FileIndexResult visitEqual(FieldRef fieldRef, Object literal) { + return REMAIN; } @Override - public Boolean visitGreaterThan(FieldRef fieldRef, Object literal) { - return true; + public FileIndexResult visitGreaterThan(FieldRef fieldRef, Object literal) { + return REMAIN; } @Override - public Boolean visitIn(FieldRef fieldRef, List literals) { + public FileIndexResult visitIn(FieldRef fieldRef, List literals) { + FileIndexResult fileIndexResult = null; for (Object key : literals) { - if (visitEqual(fieldRef, key)) { - return true; - } + fileIndexResult = + fileIndexResult == null + ? visitEqual(fieldRef, key) + : fileIndexResult.or(visitEqual(fieldRef, key)); } - return false; + return fileIndexResult; } @Override - public Boolean visitNotIn(FieldRef fieldRef, List literals) { + public FileIndexResult visitNotIn(FieldRef fieldRef, List literals) { + FileIndexResult fileIndexResult = null; for (Object key : literals) { - if (visitNotEqual(fieldRef, key)) { - return true; - } + fileIndexResult = + fileIndexResult == null + ? visitNotEqual(fieldRef, key) + : fileIndexResult.or(visitNotEqual(fieldRef, key)); } - return false; + return fileIndexResult; } @Override - public Boolean visitAnd(List children) { + public FileIndexResult visitAnd(List children) { throw new UnsupportedOperationException("Should not invoke this"); } @Override - public Boolean visitOr(List children) { + public FileIndexResult visitOr(List children) { throw new UnsupportedOperationException("Should not invoke this"); } } diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexResult.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexResult.java new file mode 100644 index 000000000000..0aaaec699190 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexResult.java @@ -0,0 +1,65 @@ +/* + * 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.fileindex; + +/** File index result to decide whether filter a file. */ +public interface FileIndexResult { + + FileIndexResult REMAIN = + new FileIndexResult() { + @Override + public boolean remain() { + return true; + } + + @Override + public FileIndexResult and(FileIndexResult fileIndexResult) { + return fileIndexResult; + } + + @Override + public FileIndexResult or(FileIndexResult fileIndexResult) { + return this; + } + }; + + FileIndexResult SKIP = + new FileIndexResult() { + @Override + public boolean remain() { + return false; + } + + @Override + public FileIndexResult and(FileIndexResult fileIndexResult) { + return this; + } + + @Override + public FileIndexResult or(FileIndexResult fileIndexResult) { + return fileIndexResult; + } + }; + + boolean remain(); + + FileIndexResult and(FileIndexResult fileIndexResult); + + FileIndexResult or(FileIndexResult fileIndexResult); +} diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexer.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexer.java index fa3085d79cb9..d6c51286a519 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexer.java +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexer.java @@ -18,25 +18,24 @@ package org.apache.paimon.fileindex; -import org.apache.paimon.fileindex.bloomfilter.BloomFilterFileIndex; +import org.apache.paimon.fs.SeekableInputStream; import org.apache.paimon.options.Options; import org.apache.paimon.types.DataType; -import static org.apache.paimon.fileindex.bloomfilter.BloomFilterFileIndex.BLOOM_FILTER; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** File index interface. To build a file index. */ public interface FileIndexer { + Logger LOG = LoggerFactory.getLogger(FileIndexer.class); + FileIndexWriter createWriter(); - FileIndexReader createReader(byte[] serializedBytes); + FileIndexReader createReader(SeekableInputStream inputStream, int start, int length); static FileIndexer create(String type, DataType dataType, Options options) { - switch (type) { - case BLOOM_FILTER: - return new BloomFilterFileIndex(dataType, options); - default: - throw new RuntimeException("Doesn't support filter type: " + type); - } + FileIndexerFactory fileIndexerFactory = FileIndexerFactoryUtils.load(type); + return fileIndexerFactory.create(dataType, options); } } diff --git a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexerFactory.java similarity index 73% rename from paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala rename to paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexerFactory.java index 5b1c65525404..85dcc97f4640 100644 --- a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexerFactory.java @@ -16,13 +16,15 @@ * limitations under the License. */ -package org.apache.paimon.spark.sql +package org.apache.paimon.fileindex; -trait WithTableOptions { +import org.apache.paimon.options.Options; +import org.apache.paimon.types.DataType; - // 3: fixed bucket, -1: dynamic bucket - protected val bucketModes: Seq[Int] = Seq(3, -1) +/** File index factory to construct {@link FileIndexer}. */ +public interface FileIndexerFactory { - protected val withPk: Seq[Boolean] = Seq(true, false) + String identifier(); + FileIndexer create(DataType type, Options options); } diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexerFactoryUtils.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexerFactoryUtils.java new file mode 100644 index 000000000000..269ba77266e3 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexerFactoryUtils.java @@ -0,0 +1,56 @@ +/* + * 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.fileindex; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; +import java.util.ServiceLoader; + +/** Load utils to load FileIndexerFactory. */ +public class FileIndexerFactoryUtils { + + private static final Logger LOG = LoggerFactory.getLogger(FileIndexerFactoryUtils.class); + + private static final Map factories = new HashMap<>(); + + static { + ServiceLoader serviceLoader = + ServiceLoader.load(FileIndexerFactory.class); + + for (FileIndexerFactory indexerFactory : serviceLoader) { + if (factories.put(indexerFactory.identifier(), indexerFactory) != null) { + LOG.warn( + "Found multiple FileIndexer for type: " + + indexerFactory.identifier() + + ", choose one of them"); + } + } + } + + static FileIndexerFactory load(String type) { + FileIndexerFactory fileIndexerFactory = factories.get(type); + if (fileIndexerFactory == null) { + throw new RuntimeException("Can't find file index for type: " + type); + } + return fileIndexerFactory; + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/bloomfilter/BloomFilterFileIndex.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/bloomfilter/BloomFilterFileIndex.java index 48f109a631ac..690ba7b6f8bf 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fileindex/bloomfilter/BloomFilterFileIndex.java +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/bloomfilter/BloomFilterFileIndex.java @@ -19,16 +19,24 @@ package org.apache.paimon.fileindex.bloomfilter; import org.apache.paimon.fileindex.FileIndexReader; +import org.apache.paimon.fileindex.FileIndexResult; import org.apache.paimon.fileindex.FileIndexWriter; import org.apache.paimon.fileindex.FileIndexer; +import org.apache.paimon.fs.SeekableInputStream; import org.apache.paimon.options.Options; import org.apache.paimon.predicate.FieldRef; import org.apache.paimon.types.DataType; import org.apache.paimon.utils.BloomFilter64; import org.apache.paimon.utils.BloomFilter64.BitSet; +import org.apache.paimon.utils.IOUtils; import org.apache.hadoop.util.bloom.HashFunction; +import java.io.IOException; + +import static org.apache.paimon.fileindex.FileIndexResult.REMAIN; +import static org.apache.paimon.fileindex.FileIndexResult.SKIP; + /** * Bloom filter for file index. * @@ -39,8 +47,6 @@ */ public class BloomFilterFileIndex implements FileIndexer { - public static final String BLOOM_FILTER = "bloom-filter"; - private static final int DEFAULT_ITEMS = 1_000_000; private static final double DEFAULT_FPP = 0.1; @@ -57,18 +63,21 @@ public BloomFilterFileIndex(DataType dataType, Options options) { this.fpp = options.getDouble(FPP, DEFAULT_FPP); } - public String name() { - return BLOOM_FILTER; - } - @Override public FileIndexWriter createWriter() { return new Writer(dataType, items, fpp); } @Override - public FileIndexReader createReader(byte[] serializedBytes) { - return new Reader(dataType, serializedBytes); + public FileIndexReader createReader(SeekableInputStream inputStream, int start, int length) { + try { + inputStream.seek(start); + byte[] serializedBytes = new byte[length]; + IOUtils.readFully(inputStream, serializedBytes); + return new Reader(dataType, serializedBytes); + } catch (IOException e) { + throw new RuntimeException(e); + } } private static class Writer extends FileIndexWriter { @@ -118,8 +127,8 @@ public Reader(DataType type, byte[] serializedBytes) { } @Override - public Boolean visitEqual(FieldRef fieldRef, Object key) { - return key == null || filter.testHash(hashFunction.hash(key)); + public FileIndexResult visitEqual(FieldRef fieldRef, Object key) { + return key == null || filter.testHash(hashFunction.hash(key)) ? REMAIN : SKIP; } } } diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/bloomfilter/BloomFilterFileIndexFactory.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/bloomfilter/BloomFilterFileIndexFactory.java new file mode 100644 index 000000000000..ec77a0a04075 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/bloomfilter/BloomFilterFileIndexFactory.java @@ -0,0 +1,40 @@ +/* + * 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.fileindex.bloomfilter; + +import org.apache.paimon.fileindex.FileIndexer; +import org.apache.paimon.fileindex.FileIndexerFactory; +import org.apache.paimon.options.Options; +import org.apache.paimon.types.DataType; + +/** Index factory to construct {@link BloomFilterFileIndex}. */ +public class BloomFilterFileIndexFactory implements FileIndexerFactory { + + public static final String BLOOM_FILTER = "bloom-filter"; + + @Override + public String identifier() { + return BLOOM_FILTER; + } + + @Override + public FileIndexer create(DataType type, Options options) { + return new BloomFilterFileIndex(type, options); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/empty/EmptyFileIndexReader.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/empty/EmptyFileIndexReader.java index cf2eb681253a..77e66f467598 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fileindex/empty/EmptyFileIndexReader.java +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/empty/EmptyFileIndexReader.java @@ -19,10 +19,13 @@ package org.apache.paimon.fileindex.empty; import org.apache.paimon.fileindex.FileIndexReader; +import org.apache.paimon.fileindex.FileIndexResult; import org.apache.paimon.predicate.FieldRef; import java.util.List; +import static org.apache.paimon.fileindex.FileIndexResult.SKIP; + /** Empty file index which has no writer and no serialized bytes. */ public class EmptyFileIndexReader extends FileIndexReader { @@ -30,42 +33,42 @@ public class EmptyFileIndexReader extends FileIndexReader { public static final EmptyFileIndexReader INSTANCE = new EmptyFileIndexReader(); @Override - public Boolean visitEqual(FieldRef fieldRef, Object literal) { - return false; + public FileIndexResult visitEqual(FieldRef fieldRef, Object literal) { + return SKIP; } @Override - public Boolean visitIsNotNull(FieldRef fieldRef) { - return false; + public FileIndexResult visitIsNotNull(FieldRef fieldRef) { + return SKIP; } @Override - public Boolean visitStartsWith(FieldRef fieldRef, Object literal) { - return false; + public FileIndexResult visitStartsWith(FieldRef fieldRef, Object literal) { + return SKIP; } @Override - public Boolean visitLessThan(FieldRef fieldRef, Object literal) { - return false; + public FileIndexResult visitLessThan(FieldRef fieldRef, Object literal) { + return SKIP; } @Override - public Boolean visitGreaterOrEqual(FieldRef fieldRef, Object literal) { - return false; + public FileIndexResult visitGreaterOrEqual(FieldRef fieldRef, Object literal) { + return SKIP; } @Override - public Boolean visitLessOrEqual(FieldRef fieldRef, Object literal) { - return false; + public FileIndexResult visitLessOrEqual(FieldRef fieldRef, Object literal) { + return SKIP; } @Override - public Boolean visitGreaterThan(FieldRef fieldRef, Object literal) { - return false; + public FileIndexResult visitGreaterThan(FieldRef fieldRef, Object literal) { + return SKIP; } @Override - public Boolean visitIn(FieldRef fieldRef, List literals) { - return false; + public FileIndexResult visitIn(FieldRef fieldRef, List literals) { + return SKIP; } } diff --git a/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderFactory.java b/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderFactory.java index ce92bb751252..d2fc91501636 100644 --- a/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderFactory.java +++ b/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderFactory.java @@ -24,10 +24,9 @@ import org.apache.paimon.reader.RecordReader; import java.io.IOException; -import java.io.Serializable; /** A factory to create {@link RecordReader} for file. */ -public interface FormatReaderFactory extends Serializable { +public interface FormatReaderFactory { RecordReader createReader(Context context) throws IOException; 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 4e51372c25f2..baffbcf3f997 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 @@ -97,6 +97,10 @@ public DataField copy() { return new DataField(id, name, type.copy(), description); } + public DataField copy(boolean isNullable) { + return new DataField(id, name, type.copy(isNullable), description); + } + public String asSQLString() { return formatString(type.asSQLString()); } 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 fe11976de698..a691dbc0b57d 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 @@ -99,6 +99,15 @@ public int getFieldIndex(String fieldName) { return -1; } + public int[] getFieldIndices(List projectFields) { + List fieldNames = getFieldNames(); + int[] projection = new int[projectFields.size()]; + for (int i = 0; i < projection.length; i++) { + projection[i] = fieldNames.indexOf(projectFields.get(i)); + } + return projection; + } + public boolean containsField(String fieldName) { for (DataField field : fields) { if (field.name().equals(fieldName)) { diff --git a/paimon-common/src/main/resources/META-INF/NOTICE b/paimon-common/src/main/resources/META-INF/NOTICE index c58d808b7bcb..4cc302361026 100644 --- a/paimon-common/src/main/resources/META-INF/NOTICE +++ b/paimon-common/src/main/resources/META-INF/NOTICE @@ -10,7 +10,7 @@ This project bundles the following dependencies under the Apache Software Licens This project bundles the following dependencies under the BSD 3-clause license. You find them under licenses/LICENSE.antlr-runtime and licenses/LICENSE.janino. -- org.antlr:antlr4-runtime:4.8 +- org.antlr:antlr4-runtime:4.9.3 - org.codehaus.janino:janino:3.0.11 - org.codehaus.janino:commons-compiler:3.0.11 - it.unimi.dsi:fastutil:8.5.12 diff --git a/paimon-common/src/main/resources/META-INF/services/org.apache.paimon.fileindex.FileIndexerFactory b/paimon-common/src/main/resources/META-INF/services/org.apache.paimon.fileindex.FileIndexerFactory new file mode 100644 index 000000000000..67abe62a60b7 --- /dev/null +++ b/paimon-common/src/main/resources/META-INF/services/org.apache.paimon.fileindex.FileIndexerFactory @@ -0,0 +1,16 @@ +# 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. + +org.apache.paimon.fileindex.bloomfilter.BloomFilterFileIndexFactory \ No newline at end of file diff --git a/paimon-common/src/test/java/org/apache/paimon/fileindex/bloomfilter/BloomFilterFileIndexTest.java b/paimon-common/src/test/java/org/apache/paimon/fileindex/bloomfilter/BloomFilterFileIndexTest.java index 9834598c8e08..88ffc0daedb2 100644 --- a/paimon-common/src/test/java/org/apache/paimon/fileindex/bloomfilter/BloomFilterFileIndexTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/fileindex/bloomfilter/BloomFilterFileIndexTest.java @@ -20,6 +20,7 @@ import org.apache.paimon.fileindex.FileIndexReader; import org.apache.paimon.fileindex.FileIndexWriter; +import org.apache.paimon.fs.ByteArraySeekableStream; import org.apache.paimon.options.Options; import org.apache.paimon.types.DataTypes; @@ -60,17 +61,20 @@ public void testAddFindByRandom() { testData.forEach(writer::write); - FileIndexReader reader = filter.createReader(writer.serializedBytes()); + byte[] serializedBytes = writer.serializedBytes(); + FileIndexReader reader = + filter.createReader( + new ByteArraySeekableStream(serializedBytes), 0, serializedBytes.length); for (byte[] bytes : testData) { - Assertions.assertThat(reader.visitEqual(null, bytes)).isTrue(); + Assertions.assertThat(reader.visitEqual(null, bytes).remain()).isTrue(); } int errorCount = 0; int num = 1000000; for (int i = 0; i < num; i++) { byte[] ra = random(); - if (reader.visitEqual(null, ra)) { + if (reader.visitEqual(null, ra).remain()) { errorCount++; } } @@ -100,17 +104,20 @@ public void testAddFindByRandomLong() { testData.forEach(writer::write); - FileIndexReader reader = filter.createReader(writer.serializedBytes()); + byte[] serializedBytes = writer.serializedBytes(); + FileIndexReader reader = + filter.createReader( + new ByteArraySeekableStream(serializedBytes), 0, serializedBytes.length); for (Long value : testData) { - Assertions.assertThat(reader.visitEqual(null, value)).isTrue(); + Assertions.assertThat(reader.visitEqual(null, value).remain()).isTrue(); } int errorCount = 0; int num = 1000000; for (int i = 0; i < num; i++) { Long ra = RANDOM.nextLong(); - if (reader.visitEqual(null, ra)) { + if (reader.visitEqual(null, ra).remain()) { errorCount++; } } diff --git a/paimon-common/src/test/java/org/apache/paimon/utils/DateTimeUtilsTest.java b/paimon-common/src/test/java/org/apache/paimon/utils/DateTimeUtilsTest.java index 6c1376d299f5..dc94170e8967 100644 --- a/paimon-common/src/test/java/org/apache/paimon/utils/DateTimeUtilsTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/utils/DateTimeUtilsTest.java @@ -23,6 +23,7 @@ import org.junit.jupiter.api.Test; import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; import static org.assertj.core.api.Assertions.assertThat; @@ -45,6 +46,28 @@ public void testFormatLocalDateTime() { } } + @Test + public void testParseTimestampData() { + String dt = "2024-01-14 19:35:00.012"; + Timestamp ts = DateTimeUtils.parseTimestampData(dt, 3); + assertThat(dt) + .isEqualTo( + ts.toLocalDateTime() + .format(DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS"))); + + dt = "2024-01-14 19:35:20"; + ts = DateTimeUtils.parseTimestampData(dt, 3); + assertThat(dt) + .isEqualTo( + ts.toLocalDateTime() + .format(DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"))); + + dt = "2024-01-14"; + ts = DateTimeUtils.parseTimestampData(dt, 3); + assertThat(dt) + .isEqualTo(ts.toLocalDateTime().format(DateTimeFormatter.ofPattern("yyyy-MM-dd"))); + } + @Test public void testTimestamp() { int nanos = 100; diff --git a/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java b/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java index c5a47ff54190..2878a6061fc4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java @@ -289,7 +289,7 @@ private CommitIncrement drainIncrement() { compactBefore.clear(); compactAfter.clear(); - return new CommitIncrement(dataIncrement, compactIncrement); + return new CommitIncrement(dataIncrement, compactIncrement, null); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/compact/CompactDeletionFile.java b/paimon-core/src/main/java/org/apache/paimon/compact/CompactDeletionFile.java new file mode 100644 index 000000000000..390ab7af90ff --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/compact/CompactDeletionFile.java @@ -0,0 +1,141 @@ +/* + * 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.compact; + +import org.apache.paimon.deletionvectors.DeletionVectorsMaintainer; +import org.apache.paimon.index.IndexFileHandler; +import org.apache.paimon.index.IndexFileMeta; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Optional; + +/** Deletion File from compaction. */ +public interface CompactDeletionFile { + + Optional getOrCompute(); + + CompactDeletionFile mergeOldFile(CompactDeletionFile old); + + void clean(); + + /** + * Used by async compaction, when compaction task is completed, deletions file will be generated + * immediately, so when updateCompactResult, we need to merge old deletion files (just delete + * them). + */ + static CompactDeletionFile generateFiles(DeletionVectorsMaintainer maintainer) { + List files = maintainer.writeDeletionVectorsIndex(); + if (files.size() > 1) { + throw new IllegalStateException( + "Should only generate one compact deletion file, this is a bug."); + } + + return new GeneratedDeletionFile( + files.isEmpty() ? null : files.get(0), maintainer.indexFileHandler()); + } + + /** For sync compaction, only create deletion files when prepareCommit. */ + static CompactDeletionFile lazyGeneration(DeletionVectorsMaintainer maintainer) { + return new LazyCompactDeletionFile(maintainer); + } + + /** A generated files implementation of {@link CompactDeletionFile}. */ + class GeneratedDeletionFile implements CompactDeletionFile { + + @Nullable private final IndexFileMeta deletionFile; + private final IndexFileHandler fileHandler; + + private boolean getInvoked = false; + + public GeneratedDeletionFile( + @Nullable IndexFileMeta deletionFile, IndexFileHandler fileHandler) { + this.deletionFile = deletionFile; + this.fileHandler = fileHandler; + } + + @Override + public Optional getOrCompute() { + this.getInvoked = true; + return Optional.ofNullable(deletionFile); + } + + @Override + public CompactDeletionFile mergeOldFile(CompactDeletionFile old) { + if (!(old instanceof GeneratedDeletionFile)) { + throw new IllegalStateException( + "old should be a GeneratedDeletionFile, but it is: " + old.getClass()); + } + + if (((GeneratedDeletionFile) old).getInvoked) { + throw new IllegalStateException("old should not be get, this is a bug."); + } + + if (deletionFile == null) { + return old; + } + + old.clean(); + return this; + } + + @Override + public void clean() { + if (deletionFile != null) { + fileHandler.deleteIndexFile(deletionFile); + } + } + } + + /** A lazy generation implementation of {@link CompactDeletionFile}. */ + class LazyCompactDeletionFile implements CompactDeletionFile { + + private final DeletionVectorsMaintainer maintainer; + + private boolean generated = false; + + public LazyCompactDeletionFile(DeletionVectorsMaintainer maintainer) { + this.maintainer = maintainer; + } + + @Override + public Optional getOrCompute() { + generated = true; + return generateFiles(maintainer).getOrCompute(); + } + + @Override + public CompactDeletionFile mergeOldFile(CompactDeletionFile old) { + if (!(old instanceof LazyCompactDeletionFile)) { + throw new IllegalStateException( + "old should be a LazyCompactDeletionFile, but it is: " + old.getClass()); + } + + if (((LazyCompactDeletionFile) old).generated) { + throw new IllegalStateException("old should not be generated, this is a bug."); + } + + return this; + } + + @Override + public void clean() {} + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/compact/CompactResult.java b/paimon-core/src/main/java/org/apache/paimon/compact/CompactResult.java index 6a004d305202..08d7de5dab7f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/compact/CompactResult.java +++ b/paimon-core/src/main/java/org/apache/paimon/compact/CompactResult.java @@ -20,6 +20,8 @@ import org.apache.paimon.io.DataFileMeta; +import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -31,6 +33,8 @@ public class CompactResult { private final List after; private final List changelog; + @Nullable private CompactDeletionFile deletionFile; + public CompactResult() { this(Collections.emptyList(), Collections.emptyList()); } @@ -62,9 +66,23 @@ public List changelog() { return changelog; } + public void setDeletionFile(@Nullable CompactDeletionFile deletionFile) { + this.deletionFile = deletionFile; + } + + @Nullable + public CompactDeletionFile deletionFile() { + return deletionFile; + } + public void merge(CompactResult that) { before.addAll(that.before); after.addAll(that.after); changelog.addAll(that.changelog); + + if (deletionFile != null || that.deletionFile != null) { + throw new UnsupportedOperationException( + "There is a bug, deletionFile can't be set before merge."); + } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorsMaintainer.java b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorsMaintainer.java index 6e04f492293e..8cc5639943ba 100644 --- a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorsMaintainer.java +++ b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorsMaintainer.java @@ -89,13 +89,12 @@ public void removeDeletionVectorOf(String fileName) { } /** - * Prepares to commit: write new deletion vectors index file if any modifications have been - * made. + * Write new deletion vectors index file if any modifications have been made. * * @return A list containing the metadata of the deletion vectors index file, or an empty list * if no changes need to be committed. */ - public List prepareCommit() { + public List writeDeletionVectorsIndex() { if (modified) { modified = false; return indexFileHandler.writeDeletionVectorsIndex(deletionVectors); @@ -114,6 +113,10 @@ public Optional deletionVectorOf(String fileName) { return Optional.ofNullable(deletionVectors.get(fileName)); } + public IndexFileHandler indexFileHandler() { + return indexFileHandler; + } + @VisibleForTesting public Map deletionVectors() { return deletionVectors; 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 684f20952b40..ae4c6f51b7d4 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 @@ -173,12 +173,27 @@ public List readManifestWithIOException(String indexManifest return indexManifestFile.readWithIOException(indexManifest); } + private IndexFile indexFile(IndexFileMeta file) { + switch (file.indexType()) { + case HASH_INDEX: + return hashIndex; + case DELETION_VECTORS_INDEX: + return deletionVectorsIndex; + default: + throw new IllegalArgumentException("Unknown index type: " + file.indexType()); + } + } + public boolean existsIndexFile(IndexManifestEntry file) { - return hashIndex.exists(file.indexFile().fileName()); + return indexFile(file.indexFile()).exists(file.indexFile().fileName()); } public void deleteIndexFile(IndexManifestEntry file) { - hashIndex.delete(file.indexFile().fileName()); + deleteIndexFile(file.indexFile()); + } + + public void deleteIndexFile(IndexFileMeta file) { + indexFile(file).delete(file.fileName()); } public void deleteManifest(String indexManifest) { diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/MergeTreeWriter.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/MergeTreeWriter.java index a42952914eb0..cf2e6c25159b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/MergeTreeWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/MergeTreeWriter.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions.ChangelogProducer; import org.apache.paimon.KeyValue; import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.compact.CompactDeletionFile; import org.apache.paimon.compact.CompactManager; import org.apache.paimon.compact.CompactResult; import org.apache.paimon.data.InternalRow; @@ -78,6 +79,8 @@ public class MergeTreeWriter implements RecordWriter, MemoryOwner { private final LinkedHashSet compactAfter; private final LinkedHashSet compactChangelog; + @Nullable private CompactDeletionFile compactDeletionFile; + private long newSequenceNumber; private WriteBuffer writeBuffer; @@ -128,6 +131,7 @@ public MergeTreeWriter( .forEach(f -> compactBefore.put(f.fileName(), f)); compactAfter.addAll(increment.compactIncrement().compactAfter()); compactChangelog.addAll(increment.compactIncrement().changelogFiles()); + updateCompactDeletionFile(increment.compactDeletionFile()); } } @@ -283,6 +287,7 @@ private CommitIncrement drainIncrement() { new ArrayList<>(compactBefore.values()), new ArrayList<>(compactAfter), new ArrayList<>(compactChangelog)); + CompactDeletionFile drainDeletionFile = this.compactDeletionFile; newFiles.clear(); deletedFiles.clear(); @@ -290,8 +295,14 @@ private CommitIncrement drainIncrement() { compactBefore.clear(); compactAfter.clear(); compactChangelog.clear(); + this.compactDeletionFile = null; - return new CommitIncrement(dataIncrement, compactIncrement); + return new CommitIncrement(dataIncrement, compactIncrement, drainDeletionFile); + } + + private void trySyncLatestCompaction(boolean blocking) throws Exception { + Optional result = compactManager.getCompactionResult(blocking); + result.ifPresent(this::updateCompactResult); } private void updateCompactResult(CompactResult result) { @@ -314,11 +325,17 @@ private void updateCompactResult(CompactResult result) { } compactAfter.addAll(result.after()); compactChangelog.addAll(result.changelog()); + + updateCompactDeletionFile(result.deletionFile()); } - private void trySyncLatestCompaction(boolean blocking) throws Exception { - Optional result = compactManager.getCompactionResult(blocking); - result.ifPresent(this::updateCompactResult); + private void updateCompactDeletionFile(@Nullable CompactDeletionFile newDeletionFile) { + if (newDeletionFile != null) { + compactDeletionFile = + compactDeletionFile == null + ? newDeletionFile + : newDeletionFile.mergeOldFile(compactDeletionFile); + } } @Override @@ -356,5 +373,9 @@ public void close() throws Exception { for (DataFileMeta file : delete) { writerFactory.deleteFile(file.fileName(), file.level()); } + + if (compactDeletionFile != null) { + compactDeletionFile.clean(); + } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManager.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManager.java index 80316eacbdee..28853a12381f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManager.java @@ -20,10 +20,12 @@ import org.apache.paimon.KeyValueFileStore; import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.compact.CompactDeletionFile; import org.apache.paimon.compact.CompactFutureManager; import org.apache.paimon.compact.CompactResult; import org.apache.paimon.compact.CompactUnit; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.deletionvectors.DeletionVectorsMaintainer; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.mergetree.LevelSortedRun; import org.apache.paimon.mergetree.Levels; @@ -42,6 +44,7 @@ import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.function.Supplier; import java.util.stream.Collectors; /** Compact manager for {@link KeyValueFileStore}. */ @@ -58,7 +61,8 @@ public class MergeTreeCompactManager extends CompactFutureManager { private final CompactRewriter rewriter; @Nullable private final CompactionMetrics.Reporter metricsReporter; - private final boolean deletionVectorsEnabled; + @Nullable private final DeletionVectorsMaintainer dvMaintainer; + private final boolean lazyGenDeletionFile; public MergeTreeCompactManager( ExecutorService executor, @@ -69,7 +73,8 @@ public MergeTreeCompactManager( int numSortedRunStopTrigger, CompactRewriter rewriter, @Nullable CompactionMetrics.Reporter metricsReporter, - boolean deletionVectorsEnabled) { + @Nullable DeletionVectorsMaintainer dvMaintainer, + boolean lazyGenDeletionFile) { this.executor = executor; this.levels = levels; this.strategy = strategy; @@ -78,7 +83,8 @@ public MergeTreeCompactManager( this.keyComparator = keyComparator; this.rewriter = rewriter; this.metricsReporter = metricsReporter; - this.deletionVectorsEnabled = deletionVectorsEnabled; + this.dvMaintainer = dvMaintainer; + this.lazyGenDeletionFile = lazyGenDeletionFile; MetricUtils.safeCall(this::reportLevel0FileCount, LOG); } @@ -149,7 +155,7 @@ public void triggerCompaction(boolean fullCompaction) { boolean dropDelete = unit.outputLevel() != 0 && (unit.outputLevel() >= levels.nonEmptyHighestLevel() - || deletionVectorsEnabled); + || dvMaintainer != null); if (LOG.isDebugEnabled()) { LOG.debug( @@ -175,6 +181,14 @@ public Levels levels() { } private void submitCompaction(CompactUnit unit, boolean dropDelete) { + Supplier compactDfSupplier = () -> null; + if (dvMaintainer != null) { + compactDfSupplier = + lazyGenDeletionFile + ? () -> CompactDeletionFile.lazyGeneration(dvMaintainer) + : () -> CompactDeletionFile.generateFiles(dvMaintainer); + } + MergeTreeCompactTask task = new MergeTreeCompactTask( keyComparator, @@ -183,7 +197,8 @@ private void submitCompaction(CompactUnit unit, boolean dropDelete) { unit, dropDelete, levels.maxLevel(), - metricsReporter); + metricsReporter, + compactDfSupplier); if (LOG.isDebugEnabled()) { LOG.debug( "Pick these files (name, level, size) for compaction: {}", diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactTask.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactTask.java index 5f96743fee04..a1b64072d817 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactTask.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactTask.java @@ -18,6 +18,7 @@ package org.apache.paimon.mergetree.compact; +import org.apache.paimon.compact.CompactDeletionFile; import org.apache.paimon.compact.CompactResult; import org.apache.paimon.compact.CompactTask; import org.apache.paimon.compact.CompactUnit; @@ -31,6 +32,7 @@ import java.util.ArrayList; import java.util.Comparator; import java.util.List; +import java.util.function.Supplier; import static java.util.Collections.singletonList; @@ -40,6 +42,7 @@ public class MergeTreeCompactTask extends CompactTask { private final long minFileSize; private final CompactRewriter rewriter; private final int outputLevel; + private final Supplier compactDfSupplier; private final List> partitioned; @@ -56,11 +59,13 @@ public MergeTreeCompactTask( CompactUnit unit, boolean dropDelete, int maxLevel, - @Nullable CompactionMetrics.Reporter metricsReporter) { + @Nullable CompactionMetrics.Reporter metricsReporter, + Supplier compactDfSupplier) { super(metricsReporter); this.minFileSize = minFileSize; this.rewriter = rewriter; this.outputLevel = unit.outputLevel(); + this.compactDfSupplier = compactDfSupplier; this.partitioned = new IntervalPartition(unit.files(), keyComparator).partition(); this.dropDelete = dropDelete; this.maxLevel = maxLevel; @@ -98,6 +103,7 @@ protected CompactResult doCompact() throws Exception { } } rewrite(candidate, result); + result.setDeletionFile(compactDfSupplier.get()); return result; } diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldCountAgg.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldCountAgg.java index f03290cb81a9..5fb7b2bb9e75 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldCountAgg.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldCountAgg.java @@ -38,7 +38,7 @@ String name() { public Object agg(Object accumulator, Object inputField) { Object count; if (accumulator == null || inputField == null) { - count = (accumulator == null ? 1 : accumulator); + count = (accumulator == null ? (inputField == null ? 0 : 1) : accumulator); } else { // ordered by type root definition switch (fieldType.getTypeRoot()) { @@ -59,7 +59,7 @@ public Object agg(Object accumulator, Object inputField) { public Object retract(Object accumulator, Object inputField) { Object count; if (accumulator == null || inputField == null) { - count = (accumulator == null ? 1 : accumulator); + count = (accumulator == null ? (inputField == null ? 0 : -1) : accumulator); } else { // ordered by type root definition switch (fieldType.getTypeRoot()) { diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java index bd32d22e0119..0fd062d64ca7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java @@ -20,6 +20,7 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.compact.CompactDeletionFile; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.deletionvectors.DeletionVectorsMaintainer; import org.apache.paimon.disk.IOManager; @@ -200,8 +201,9 @@ public List prepareCommit(boolean waitCompaction, long commitIden if (writerContainer.indexMaintainer != null) { newIndexFiles.addAll(writerContainer.indexMaintainer.prepareCommit()); } - if (writerContainer.deletionVectorsMaintainer != null) { - newIndexFiles.addAll(writerContainer.deletionVectorsMaintainer.prepareCommit()); + CompactDeletionFile compactDeletionFile = increment.compactDeletionFile(); + if (compactDeletionFile != null) { + compactDeletionFile.getOrCompute().ifPresent(newIndexFiles::add); } CommitMessageImpl committable = new CommitMessageImpl( diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java index 1bdc3042d628..d693c431b102 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java @@ -249,7 +249,8 @@ private CompactManager createCompactManager( compactionMetrics == null ? null : compactionMetrics.createReporter(partition, bucket), - options.deletionVectorsEnabled()); + dvMaintainer, + options.prepareCommitWaitCompaction()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/PartitionExpire.java b/paimon-core/src/main/java/org/apache/paimon/operation/PartitionExpire.java index 94a35cb54e1e..f99ff57afd8d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/PartitionExpire.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/PartitionExpire.java @@ -87,7 +87,7 @@ void setLastCheck(LocalDateTime time) { @VisibleForTesting void expire(LocalDateTime now, long commitIdentifier) { - if (now.isAfter(lastCheck.plus(checkInterval))) { + if (checkInterval.isZero() || now.isAfter(lastCheck.plus(checkInterval))) { doExpire(now.minus(expirationTime), commitIdentifier); lastCheck = now; } diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaMergingUtils.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaMergingUtils.java index a60e6128f76a..9591175e0a44 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaMergingUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaMergingUtils.java @@ -130,6 +130,7 @@ public static DataType merge( updateFields.stream() .filter(field -> !baseFieldMap.containsKey(field.name())) .map(field -> assignIdForNewField(field, highestFieldId)) + .map(field -> field.copy(true)) .collect(Collectors.toList()); updatedFields.addAll(newFields); diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java index 05f77e09fc2e..791d38b2cee2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java @@ -60,6 +60,7 @@ import static org.apache.paimon.CoreOptions.SCAN_MODE; import static org.apache.paimon.CoreOptions.SCAN_SNAPSHOT_ID; import static org.apache.paimon.CoreOptions.SCAN_TAG_NAME; +import static org.apache.paimon.CoreOptions.SCAN_TIMESTAMP; import static org.apache.paimon.CoreOptions.SCAN_TIMESTAMP_MILLIS; import static org.apache.paimon.CoreOptions.SCAN_WATERMARK; import static org.apache.paimon.CoreOptions.SNAPSHOT_NUM_RETAINED_MAX; @@ -216,8 +217,8 @@ private static void validateOnlyContainPrimitiveType( private static void validateStartupMode(CoreOptions options) { if (options.startupMode() == CoreOptions.StartupMode.FROM_TIMESTAMP) { - checkOptionExistInMode( - options, SCAN_TIMESTAMP_MILLIS, CoreOptions.StartupMode.FROM_TIMESTAMP); + checkExactOneOptionExistInMode( + options, options.startupMode(), SCAN_TIMESTAMP_MILLIS, SCAN_TIMESTAMP); checkOptionsConflict( options, Arrays.asList( @@ -226,7 +227,7 @@ private static void validateStartupMode(CoreOptions options) { SCAN_TAG_NAME, INCREMENTAL_BETWEEN_TIMESTAMP, INCREMENTAL_BETWEEN), - Collections.singletonList(SCAN_TIMESTAMP_MILLIS)); + Arrays.asList(SCAN_TIMESTAMP_MILLIS, SCAN_TIMESTAMP)); } else if (options.startupMode() == CoreOptions.StartupMode.FROM_SNAPSHOT) { checkExactOneOptionExistInMode( options, @@ -238,6 +239,7 @@ private static void validateStartupMode(CoreOptions options) { options, Arrays.asList( SCAN_TIMESTAMP_MILLIS, + SCAN_TIMESTAMP, SCAN_FILE_CREATION_TIME_MILLIS, INCREMENTAL_BETWEEN_TIMESTAMP, INCREMENTAL_BETWEEN), @@ -254,6 +256,7 @@ private static void validateStartupMode(CoreOptions options) { SCAN_SNAPSHOT_ID, SCAN_TIMESTAMP_MILLIS, SCAN_FILE_CREATION_TIME_MILLIS, + SCAN_TIMESTAMP, SCAN_TAG_NAME), Arrays.asList(INCREMENTAL_BETWEEN, INCREMENTAL_BETWEEN_TIMESTAMP)); } else if (options.startupMode() == CoreOptions.StartupMode.FROM_SNAPSHOT_FULL) { @@ -262,6 +265,7 @@ private static void validateStartupMode(CoreOptions options) { options, Arrays.asList( SCAN_TIMESTAMP_MILLIS, + SCAN_TIMESTAMP, SCAN_FILE_CREATION_TIME_MILLIS, SCAN_TAG_NAME, INCREMENTAL_BETWEEN_TIMESTAMP, @@ -285,6 +289,7 @@ private static void validateStartupMode(CoreOptions options) { checkOptionNotExistInMode(options, SCAN_TIMESTAMP_MILLIS, options.startupMode()); checkOptionNotExistInMode( options, SCAN_FILE_CREATION_TIME_MILLIS, options.startupMode()); + checkOptionNotExistInMode(options, SCAN_TIMESTAMP, options.startupMode()); checkOptionNotExistInMode(options, SCAN_SNAPSHOT_ID, options.startupMode()); checkOptionNotExistInMode(options, SCAN_TAG_NAME, options.startupMode()); checkOptionNotExistInMode( diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java index bb29a0bd7dda..99a9298d3324 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java @@ -228,6 +228,9 @@ private TableScan.Plan tablePlan(FileStoreTable storeTable) { String[] partFields = partitionStr.split(", "); LinkedHashMap partSpec = new LinkedHashMap<>(); List partitionKeys = storeTable.partitionKeys(); + if (partitionKeys.size() != partFields.length) { + return Collections::emptyList; + } for (int i = 0; i < partitionKeys.size(); i++) { partSpec.put(partitionKeys.get(i), partFields[i]); } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/CommitIncrement.java b/paimon-core/src/main/java/org/apache/paimon/utils/CommitIncrement.java index a51f87f75e7e..3c16378f8f57 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/CommitIncrement.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/CommitIncrement.java @@ -18,18 +18,26 @@ package org.apache.paimon.utils; +import org.apache.paimon.compact.CompactDeletionFile; import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataIncrement; +import javax.annotation.Nullable; + /** Changes to commit. */ public class CommitIncrement { private final DataIncrement dataIncrement; private final CompactIncrement compactIncrement; + @Nullable private final CompactDeletionFile compactDeletionFile; - public CommitIncrement(DataIncrement dataIncrement, CompactIncrement compactIncrement) { + public CommitIncrement( + DataIncrement dataIncrement, + CompactIncrement compactIncrement, + @Nullable CompactDeletionFile compactDeletionFile) { this.dataIncrement = dataIncrement; this.compactIncrement = compactIncrement; + this.compactDeletionFile = compactDeletionFile; } public DataIncrement newFilesIncrement() { @@ -40,8 +48,13 @@ public CompactIncrement compactIncrement() { return compactIncrement; } + @Nullable + public CompactDeletionFile compactDeletionFile() { + return compactDeletionFile; + } + @Override public String toString() { - return dataIncrement.toString() + "\n" + compactIncrement; + return dataIncrement.toString() + "\n" + compactIncrement + "\n" + compactDeletionFile; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsFile.java b/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsFile.java index c05e5d80d9d3..a457a868aa0b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsFile.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsFile.java @@ -68,6 +68,10 @@ public ObjectsFile( cache == null ? null : new ObjectsCache<>(cache, serializer, this::createIterator); } + public FileIO fileIO() { + return fileIO; + } + public long fileSize(String fileName) { try { return fileIO.getFileSize(pathFactory.toPath(fileName)); diff --git a/paimon-core/src/test/java/org/apache/paimon/CoreOptionsTest.java b/paimon-core/src/test/java/org/apache/paimon/CoreOptionsTest.java index b7b2f9a54cce..b8d134916e9e 100644 --- a/paimon-core/src/test/java/org/apache/paimon/CoreOptionsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/CoreOptionsTest.java @@ -38,6 +38,11 @@ public void testDefaultStartupMode() { conf.set(CoreOptions.SCAN_TIMESTAMP_MILLIS, System.currentTimeMillis()); assertThat(new CoreOptions(conf).startupMode()) .isEqualTo(CoreOptions.StartupMode.FROM_TIMESTAMP); + + conf = new Options(); + conf.set(CoreOptions.SCAN_TIMESTAMP, "2023-12-06 12:12:12"); + assertThat(new CoreOptions(conf).startupMode()) + .isEqualTo(CoreOptions.StartupMode.FROM_TIMESTAMP); } @Test diff --git a/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java b/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java index fd9f784734f2..6ca3b3a919e7 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java @@ -144,7 +144,7 @@ public CommitMessageImpl writeDVIndexFiles( bucket, DataIncrement.emptyIncrement(), CompactIncrement.emptyIncrement(), - new IndexIncrement(dvMaintainer.prepareCommit())); + new IndexIncrement(dvMaintainer.writeDeletionVectorsIndex())); } public static TestAppendFileStore createAppendStore( diff --git a/paimon-core/src/test/java/org/apache/paimon/deletionvectors/DeletionVectorsMaintainerTest.java b/paimon-core/src/test/java/org/apache/paimon/deletionvectors/DeletionVectorsMaintainerTest.java index dd6d3102c899..308a1ea97153 100644 --- a/paimon-core/src/test/java/org/apache/paimon/deletionvectors/DeletionVectorsMaintainerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/deletionvectors/DeletionVectorsMaintainerTest.java @@ -19,6 +19,7 @@ package org.apache.paimon.deletionvectors; import org.apache.paimon.catalog.PrimaryKeyTableTestBase; +import org.apache.paimon.compact.CompactDeletionFile; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.index.IndexFileHandler; import org.apache.paimon.index.IndexFileMeta; @@ -28,10 +29,13 @@ import org.apache.paimon.table.sink.BatchTableCommit; import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.CommitMessageImpl; +import org.apache.paimon.utils.FileIOUtils; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.io.File; +import java.io.IOException; import java.util.Collections; import java.util.List; import java.util.Map; @@ -61,7 +65,7 @@ public void test0() { assertThat(dvMaintainer.deletionVectorOf("f1")).isPresent(); assertThat(dvMaintainer.deletionVectorOf("f3")).isEmpty(); - List fileMetas = dvMaintainer.prepareCommit(); + List fileMetas = dvMaintainer.writeDeletionVectorsIndex(); Map deletionVectors = fileHandler.readAllDeletionVectors(fileMetas); assertThat(deletionVectors.get("f1").isDeleted(1)).isTrue(); @@ -83,7 +87,7 @@ public void test1() { deletionVector1.delete(5); dvMaintainer.notifyNewDeletion("f1", deletionVector1); - List fileMetas1 = dvMaintainer.prepareCommit(); + List fileMetas1 = dvMaintainer.writeDeletionVectorsIndex(); assertThat(fileMetas1.size()).isEqualTo(1); CommitMessage commitMessage = new CommitMessageImpl( @@ -104,7 +108,7 @@ public void test1() { deletionVector2.delete(2); dvMaintainer.notifyNewDeletion("f1", deletionVector2); - List fileMetas2 = dvMaintainer.prepareCommit(); + List fileMetas2 = dvMaintainer.writeDeletionVectorsIndex(); assertThat(fileMetas2.size()).isEqualTo(1); commitMessage = new CommitMessageImpl( @@ -122,4 +126,44 @@ public void test1() { assertThat(deletionVector3.isDeleted(1)).isTrue(); assertThat(deletionVector3.isDeleted(2)).isTrue(); } + + @Test + public void testCompactDeletion() throws IOException { + DeletionVectorsMaintainer.Factory factory = + new DeletionVectorsMaintainer.Factory(fileHandler); + DeletionVectorsMaintainer dvMaintainer = + factory.createOrRestore(null, BinaryRow.EMPTY_ROW, 0); + + File indexDir = new File(tempPath.toFile(), "/default.db/T/index"); + + // test generate files + + dvMaintainer.notifyNewDeletion("f1", 1); + CompactDeletionFile deletionFile1 = CompactDeletionFile.generateFiles(dvMaintainer); + assertThat(indexDir.listFiles()).hasSize(1); + + dvMaintainer.notifyNewDeletion("f2", 4); + CompactDeletionFile deletionFile2 = CompactDeletionFile.generateFiles(dvMaintainer); + assertThat(indexDir.listFiles()).hasSize(2); + + deletionFile2.mergeOldFile(deletionFile1); + assertThat(indexDir.listFiles()).hasSize(1); + FileIOUtils.deleteDirectory(indexDir); + + // test lazyGeneration + + dvMaintainer.notifyNewDeletion("f1", 3); + CompactDeletionFile deletionFile3 = CompactDeletionFile.lazyGeneration(dvMaintainer); + assertThat(indexDir.listFiles()).isNull(); + + dvMaintainer.notifyNewDeletion("f2", 5); + CompactDeletionFile deletionFile4 = CompactDeletionFile.lazyGeneration(dvMaintainer); + assertThat(indexDir.listFiles()).isNull(); + + deletionFile4.mergeOldFile(deletionFile3); + assertThat(indexDir.listFiles()).isNull(); + + deletionFile4.getOrCompute(); + assertThat(indexDir.listFiles()).hasSize(1); + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/MergeTreeTestBase.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/MergeTreeTestBase.java index 33b81a37319b..92f14b3ed41c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/MergeTreeTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/MergeTreeTestBase.java @@ -458,6 +458,7 @@ private MergeTreeCompactManager createCompactManager( options.numSortedRunStopTrigger(), new TestRewriter(), null, + null, false); } @@ -479,6 +480,7 @@ public MockFailResultCompactionManager( numSortedRunStopTrigger, rewriter, null, + null, false); } diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManagerTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManagerTest.java index d5aee3ccc727..8484525c82c8 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManagerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/MergeTreeCompactManagerTest.java @@ -206,6 +206,7 @@ private void innerTest( Integer.MAX_VALUE, new TestRewriter(expectedDropDelete), null, + null, false); manager.triggerCompaction(false); manager.getCompactionResult(true); diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/aggregate/FieldAggregatorTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/aggregate/FieldAggregatorTest.java index e164bda31f21..5bca65ed74a4 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/aggregate/FieldAggregatorTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/aggregate/FieldAggregatorTest.java @@ -155,10 +155,11 @@ public void testFieldSumIntAgg() { @Test public void testFieldCountIntAgg() { FieldCountAgg fieldCountAgg = new FieldCountAgg(new IntType()); - assertThat(fieldCountAgg.agg(null, 10)).isEqualTo(1); - assertThat(fieldCountAgg.agg(1, 5)).isEqualTo(2); - assertThat(fieldCountAgg.agg(2, 15)).isEqualTo(3); - assertThat(fieldCountAgg.agg(3, 25)).isEqualTo(4); + assertThat(fieldCountAgg.agg(null, null)).isEqualTo(0); + assertThat(fieldCountAgg.agg(1, null)).isEqualTo(1); + assertThat(fieldCountAgg.agg(null, 15)).isEqualTo(1); + assertThat(fieldCountAgg.agg(1, 0)).isEqualTo(2); + assertThat(fieldCountAgg.agg(3, 6)).isEqualTo(4); } @Test diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaValidationTest.java b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaValidationTest.java new file mode 100644 index 000000000000..1bfe45879acb --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaValidationTest.java @@ -0,0 +1,106 @@ +/* + * 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.schema; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypes; + +import org.assertj.core.api.ThrowableAssert; +import org.junit.jupiter.api.Test; + +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.paimon.CoreOptions.BUCKET; +import static org.apache.paimon.CoreOptions.SCAN_SNAPSHOT_ID; +import static org.apache.paimon.schema.SchemaValidation.validateTableSchema; +import static org.assertj.core.api.Assertions.assertThatNoException; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +class SchemaValidationTest { + + private void validateTableSchemaExec(Map options) { + List fields = + Arrays.asList( + new DataField(0, "f0", DataTypes.INT()), + new DataField(1, "f1", DataTypes.INT()), + new DataField(2, "f2", DataTypes.INT())); + List partitionKeys = Collections.singletonList("f0"); + List primaryKeys = Collections.singletonList("f1"); + options.put(BUCKET.key(), String.valueOf(-1)); + validateTableSchema( + new TableSchema(1, fields, 10, partitionKeys, primaryKeys, options, "")); + } + + @Test + public void testOnlyTimestampMillis() { + Map options = new HashMap<>(); + options.put(CoreOptions.SCAN_MODE.key(), CoreOptions.StartupMode.FROM_TIMESTAMP.toString()); + options.put( + CoreOptions.SCAN_TIMESTAMP_MILLIS.key(), + String.valueOf(System.currentTimeMillis())); + ThrowableAssert.ThrowingCallable validate = () -> validateTableSchemaExec(options); + assertThatNoException().isThrownBy(validate); + } + + @Test + public void testOnlyTimestamp() { + String timestampString = + LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS")); + Map options = new HashMap<>(); + options.put(CoreOptions.SCAN_MODE.key(), CoreOptions.StartupMode.FROM_TIMESTAMP.toString()); + options.put(CoreOptions.SCAN_TIMESTAMP.key(), timestampString); + ThrowableAssert.ThrowingCallable validate = () -> validateTableSchemaExec(options); + assertThatNoException().isThrownBy(validate); + } + + @Test + public void testFromTimestampConflict() { + String timestampString = + LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS")); + Map options = new HashMap<>(); + options.put(CoreOptions.SCAN_MODE.key(), CoreOptions.StartupMode.FROM_TIMESTAMP.toString()); + options.put(CoreOptions.SCAN_TIMESTAMP.key(), timestampString); + options.put( + CoreOptions.SCAN_TIMESTAMP_MILLIS.key(), + String.valueOf(System.currentTimeMillis())); + assertThatThrownBy(() -> validateTableSchemaExec(options)) + .hasMessageContaining( + "must set only one key in [scan.timestamp-millis,scan.timestamp] when you use from-timestamp for scan.mode"); + } + + @Test + public void testFromSnapshotConflict() { + String timestampString = + LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS")); + Map options = new HashMap<>(); + options.put(CoreOptions.SCAN_MODE.key(), CoreOptions.StartupMode.FROM_TIMESTAMP.toString()); + options.put(CoreOptions.SCAN_TIMESTAMP.key(), timestampString); + options.put(SCAN_SNAPSHOT_ID.key(), String.valueOf(-1)); + assertThatThrownBy(() -> validateTableSchemaExec(options)) + .hasMessageContaining( + "[scan.snapshot-id] must be null when you set [scan.timestamp-millis,scan.timestamp]"); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java index 5eb0b5db6549..baf3beb8f105 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java @@ -26,7 +26,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.serializer.InternalRowSerializer; import org.apache.paimon.fileindex.FileIndexOptions; -import org.apache.paimon.fileindex.bloomfilter.BloomFilterFileIndex; +import org.apache.paimon.fileindex.bloomfilter.BloomFilterFileIndexFactory; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; @@ -386,26 +386,26 @@ public void testBloomFilterInMemory() throws Exception { options.set( FileIndexOptions.FILE_INDEX + "." - + BloomFilterFileIndex.BLOOM_FILTER + + BloomFilterFileIndexFactory.BLOOM_FILTER + "." + CoreOptions.COLUMNS, "index_column, index_column2, index_column3"); options.set( FileIndexOptions.FILE_INDEX + "." - + BloomFilterFileIndex.BLOOM_FILTER + + BloomFilterFileIndexFactory.BLOOM_FILTER + ".index_column.items", "150"); options.set( FileIndexOptions.FILE_INDEX + "." - + BloomFilterFileIndex.BLOOM_FILTER + + BloomFilterFileIndexFactory.BLOOM_FILTER + ".index_column2.items", "150"); options.set( FileIndexOptions.FILE_INDEX + "." - + BloomFilterFileIndex.BLOOM_FILTER + + BloomFilterFileIndexFactory.BLOOM_FILTER + ".index_column3.items", "150"); options.set(FILE_INDEX_IN_MANIFEST_THRESHOLD.key(), "500 B"); @@ -452,7 +452,7 @@ public void testBloomFilterInDisk() throws Exception { options.set( FileIndexOptions.FILE_INDEX + "." - + BloomFilterFileIndex.BLOOM_FILTER + + BloomFilterFileIndexFactory.BLOOM_FILTER + "." + CoreOptions.COLUMNS, "index_column, index_column2, index_column3"); @@ -527,32 +527,32 @@ public void testBloomFilterForMapField() throws Exception { options.set( FileIndexOptions.FILE_INDEX + "." - + BloomFilterFileIndex.BLOOM_FILTER + + BloomFilterFileIndexFactory.BLOOM_FILTER + "." + CoreOptions.COLUMNS, "index_column, index_column2, index_column3[a], index_column3[b], index_column3[c], index_column3[d]"); options.set( FileIndexOptions.FILE_INDEX + "." - + BloomFilterFileIndex.BLOOM_FILTER + + BloomFilterFileIndexFactory.BLOOM_FILTER + ".index_column.items", "150"); options.set( FileIndexOptions.FILE_INDEX + "." - + BloomFilterFileIndex.BLOOM_FILTER + + BloomFilterFileIndexFactory.BLOOM_FILTER + ".index_column2.items", "150"); options.set( FileIndexOptions.FILE_INDEX + "." - + BloomFilterFileIndex.BLOOM_FILTER + + BloomFilterFileIndexFactory.BLOOM_FILTER + ".index_column3.items", "150"); options.set( FileIndexOptions.FILE_INDEX + "." - + BloomFilterFileIndex.BLOOM_FILTER + + BloomFilterFileIndexFactory.BLOOM_FILTER + ".index_column3[a].items", "10000"); }); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/StartupModeTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/StartupModeTest.java index 7608be614df7..a31bbd6fc4e4 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/StartupModeTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/StartupModeTest.java @@ -35,6 +35,8 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -143,6 +145,38 @@ public void testStartFromTimestamp() throws Exception { .isEqualTo(snapshotReader.withSnapshot(3).withMode(ScanMode.ALL).read().splits()); } + @Test + public void testStartFromTimestampString() throws Exception { + initializeTable(StartupMode.LATEST); + initializeTestData(); // initialize 3 commits + String timestampString = + LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS")); + Thread.sleep(10L); + + // write next data + writeAndCommit(4, rowData(1, 10, 103L)); + + Map properties = new HashMap<>(); + properties.put(CoreOptions.SCAN_MODE.key(), StartupMode.FROM_TIMESTAMP.toString()); + properties.put(CoreOptions.SCAN_TIMESTAMP.key(), timestampString); + FileStoreTable readTable = table.copy(properties); + + // streaming Mode + StreamTableScan dataTableScan = readTable.newStreamScan(); + TableScan.Plan firstPlan = dataTableScan.plan(); + TableScan.Plan secondPlan = dataTableScan.plan(); + + assertThat(firstPlan.splits()).isEmpty(); + assertThat(secondPlan.splits()) + .isEqualTo(snapshotReader.withSnapshot(4).withMode(ScanMode.DELTA).read().splits()); + + // batch mode + TableScan batchScan = readTable.newScan(); + TableScan.Plan plan = batchScan.plan(); + assertThat(plan.splits()) + .isEqualTo(snapshotReader.withSnapshot(3).withMode(ScanMode.ALL).read().splits()); + } + @Test public void testStartFromCompactedFull() throws Exception { initializeTable(StartupMode.COMPACTED_FULL); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java index 488434013443..c0383efd584f 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java @@ -22,7 +22,7 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.serializer.InternalRowSerializer; -import org.apache.paimon.fileindex.bloomfilter.BloomFilterFileIndex; +import org.apache.paimon.fileindex.bloomfilter.BloomFilterFileIndexFactory; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; @@ -352,7 +352,7 @@ private FileStoreTable createFileStoreTable( options.set( CoreOptions.FILE_INDEX + "." - + BloomFilterFileIndex.BLOOM_FILTER + + BloomFilterFileIndexFactory.BLOOM_FILTER + "." + CoreOptions.COLUMNS, rowType.getFieldNames().get(0)); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java index afd660fd3471..33fd8feebca5 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java @@ -74,10 +74,11 @@ public void before() throws Exception { Schema schema = Schema.newBuilder() .column("pk", DataTypes.INT()) - .column("pt", DataTypes.INT()) + .column("pt1", DataTypes.INT()) + .column("pt2", DataTypes.INT()) .column("col1", DataTypes.INT()) - .partitionKeys("pt") - .primaryKey("pk", "pt") + .partitionKeys("pt1", "pt2") + .primaryKey("pk", "pt1", "pt2") .option(CoreOptions.CHANGELOG_PRODUCER.key(), "input") .option(CoreOptions.BUCKET.key(), "2") .option(CoreOptions.SEQUENCE_FIELD.key(), "col1") @@ -93,24 +94,27 @@ public void before() throws Exception { snapshotManager = new SnapshotManager(fileIO, tablePath); // snapshot 1: append - write(table, GenericRow.of(1, 1, 1), GenericRow.of(1, 2, 5)); + write(table, GenericRow.of(1, 1, 10, 1), GenericRow.of(1, 2, 20, 5)); // snapshot 2: append - write(table, GenericRow.of(2, 1, 3), GenericRow.of(2, 2, 4)); + write(table, GenericRow.of(2, 1, 10, 3), GenericRow.of(2, 2, 20, 4)); } @Test public void testReadWithFilter() throws Exception { - compact(table, row(2), 0); - write(table, GenericRow.of(3, 1, 1)); + compact(table, row(2, 20), 0); + write(table, GenericRow.of(3, 1, 10, 1)); assertThat(readPartBucketLevel(null)) - .containsExactlyInAnyOrder("[1]-0-0", "[1]-0-0", "[1]-1-0", "[2]-0-5"); + .containsExactlyInAnyOrder( + "[1, 10]-0-0", "[1, 10]-0-0", "[1, 10]-1-0", "[2, 20]-0-5"); PredicateBuilder builder = new PredicateBuilder(FilesTable.TABLE_TYPE); - assertThat(readPartBucketLevel(builder.equal(0, "[2]"))) - .containsExactlyInAnyOrder("[2]-0-5"); - assertThat(readPartBucketLevel(builder.equal(1, 1))).containsExactlyInAnyOrder("[1]-1-0"); - assertThat(readPartBucketLevel(builder.equal(5, 5))).containsExactlyInAnyOrder("[2]-0-5"); + assertThat(readPartBucketLevel(builder.equal(0, "[2, 20]"))) + .containsExactlyInAnyOrder("[2, 20]-0-5"); + assertThat(readPartBucketLevel(builder.equal(1, 1))) + .containsExactlyInAnyOrder("[1, 10]-1-0"); + assertThat(readPartBucketLevel(builder.equal(5, 5))) + .containsExactlyInAnyOrder("[2, 20]-0-5"); } private List readPartBucketLevel(Predicate predicate) throws IOException { @@ -137,6 +141,12 @@ public void testReadFilesFromLatest() throws Exception { assertThat(result).containsExactlyInAnyOrderElementsOf(expectedRow); } + @Test + public void testReadWithNotFullPartitionKey() throws Exception { + PredicateBuilder builder = new PredicateBuilder(FilesTable.TABLE_TYPE); + assertThat(readPartBucketLevel(builder.equal(0, "[2]"))).isEmpty(); + } + @Test public void testReadFilesFromSpecifiedSnapshot() throws Exception { List expectedRow = getExceptedResult(1L); @@ -169,15 +179,17 @@ private List getExceptedResult(long snapshotId) { List expectedRow = new ArrayList<>(); for (ManifestEntry fileEntry : files) { - String partition = String.valueOf(fileEntry.partition().getInt(0)); + String partition1 = String.valueOf(fileEntry.partition().getInt(0)); + String partition2 = String.valueOf(fileEntry.partition().getInt(1)); DataFileMeta file = fileEntry.file(); String minKey = String.valueOf(file.minKey().getInt(0)); String maxKey = String.valueOf(file.maxKey().getInt(0)); - String minCol1 = String.valueOf(file.valueStats().minValues().getInt(2)); - String maxCol1 = String.valueOf(file.valueStats().maxValues().getInt(2)); + String minCol1 = String.valueOf(file.valueStats().minValues().getInt(3)); + String maxCol1 = String.valueOf(file.valueStats().maxValues().getInt(3)); expectedRow.add( GenericRow.of( - BinaryString.fromString(Arrays.toString(new String[] {partition})), + BinaryString.fromString( + Arrays.toString(new String[] {partition1, partition2})), fileEntry.bucket(), BinaryString.fromString(file.fileName()), BinaryString.fromString("orc"), @@ -188,13 +200,15 @@ private List getExceptedResult(long snapshotId) { BinaryString.fromString(Arrays.toString(new String[] {minKey})), BinaryString.fromString(Arrays.toString(new String[] {maxKey})), BinaryString.fromString( - String.format("{col1=%s, pk=%s, pt=%s}", 0, 0, 0)), + String.format("{col1=%s, pk=%s, pt1=%s, pt2=%s}", 0, 0, 0, 0)), BinaryString.fromString( String.format( - "{col1=%s, pk=%s, pt=%s}", minCol1, minKey, partition)), + "{col1=%s, pk=%s, pt1=%s, pt2=%s}", + minCol1, minKey, partition1, partition2)), BinaryString.fromString( String.format( - "{col1=%s, pk=%s, pt=%s}", maxCol1, maxKey, partition)), + "{col1=%s, pk=%s, pt1=%s, pt2=%s}", + maxCol1, maxKey, partition1, partition2)), file.minSequenceNumber(), file.maxSequenceNumber(), file.creationTime())); diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/TraceableFileIO.java b/paimon-core/src/test/java/org/apache/paimon/utils/TraceableFileIO.java index 623fe3147562..eb616a9ab294 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/TraceableFileIO.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/TraceableFileIO.java @@ -32,7 +32,6 @@ import javax.annotation.concurrent.GuardedBy; import java.io.IOException; -import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; @@ -251,15 +250,25 @@ public String toString() { } public static List openInputStreams(Predicate filter) { - // copy out to avoid ConcurrentModificationException - return new ArrayList<>(OPEN_INPUT_STREAMS) - .stream().filter(s -> filter.test(s.file)).collect(Collectors.toList()); + LOCK.lock(); + try { + return OPEN_INPUT_STREAMS.stream() + .filter(s -> filter.test(s.file)) + .collect(Collectors.toList()); + } finally { + LOCK.unlock(); + } } public static List openOutputStreams(Predicate filter) { - // copy out to avoid ConcurrentModificationException - return new ArrayList<>(OPEN_OUTPUT_STREAMS) - .stream().filter(s -> filter.test(s.file)).collect(Collectors.toList()); + LOCK.lock(); + try { + return OPEN_OUTPUT_STREAMS.stream() + .filter(s -> filter.test(s.file)) + .collect(Collectors.toList()); + } finally { + LOCK.unlock(); + } } /** Loader for {@link TraceableFileIO}. */ diff --git a/paimon-flink/paimon-flink-1.14/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java b/paimon-flink/paimon-flink-1.14/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java new file mode 100644 index 000000000000..315f2520f247 --- /dev/null +++ b/paimon-flink/paimon-flink-1.14/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java @@ -0,0 +1,180 @@ +/* + * 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; + +import org.apache.paimon.CoreOptions; + +import org.apache.flink.types.Row; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** ITCase for batch file store. */ +public class BatchFileStoreITCase extends CatalogITCaseBase { + + @Override + protected List ddl() { + return Collections.singletonList("CREATE TABLE IF NOT EXISTS T (a INT, b INT, c INT)"); + } + + @Override + protected int defaultParallelism() { + return 1; + } + + @Test + public void testOverwriteEmpty() { + batchSql("INSERT INTO T VALUES (1, 11, 111), (2, 22, 222)"); + assertThat(batchSql("SELECT * FROM T")) + .containsExactlyInAnyOrder(Row.of(1, 11, 111), Row.of(2, 22, 222)); + batchSql("INSERT OVERWRITE T SELECT * FROM T WHERE 1 <> 1"); + assertThat(batchSql("SELECT * FROM T")).isEmpty(); + } + + @Test + public void testTimeTravelRead() throws Exception { + batchSql("INSERT INTO T VALUES (1, 11, 111), (2, 22, 222)"); + long time1 = System.currentTimeMillis(); + + Thread.sleep(10); + batchSql("INSERT INTO T VALUES (3, 33, 333), (4, 44, 444)"); + long time2 = System.currentTimeMillis(); + + Thread.sleep(10); + batchSql("INSERT INTO T VALUES (5, 55, 555), (6, 66, 666)"); + long time3 = System.currentTimeMillis(); + + Thread.sleep(10); + batchSql("INSERT INTO T VALUES (7, 77, 777), (8, 88, 888)"); + + paimonTable("T").createTag("tag2", 2); + + assertThat(batchSql("SELECT * FROM T /*+ OPTIONS('scan.snapshot-id'='1') */")) + .containsExactlyInAnyOrder(Row.of(1, 11, 111), Row.of(2, 22, 222)); + assertThat( + batchSql( + "SELECT * FROM T /*+ OPTIONS('scan.mode'='from-snapshot-full','scan.snapshot-id'='1') */")) + .containsExactlyInAnyOrder(Row.of(1, 11, 111), Row.of(2, 22, 222)); + + assertThat(batchSql("SELECT * FROM T /*+ OPTIONS('scan.snapshot-id'='0') */")).isEmpty(); + assertThat( + batchSql( + "SELECT * FROM T /*+ OPTIONS('scan.mode'='from-snapshot-full','scan.snapshot-id'='0') */")) + .isEmpty(); + + assertThat( + batchSql( + String.format( + "SELECT * FROM T /*+ OPTIONS('scan.timestamp-millis'='%s') */", + time1))) + .containsExactlyInAnyOrder(Row.of(1, 11, 111), Row.of(2, 22, 222)); + + assertThat(batchSql("SELECT * FROM T /*+ OPTIONS('scan.snapshot-id'='2') */")) + .containsExactlyInAnyOrder( + Row.of(1, 11, 111), + Row.of(2, 22, 222), + Row.of(3, 33, 333), + Row.of(4, 44, 444)); + assertThat( + batchSql( + "SELECT * FROM T /*+ OPTIONS('scan.mode'='from-snapshot-full','scan.snapshot-id'='2') */")) + .containsExactlyInAnyOrder( + Row.of(1, 11, 111), + Row.of(2, 22, 222), + Row.of(3, 33, 333), + Row.of(4, 44, 444)); + assertThat( + batchSql( + String.format( + "SELECT * FROM T /*+ OPTIONS('scan.timestamp-millis'='%s') */", + time2))) + .containsExactlyInAnyOrder( + Row.of(1, 11, 111), + Row.of(2, 22, 222), + Row.of(3, 33, 333), + Row.of(4, 44, 444)); + + assertThat(batchSql("SELECT * FROM T /*+ OPTIONS('scan.snapshot-id'='3') */")) + .containsExactlyInAnyOrder( + Row.of(1, 11, 111), + Row.of(2, 22, 222), + Row.of(3, 33, 333), + Row.of(4, 44, 444), + Row.of(5, 55, 555), + Row.of(6, 66, 666)); + assertThat( + batchSql( + "SELECT * FROM T /*+ OPTIONS('scan.mode'='from-snapshot-full','scan.snapshot-id'='3') */")) + .containsExactlyInAnyOrder( + Row.of(1, 11, 111), + Row.of(2, 22, 222), + Row.of(3, 33, 333), + Row.of(4, 44, 444), + Row.of(5, 55, 555), + Row.of(6, 66, 666)); + assertThat( + batchSql( + String.format( + "SELECT * FROM T /*+ OPTIONS('scan.timestamp-millis'='%s') */", + time3))) + .containsExactlyInAnyOrder( + Row.of(1, 11, 111), + Row.of(2, 22, 222), + Row.of(3, 33, 333), + Row.of(4, 44, 444), + Row.of(5, 55, 555), + Row.of(6, 66, 666)); + + assertThatThrownBy( + () -> + batchSql( + String.format( + "SELECT * FROM T /*+ OPTIONS('scan.timestamp-millis'='%s', 'scan.snapshot-id'='1') */", + time3))) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage( + "[scan.snapshot-id] must be null when you set [scan.timestamp-millis,scan.timestamp]"); + + assertThatThrownBy( + () -> + batchSql( + "SELECT * FROM T /*+ OPTIONS('scan.mode'='full', 'scan.snapshot-id'='1') */")) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage( + "%s must be null when you use latest-full for scan.mode", + CoreOptions.SCAN_SNAPSHOT_ID.key()); + + // travel to tag + assertThat(batchSql("SELECT * FROM T /*+ OPTIONS('scan.tag-name'='tag2') */")) + .containsExactlyInAnyOrder( + Row.of(1, 11, 111), + Row.of(2, 22, 222), + Row.of(3, 33, 333), + Row.of(4, 44, 444)); + + assertThatThrownBy( + () -> batchSql("SELECT * FROM T /*+ OPTIONS('scan.tag-name'='unknown') */")) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Tag 'unknown' doesn't exist."); + } +} diff --git a/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java b/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java index abf2564a1b5b..2902fd5d726b 100644 --- a/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java +++ b/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java @@ -153,7 +153,7 @@ public void testTimeTravelRead() throws Exception { time3))) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage( - "[scan.snapshot-id] must be null when you set [scan.timestamp-millis]"); + "[scan.snapshot-id] must be null when you set [scan.timestamp-millis,scan.timestamp]"); assertThatThrownBy( () -> diff --git a/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/procedure/ProcedurePositionalArgumentsITCase.java b/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/procedure/ProcedurePositionalArgumentsITCase.java index d47947caa248..443b60003bc7 100644 --- a/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/procedure/ProcedurePositionalArgumentsITCase.java +++ b/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/procedure/ProcedurePositionalArgumentsITCase.java @@ -19,9 +19,15 @@ package org.apache.paimon.flink.procedure; import org.apache.paimon.flink.CatalogITCaseBase; +import org.apache.paimon.table.FileStoreTable; import org.junit.jupiter.api.Test; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatCode; /** Ensure that the legacy multiply overloaded CALL with positional arguments can be invoked. */ @@ -48,4 +54,30 @@ public void testCallCompact() { assertThatCode(() -> sql("CALL sys.compact('default.T', '', '', '', 'sink.parallelism=1')")) .doesNotThrowAnyException(); } + + @Test + public void testExpirePartitionsProcedure() throws Exception { + sql( + "CREATE TABLE T (" + + " k STRING," + + " dt STRING," + + " PRIMARY KEY (k, dt) NOT ENFORCED" + + ") PARTITIONED BY (dt) WITH (" + + " 'bucket' = '1'" + + ")"); + FileStoreTable table = paimonTable("T"); + sql("INSERT INTO T VALUES ('1', '2024-06-01')"); + sql("INSERT INTO T VALUES ('2', '9024-06-01')"); + assertThat(read(table)).containsExactlyInAnyOrder("1:2024-06-01", "2:9024-06-01"); + sql("CALL sys.expire_partitions('default.T', '1 d', 'yyyy-MM-dd')"); + assertThat(read(table)).containsExactlyInAnyOrder("2:9024-06-01"); + } + + private List read(FileStoreTable table) throws IOException { + List ret = new ArrayList<>(); + table.newRead() + .createReader(table.newScan().plan().splits()) + .forEachRemaining(row -> ret.add(row.getString(0) + ":" + row.getString(1))); + return ret; + } } diff --git a/paimon-flink/paimon-flink-1.19/pom.xml b/paimon-flink/paimon-flink-1.19/pom.xml index 3542979a1be0..301cb0a10a49 100644 --- a/paimon-flink/paimon-flink-1.19/pom.xml +++ b/paimon-flink/paimon-flink-1.19/pom.xml @@ -34,7 +34,7 @@ under the License. Paimon : Flink : 1.19 - 1.19.0 + 1.19.1 diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactory.java index efd1f432e56d..8a9ec42a2542 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactory.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactory.java @@ -23,6 +23,7 @@ import org.apache.paimon.flink.log.LogStoreRegister; import org.apache.paimon.flink.log.LogStoreTableFactory; import org.apache.paimon.options.Options; +import org.apache.paimon.utils.DateTimeUtils; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; @@ -41,11 +42,13 @@ import java.util.List; import java.util.Properties; +import java.util.TimeZone; import static org.apache.kafka.clients.consumer.ConsumerConfig.ISOLATION_LEVEL_CONFIG; import static org.apache.paimon.CoreOptions.LOG_CHANGELOG_MODE; import static org.apache.paimon.CoreOptions.LOG_CONSISTENCY; import static org.apache.paimon.CoreOptions.LogConsistency; +import static org.apache.paimon.CoreOptions.SCAN_TIMESTAMP; import static org.apache.paimon.CoreOptions.SCAN_TIMESTAMP_MILLIS; import static org.apache.paimon.flink.factories.FlinkFactoryUtil.createFlinkTableFactoryHelper; import static org.apache.paimon.flink.kafka.KafkaLogOptions.TOPIC; @@ -87,6 +90,14 @@ public KafkaLogSourceProvider createSourceProvider( LogStoreTableFactory.getValueDecodingFormat(helper) .createRuntimeDecoder(sourceContext, physicalType); Options options = toOptions(helper.getOptions()); + Long timestampMills = options.get(SCAN_TIMESTAMP_MILLIS); + String timestampString = options.get(SCAN_TIMESTAMP); + + if (timestampMills == null && timestampString != null) { + timestampMills = + DateTimeUtils.parseTimestampData(timestampString, 3, TimeZone.getDefault()) + .getMillisecond(); + } return new KafkaLogSourceProvider( topic(context), toKafkaProperties(options), @@ -98,7 +109,7 @@ public KafkaLogSourceProvider createSourceProvider( options.get(LOG_CONSISTENCY), // TODO visit all options through CoreOptions CoreOptions.startupMode(options), - options.get(SCAN_TIMESTAMP_MILLIS)); + timestampMills); } @Override 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 51cd3e220404..535c3d255f5a 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 @@ -19,7 +19,6 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.catalog.Catalog; -import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.flink.sink.CommittableStateManager; import org.apache.paimon.flink.sink.Committer; import org.apache.paimon.flink.sink.CommitterOperator; @@ -88,8 +87,7 @@ private StoreSinkWrite.WithWriteBufferProvider createWriteProvider() { state, ioManager, isOverwrite, - FlinkConnectorOptions.prepareCommitWaitCompaction( - table.coreOptions().toConfiguration()), + table.coreOptions().prepareCommitWaitCompaction(), true, memoryPoolFactory, metricGroup); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactoryTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactoryTest.java index 0b127491ac20..a3aa12fcc706 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactoryTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/kafka/KafkaLogStoreFactoryTest.java @@ -43,13 +43,17 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; +import java.util.function.Consumer; import static org.apache.paimon.CoreOptions.SCAN_MODE; import static org.apache.paimon.CoreOptions.SCAN_SNAPSHOT_ID; +import static org.apache.paimon.CoreOptions.SCAN_TIMESTAMP; import static org.apache.paimon.CoreOptions.SCAN_TIMESTAMP_MILLIS; import static org.assertj.core.api.Assertions.assertThat; @@ -58,7 +62,25 @@ public class KafkaLogStoreFactoryTest { @ParameterizedTest @EnumSource(CoreOptions.StartupMode.class) - public void testCreateKafkaLogStoreFactory(CoreOptions.StartupMode startupMode) { + public void testCreateKafkaLogStoreFactoryTimestamp(CoreOptions.StartupMode startupMode) { + String now = String.valueOf(System.currentTimeMillis()); + Consumer> setter = + (options) -> options.put(SCAN_TIMESTAMP_MILLIS.key(), now); + testCreateKafkaLogStoreFactory(startupMode, setter); + } + + @ParameterizedTest + @EnumSource(CoreOptions.StartupMode.class) + public void testCreateKafkaLogStoreFactoryTimestampStr(CoreOptions.StartupMode startupMode) { + DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); + String timestampString = LocalDateTime.now().format(formatter); + Consumer> setter = + (options) -> options.put(SCAN_TIMESTAMP.key(), timestampString); + testCreateKafkaLogStoreFactory(startupMode, setter); + } + + private static void testCreateKafkaLogStoreFactory( + CoreOptions.StartupMode startupMode, Consumer> optionsSetter) { Map dynamicOptions = new HashMap<>(); dynamicOptions.put(FlinkConnectorOptions.LOG_SYSTEM.key(), "kafka"); dynamicOptions.put(SCAN_MODE.key(), startupMode.toString()); @@ -66,8 +88,7 @@ public void testCreateKafkaLogStoreFactory(CoreOptions.StartupMode startupMode) || startupMode == CoreOptions.StartupMode.FROM_SNAPSHOT_FULL) { dynamicOptions.put(SCAN_SNAPSHOT_ID.key(), "1"); } else if (startupMode == CoreOptions.StartupMode.FROM_TIMESTAMP) { - dynamicOptions.put( - SCAN_TIMESTAMP_MILLIS.key(), String.valueOf(System.currentTimeMillis())); + optionsSetter.accept(dynamicOptions); } dynamicOptions.put(SCAN_MODE.key(), startupMode.toString()); DynamicTableFactory.Context context = diff --git a/paimon-flink/paimon-flink-common/pom.xml b/paimon-flink/paimon-flink-common/pom.xml index 09ea2bb9c678..54ef1516acd0 100644 --- a/paimon-flink/paimon-flink-common/pom.xml +++ b/paimon-flink/paimon-flink-common/pom.xml @@ -34,7 +34,7 @@ under the License. Paimon : Flink : Common - 1.19.0 + 1.19.1 diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java index c821ab445460..e5138cd4825b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java @@ -25,7 +25,6 @@ import org.apache.paimon.options.ConfigOption; import org.apache.paimon.options.ConfigOptions; import org.apache.paimon.options.MemorySize; -import org.apache.paimon.options.Options; import org.apache.paimon.options.description.DescribedEnum; import org.apache.paimon.options.description.Description; import org.apache.paimon.options.description.InlineElement; @@ -36,7 +35,6 @@ import java.util.ArrayList; import java.util.List; -import static org.apache.paimon.CoreOptions.DELETION_VECTORS_ENABLED; import static org.apache.paimon.CoreOptions.STREAMING_READ_MODE; import static org.apache.paimon.options.ConfigOptions.key; import static org.apache.paimon.options.description.TextElement.text; @@ -148,17 +146,6 @@ public class FlinkConnectorOptions { + ChangelogProducer.FULL_COMPACTION.name() + ", full compaction will be constantly triggered after this interval."); - public static final ConfigOption CHANGELOG_PRODUCER_LOOKUP_WAIT = - key("changelog-producer.lookup-wait") - .booleanType() - .defaultValue(true) - .withDescription( - "When " - + CoreOptions.CHANGELOG_PRODUCER.key() - + " is set to " - + ChangelogProducer.LOOKUP.name() - + ", commit will wait for changelog generation by lookup."); - public static final ConfigOption SCAN_WATERMARK_EMIT_STRATEGY = key("scan.watermark.emit.strategy") .enumType(WatermarkEmitStrategy.class) @@ -446,20 +433,6 @@ public static List> getOptions() { return list; } - public static boolean prepareCommitWaitCompaction(Options options) { - if (options.get(DELETION_VECTORS_ENABLED)) { - // DeletionVector (DV) is maintained in the compaction thread, but it needs to be - // read into a file during prepareCommit (write thread) to commit it. - // We must set waitCompaction to true so that there are no multiple threads - // operating DV simultaneously. - return true; - } - - ChangelogProducer changelogProducer = options.get(CoreOptions.CHANGELOG_PRODUCER); - return changelogProducer == ChangelogProducer.LOOKUP - && options.get(CHANGELOG_PRODUCER_LOOKUP_WAIT); - } - /** The mode of lookup cache. */ public enum LookupCacheMode { /** Auto mode, try to use partial mode. */ diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java index 3fd73d2b96d7..65c9c7302f0c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java @@ -94,6 +94,8 @@ public class FileStoreLookupFunction implements Serializable, Closeable { // timestamp when cache expires private transient long nextLoadTime; + protected FunctionContext functionContext; + public FileStoreLookupFunction( Table table, int[] projection, int[] joinKeyIndex, @Nullable Predicate predicate) { TableScanUtils.streamingReadingValidate(table); @@ -127,6 +129,7 @@ public FileStoreLookupFunction( } public void open(FunctionContext context) throws Exception { + this.functionContext = context; String tmpDirectory = getTmpDirectory(context); open(tmpDirectory); } @@ -352,7 +355,7 @@ private static StreamingRuntimeContext extractStreamingRuntimeContext(Object run * * @return the set of bucket IDs to be cached */ - private Set getRequireCachedBucketIds() { + protected Set getRequireCachedBucketIds() { // TODO: Implement the method when Flink support bucket shuffle for lookup join. return null; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java index d7af8bdf71f3..ed5c4c13f61e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java @@ -41,8 +41,6 @@ import org.apache.paimon.utils.TypeUtils; import org.apache.paimon.utils.UserDefinedSeqComparator; -import org.apache.paimon.shade.guava30.com.google.common.util.concurrent.MoreExecutors; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -78,7 +76,7 @@ public abstract class FullCacheLookupTable implements LookupTable { protected final int appendUdsFieldNumber; protected RocksDBStateFactory stateFactory; - private final ExecutorService refreshExecutor; + @Nullable private final ExecutorService refreshExecutor; private final AtomicReference cachedException; private final int maxPendingSnapshotCount; private final FileStoreTable table; @@ -87,7 +85,6 @@ public abstract class FullCacheLookupTable implements LookupTable { private Predicate specificPartition; public FullCacheLookupTable(Context context) { - this.context = context; this.table = context.table; List sequenceFields = new ArrayList<>(); if (table.primaryKeys().size() > 0) { @@ -108,6 +105,7 @@ public FullCacheLookupTable(Context context) { builder.field(f.name(), f.type()); }); projectedType = builder.build(); + context = context.copy(table.rowType().getFieldIndices(projectedType.getFieldNames())); this.userDefinedSeqComparator = UserDefinedSeqComparator.create(projectedType, sequenceFields); this.appendUdsFieldNumber = appendUdsFieldNumber.get(); @@ -116,6 +114,8 @@ public FullCacheLookupTable(Context context) { this.appendUdsFieldNumber = 0; } + this.context = context; + Options options = Options.fromMap(context.table.options()); this.projectedType = projectedType; this.refreshAsync = options.get(LOOKUP_REFRESH_ASYNC); @@ -126,7 +126,7 @@ public FullCacheLookupTable(Context context) { String.format( "%s-lookup-refresh", Thread.currentThread().getName()))) - : MoreExecutors.newDirectExecutorService(); + : null; this.cachedException = new AtomicReference<>(); this.maxPendingSnapshotCount = options.get(LOOKUP_REFRESH_ASYNC_PENDING_SNAPSHOT_COUNT); } @@ -187,6 +187,11 @@ protected void bootstrap() throws Exception { @Override public void refresh() throws Exception { + if (refreshExecutor == null) { + doRefresh(); + return; + } + Long latestSnapshotId = table.snapshotManager().latestSnapshotId(); Long nextSnapshotId = reader.nextSnapshotId(); if (latestSnapshotId != null @@ -219,11 +224,8 @@ public void refresh() throws Exception { cachedException.set(e); } }); - } catch (RejectedExecutionException ignored) { - LOG.warn( - "Add refresh task for lookup table {} failed", - context.table.name(), - ignored); + } catch (RejectedExecutionException e) { + LOG.warn("Add refresh task for lookup table {} failed", context.table.name(), e); } if (currentFuture != null) { refreshFuture = currentFuture; @@ -300,7 +302,9 @@ public void close() throws IOException { stateFactory.close(); FileIOUtils.deleteDirectory(context.tempPath); } finally { - refreshExecutor.shutdown(); + if (refreshExecutor != null) { + refreshExecutor.shutdown(); + } } } @@ -352,5 +356,16 @@ public Context( this.joinKey = joinKey; this.requiredCachedBucketIds = requiredCachedBucketIds; } + + public Context copy(int[] newProjection) { + return new Context( + table, + newProjection, + tablePredicate, + projectedPredicate, + tempPath, + joinKey, + requiredCachedBucketIds); + } } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedure.java new file mode 100644 index 000000000000..6e21b3522db2 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedure.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.procedure; + +import org.apache.paimon.FileStore; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.operation.PartitionExpire; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.utils.TimeUtils; + +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.time.Duration; + +/** A procedure to expire partitions. */ +public class ExpirePartitionsProcedure extends ProcedureBase { + @Override + public String identifier() { + return "expire_partitions"; + } + + @ProcedureHint( + argument = { + @ArgumentHint(name = "table", type = @DataTypeHint("STRING")), + @ArgumentHint(name = "expiration_time", type = @DataTypeHint(value = "STRING")), + @ArgumentHint(name = "timestamp_formatter", type = @DataTypeHint("STRING")) + }) + public String[] call( + ProcedureContext procedureContext, + String tableId, + String expirationTime, + String timestampFormatter) + throws Catalog.TableNotExistException { + FileStoreTable fileStoreTable = (FileStoreTable) table(tableId); + FileStore fileStore = fileStoreTable.store(); + PartitionExpire partitionExpire = + new PartitionExpire( + fileStore.partitionType(), + TimeUtils.parseDuration(expirationTime), + Duration.ofMillis(0L), + null, + timestampFormatter, + fileStore.newScan(), + fileStore.newCommit("")); + partitionExpire.expire(Long.MAX_VALUE); + return new String[] {}; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java index 3638a924ec86..f369ec31c3d5 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java @@ -18,9 +18,9 @@ package org.apache.paimon.flink.sink; +import org.apache.paimon.CoreOptions; import org.apache.paimon.CoreOptions.ChangelogProducer; import org.apache.paimon.CoreOptions.TagCreationMode; -import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.options.MemorySize; import org.apache.paimon.options.Options; @@ -62,7 +62,6 @@ import static org.apache.paimon.flink.FlinkConnectorOptions.SINK_COMMITTER_OPERATOR_CHAINING; import static org.apache.paimon.flink.FlinkConnectorOptions.SINK_MANAGED_WRITER_BUFFER_MEMORY; import static org.apache.paimon.flink.FlinkConnectorOptions.SINK_USE_MANAGED_MEMORY; -import static org.apache.paimon.flink.FlinkConnectorOptions.prepareCommitWaitCompaction; import static org.apache.paimon.flink.utils.ManagedMemoryUtils.declareManagedMemory; import static org.apache.paimon.utils.Preconditions.checkArgument; @@ -99,10 +98,11 @@ private StoreSinkWrite.Provider createWriteProvider( Options options = table.coreOptions().toConfiguration(); ChangelogProducer changelogProducer = table.coreOptions().changelogProducer(); boolean waitCompaction; - if (table.coreOptions().writeOnly()) { + CoreOptions coreOptions = table.coreOptions(); + if (coreOptions.writeOnly()) { waitCompaction = false; } else { - waitCompaction = prepareCommitWaitCompaction(options); + waitCompaction = coreOptions.prepareCommitWaitCompaction(); int deltaCommits = -1; if (options.contains(FULL_COMPACTION_DELTA_COMMITS)) { deltaCommits = options.get(FULL_COMPACTION_DELTA_COMMITS); @@ -135,7 +135,7 @@ private StoreSinkWrite.Provider createWriteProvider( } if (changelogProducer == ChangelogProducer.LOOKUP - && !options.get(FlinkConnectorOptions.CHANGELOG_PRODUCER_LOOKUP_WAIT)) { + && !coreOptions.prepareCommitWaitCompaction()) { return (table, commitUser, state, ioManager, memoryPool, metricGroup) -> { assertNoSinkMaterializer.run(); return new AsyncLookupSinkWrite( 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 25e80d524e9a..52e494b5a9db 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 @@ -22,7 +22,6 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; -import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFileMetaSerializer; import org.apache.paimon.options.Options; @@ -46,7 +45,6 @@ import static org.apache.paimon.CoreOptions.FULL_COMPACTION_DELTA_COMMITS; import static org.apache.paimon.flink.FlinkConnectorOptions.CHANGELOG_PRODUCER_FULL_COMPACTION_TRIGGER_INTERVAL; -import static org.apache.paimon.flink.FlinkConnectorOptions.prepareCommitWaitCompaction; import static org.apache.paimon.utils.SerializationUtils.deserializeBinaryRow; /** @@ -238,10 +236,11 @@ private StoreSinkWrite.Provider createWriteProvider( CoreOptions.ChangelogProducer changelogProducer = fileStoreTable.coreOptions().changelogProducer(); boolean waitCompaction; - if (fileStoreTable.coreOptions().writeOnly()) { + CoreOptions coreOptions = fileStoreTable.coreOptions(); + if (coreOptions.writeOnly()) { waitCompaction = false; } else { - waitCompaction = prepareCommitWaitCompaction(options); + waitCompaction = coreOptions.prepareCommitWaitCompaction(); int deltaCommits = -1; if (options.contains(FULL_COMPACTION_DELTA_COMMITS)) { deltaCommits = options.get(FULL_COMPACTION_DELTA_COMMITS); @@ -273,7 +272,7 @@ private StoreSinkWrite.Provider createWriteProvider( } if (changelogProducer == CoreOptions.ChangelogProducer.LOOKUP - && !options.get(FlinkConnectorOptions.CHANGELOG_PRODUCER_LOOKUP_WAIT)) { + && !coreOptions.prepareCommitWaitCompaction()) { return (table, commitUser, state, ioManager, memoryPool, metricGroup) -> new AsyncLookupSinkWrite( table, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CompactorSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CompactorSourceBuilder.java index 2b9e524665ff..275f88ad751c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CompactorSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CompactorSourceBuilder.java @@ -158,6 +158,7 @@ private Map batchCompactOptions() { { put(CoreOptions.SCAN_TIMESTAMP_MILLIS.key(), null); put(CoreOptions.SCAN_FILE_CREATION_TIME_MILLIS.key(), null); + put(CoreOptions.SCAN_TIMESTAMP.key(), null); put(CoreOptions.SCAN_SNAPSHOT_ID.key(), null); put(CoreOptions.SCAN_MODE.key(), CoreOptions.StartupMode.LATEST_FULL.toString()); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/MultiTablesCompactorUtil.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/MultiTablesCompactorUtil.java index 0fa20f192ce2..fc648b4681e6 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/MultiTablesCompactorUtil.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/MultiTablesCompactorUtil.java @@ -50,6 +50,7 @@ public static Map compactOptions(boolean isStreaming) { return new HashMap() { { put(CoreOptions.SCAN_TIMESTAMP_MILLIS.key(), null); + put(CoreOptions.SCAN_TIMESTAMP.key(), null); put(CoreOptions.SCAN_FILE_CREATION_TIME_MILLIS.key(), null); put(CoreOptions.SCAN_SNAPSHOT_ID.key(), null); put( 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 192be6c14faa..847390dacf7d 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 @@ -46,6 +46,7 @@ org.apache.paimon.flink.procedure.MigrateFileProcedure 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.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 5c502d88feaf..4176aec39e9c 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 @@ -200,7 +200,7 @@ public void testTimeTravelRead() throws Exception { time3))) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage( - "[scan.snapshot-id] must be null when you set [scan.timestamp-millis]"); + "[scan.snapshot-id] must be null when you set [scan.timestamp-millis,scan.timestamp]"); assertThatThrownBy( () -> diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java index de3ab0c5f698..ff987fddd65b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java @@ -335,7 +335,7 @@ public void testLackStartupTimestamp() { "SELECT * FROM T1 /*+ OPTIONS('log.scan'='from-timestamp') */")) .hasCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage( - "scan.timestamp-millis can not be null when you use from-timestamp for scan.mode"); + "must set only one key in [scan.timestamp-millis,scan.timestamp] when you use from-timestamp for scan.mode"); } @Test 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 c096371d30af..85bee4bb55a4 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 @@ -311,8 +311,10 @@ public void testLimit() { public void testFileIndex() { batchSql( "INSERT INTO index_table VALUES (1, 'a', 'AAA'), (1, 'a', 'AAA'), (2, 'c', 'BBB'), (3, 'c', 'BBB')"); + batchSql( + "INSERT INTO index_table VALUES (1, 'a', 'AAA'), (1, 'a', 'AAA'), (2, 'd', 'BBB'), (3, 'd', 'BBB')"); - assertThat(batchSql("SELECT * FROM index_table WHERE indexc = 'c'")) + assertThat(batchSql("SELECT * FROM index_table WHERE indexc = 'c' and (id = 2 or id = 3)")) .containsExactlyInAnyOrder(Row.of(2, "c", "BBB"), Row.of(3, "c", "BBB")); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/LookupTableTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/LookupTableTest.java index 7cca8e25d0b5..14643542e73d 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/LookupTableTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/LookupTableTest.java @@ -222,6 +222,46 @@ public void testPkTableWithSequenceField() throws Exception { assertRow(result.get(0), 1, 22, 222); } + @Test + public void testPkTableWithSequenceFieldProjection() throws Exception { + Options options = new Options(); + options.set(CoreOptions.SEQUENCE_FIELD, "f2"); + options.set(CoreOptions.BUCKET, 1); + FileStoreTable storeTable = createTable(singletonList("f0"), options); + FullCacheLookupTable.Context context = + new FullCacheLookupTable.Context( + storeTable, + new int[] {0, 1}, + null, + null, + tempDir.toFile(), + singletonList("f0"), + null); + table = FullCacheLookupTable.create(context, ThreadLocalRandom.current().nextInt(2) * 10); + table.open(); + + // first write + write(storeTable, GenericRow.of(1, 11, 111)); + table.refresh(); + List result = table.get(row(1)); + assertThat(result).hasSize(1); + assertRow(result.get(0), 1, 11); + + // second write + write(storeTable, GenericRow.of(1, 22, 222)); + table.refresh(); + result = table.get(row(1)); + assertThat(result).hasSize(1); + assertRow(result.get(0), 1, 22); + + // not update + write(storeTable, GenericRow.of(1, 33, 111)); + table.refresh(); + result = table.get(row(1)); + assertThat(result).hasSize(1); + assertRow(result.get(0), 1, 22); + } + @Test public void testPkTablePkFilter() throws Exception { FileStoreTable storeTable = createTable(singletonList("f0"), new Options()); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedureITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedureITCase.java new file mode 100644 index 000000000000..348ec36a551d --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedureITCase.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.procedure; + +import org.apache.paimon.flink.CatalogITCaseBase; +import org.apache.paimon.table.FileStoreTable; + +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** IT Case for {@link ExpirePartitionsProcedure}. */ +public class ExpirePartitionsProcedureITCase extends CatalogITCaseBase { + + @Test + public void testExpirePartitionsProcedure() throws Exception { + sql( + "CREATE TABLE T (" + + " k STRING," + + " dt STRING," + + " PRIMARY KEY (k, dt) NOT ENFORCED" + + ") PARTITIONED BY (dt) WITH (" + + " 'bucket' = '1'" + + ")"); + FileStoreTable table = paimonTable("T"); + sql("INSERT INTO T VALUES ('1', '2024-06-01')"); + sql("INSERT INTO T VALUES ('2', '9024-06-01')"); + assertThat(read(table)).containsExactlyInAnyOrder("1:2024-06-01", "2:9024-06-01"); + sql( + "CALL sys.expire_partitions(`table` => 'default.T', expiration_time => '1 d', timestamp_formatter => 'yyyy-MM-dd')"); + assertThat(read(table)).containsExactlyInAnyOrder("2:9024-06-01"); + } + + private List read(FileStoreTable table) throws IOException { + List ret = new ArrayList<>(); + table.newRead() + .createReader(table.newScan().plan().splits()) + .forEachRemaining(row -> ret.add(row.getString(0) + ":" + row.getString(1))); + return ret; + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java index 024a99c3e5ef..147110637aef 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java @@ -70,19 +70,13 @@ public void testBatchWriteGeneratorTag() throws Exception { SnapshotManager snapshotManager = table.newSnapshotReader().snapshotManager(); TagManager tagManager = table.tagManager(); - // Generate tag name - String prefix = "batch-write-"; - Instant instant = - Instant.ofEpochMilli( - Objects.requireNonNull(snapshotManager.latestSnapshot()).timeMillis()); - LocalDateTime localDateTime = LocalDateTime.ofInstant(instant, ZoneId.systemDefault()); - String tagName = prefix + localDateTime.format(DateTimeFormatter.ofPattern("yyyy-MM-dd")); - // No tag is generated before the finish method assertThat(table.tagManager().tagCount()).isEqualTo(0); committerOperator.finish(); // After the finish method, a tag is generated assertThat(table.tagManager().tagCount()).isEqualTo(1); + // Get tagName from tagManager. + String tagName = tagManager.allTagNames().get(0); // The tag is consistent with the latest snapshot assertThat(tagManager.taggedSnapshot(tagName)).isEqualTo(snapshotManager.latestSnapshot()); @@ -98,7 +92,16 @@ public void testBatchWriteGeneratorTag() throws Exception { // note that this tag has the same name with previous tag // so the previous tag will be deleted committerOperator.finish(); - + // If tagName does not exist, it happened across the day. + if (!tagManager.tagExists(tagName)) { + // Generate tag name + String prefix = "batch-write-"; + Instant instant = + Instant.ofEpochMilli( + Objects.requireNonNull(snapshotManager.latestSnapshot()).timeMillis()); + LocalDateTime localDateTime = LocalDateTime.ofInstant(instant, ZoneId.systemDefault()); + tagName = prefix + localDateTime.format(DateTimeFormatter.ofPattern("yyyy-MM-dd")); + } assertThat(tagManager.allTagNames()).containsOnly("many-tags-test2", tagName); } diff --git a/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroBulkFormat.java b/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroBulkFormat.java index 39bba1a08e06..eda5650b3168 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroBulkFormat.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroBulkFormat.java @@ -40,8 +40,6 @@ /** Provides a {@link FormatReaderFactory} for Avro records. */ public class AvroBulkFormat implements FormatReaderFactory { - private static final long serialVersionUID = 1L; - protected final RowType projectedRowType; public AvroBulkFormat(RowType projectedRowType) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java index 4657caebef2e..5093a5010773 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java @@ -36,6 +36,7 @@ import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.Pool; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory; @@ -57,9 +58,7 @@ /** An ORC reader that produces a stream of {@link ColumnarRow} records. */ public class OrcReaderFactory implements FormatReaderFactory { - private static final long serialVersionUID = 1L; - - protected final SerializableHadoopConfigWrapper hadoopConfigWrapper; + protected final Configuration hadoopConfig; protected final TypeDescription schema; @@ -79,7 +78,7 @@ public OrcReaderFactory( final RowType readType, final List conjunctPredicates, final int batchSize) { - this.hadoopConfigWrapper = new SerializableHadoopConfigWrapper(checkNotNull(hadoopConfig)); + this.hadoopConfig = checkNotNull(hadoopConfig); this.schema = toOrcType(readType); this.tableType = readType; this.conjunctPredicates = checkNotNull(conjunctPredicates); @@ -99,7 +98,7 @@ public OrcVectorizedReader createReader(FormatReaderFactory.Context context) RecordReader orcReader = createRecordReader( - hadoopConfigWrapper.getHadoopConfig(), + hadoopConfig, schema, conjunctPredicates, context.fileIO(), diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetFileFormat.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetFileFormat.java index ffeb9790913c..c8ce5ccf5c9a 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetFileFormat.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetFileFormat.java @@ -31,6 +31,7 @@ import org.apache.paimon.statistics.SimpleColStatsCollector; import org.apache.paimon.types.RowType; +import org.apache.parquet.filter2.predicate.ParquetFilters; import org.apache.parquet.hadoop.ParquetOutputFormat; import java.util.List; @@ -59,7 +60,8 @@ public FormatReaderFactory createReaderFactory( return new ParquetReaderFactory( getParquetConfiguration(formatContext), projectedRowType, - formatContext.readBatchSize()); + formatContext.readBatchSize(), + ParquetFilters.convert(filters)); } @Override diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java index 64e1e2296671..2b2c651a8538 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java @@ -39,6 +39,7 @@ import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.filter2.compat.FilterCompat; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.ParquetInputFormat; import org.apache.parquet.schema.GroupType; @@ -68,21 +69,22 @@ public class ParquetReaderFactory implements FormatReaderFactory { private static final Logger LOG = LoggerFactory.getLogger(ParquetReaderFactory.class); - private static final long serialVersionUID = 1L; - private static final String ALLOCATION_SIZE = "parquet.read.allocation.size"; private final Options conf; private final String[] projectedFields; private final DataType[] projectedTypes; private final int batchSize; + private final FilterCompat.Filter filter; private final Set unknownFieldsIndices = new HashSet<>(); - public ParquetReaderFactory(Options conf, RowType projectedType, int batchSize) { + public ParquetReaderFactory( + Options conf, RowType projectedType, int batchSize, FilterCompat.Filter filter) { this.conf = conf; this.projectedFields = projectedType.getFieldNames().toArray(new String[0]); this.projectedTypes = projectedType.getFieldTypes().toArray(new DataType[0]); this.batchSize = batchSize; + this.filter = filter; } @Override @@ -124,6 +126,7 @@ private void setReadOptions(ParquetReadOptions.Builder builder) { if (badRecordThresh != null) { builder.set(BAD_RECORD_THRESHOLD_CONF_KEY, badRecordThresh); } + builder.withRecordFilter(filter); } /** Clips `parquetSchema` according to `fieldNames`. */ diff --git a/paimon-format/src/main/java/org/apache/parquet/filter2/predicate/ParquetFilters.java b/paimon-format/src/main/java/org/apache/parquet/filter2/predicate/ParquetFilters.java new file mode 100644 index 000000000000..ef36cc6f892a --- /dev/null +++ b/paimon-format/src/main/java/org/apache/parquet/filter2/predicate/ParquetFilters.java @@ -0,0 +1,291 @@ +/* + * 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.parquet.filter2.predicate; + +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.predicate.FieldRef; +import org.apache.paimon.predicate.FunctionVisitor; +import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.BigIntType; +import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BooleanType; +import org.apache.paimon.types.CharType; +import org.apache.paimon.types.DataTypeVisitor; +import org.apache.paimon.types.DateType; +import org.apache.paimon.types.DecimalType; +import org.apache.paimon.types.DoubleType; +import org.apache.paimon.types.FloatType; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.LocalZonedTimestampType; +import org.apache.paimon.types.MapType; +import org.apache.paimon.types.MultisetType; +import org.apache.paimon.types.RowType; +import org.apache.paimon.types.SmallIntType; +import org.apache.paimon.types.TimeType; +import org.apache.paimon.types.TimestampType; +import org.apache.paimon.types.TinyIntType; +import org.apache.paimon.types.VarBinaryType; +import org.apache.paimon.types.VarCharType; + +import org.apache.parquet.filter2.compat.FilterCompat; +import org.apache.parquet.io.api.Binary; + +import java.util.List; + +/** Convert {@link Predicate} to {@link FilterCompat.Filter}. */ +public class ParquetFilters { + + private static final ConvertFilterToParquet CONVERTER = new ConvertFilterToParquet(); + + private ParquetFilters() {} + + public static FilterCompat.Filter convert(List predicates) { + FilterPredicate result = null; + if (predicates != null) { + for (Predicate predicate : predicates) { + try { + FilterPredicate parquetFilter = predicate.visit(CONVERTER); + if (result == null) { + result = parquetFilter; + } else { + result = FilterApi.and(result, parquetFilter); + } + } catch (UnsupportedOperationException ignore) { + } + } + } + + return result != null ? FilterCompat.get(result) : FilterCompat.NOOP; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private static class ConvertFilterToParquet implements FunctionVisitor { + + @Override + public FilterPredicate visitIsNotNull(FieldRef fieldRef) { + return new Operators.NotEq<>(toParquetColumn(fieldRef), null); + } + + @Override + public FilterPredicate visitIsNull(FieldRef fieldRef) { + return new Operators.Eq<>(toParquetColumn(fieldRef), null); + } + + @Override + public FilterPredicate visitLessThan(FieldRef fieldRef, Object literal) { + return new Operators.Lt(toParquetColumn(fieldRef), toParquetObject(literal)); + } + + @Override + public FilterPredicate visitGreaterOrEqual(FieldRef fieldRef, Object literal) { + return new Operators.GtEq(toParquetColumn(fieldRef), toParquetObject(literal)); + } + + @Override + public FilterPredicate visitNotEqual(FieldRef fieldRef, Object literal) { + return new Operators.NotEq(toParquetColumn(fieldRef), toParquetObject(literal)); + } + + @Override + public FilterPredicate visitLessOrEqual(FieldRef fieldRef, Object literal) { + return new Operators.LtEq(toParquetColumn(fieldRef), toParquetObject(literal)); + } + + @Override + public FilterPredicate visitEqual(FieldRef fieldRef, Object literal) { + return new Operators.Eq(toParquetColumn(fieldRef), toParquetObject(literal)); + } + + @Override + public FilterPredicate visitGreaterThan(FieldRef fieldRef, Object literal) { + return new Operators.Gt(toParquetColumn(fieldRef), toParquetObject(literal)); + } + + @Override + public FilterPredicate visitAnd(List children) { + if (children.size() != 2) { + throw new RuntimeException("Illegal and children: " + children.size()); + } + + return FilterApi.and(children.get(0), children.get(1)); + } + + @Override + public FilterPredicate visitOr(List children) { + if (children.size() != 2) { + throw new RuntimeException("Illegal and children: " + children.size()); + } + + return FilterApi.or(children.get(0), children.get(1)); + } + + @Override + public FilterPredicate visitStartsWith(FieldRef fieldRef, Object literal) { + throw new UnsupportedOperationException(); + } + + @Override + public FilterPredicate visitIn(FieldRef fieldRef, List literals) { + throw new UnsupportedOperationException(); + } + + @Override + public FilterPredicate visitNotIn(FieldRef fieldRef, List literals) { + throw new UnsupportedOperationException(); + } + } + + private static Operators.Column toParquetColumn(FieldRef fieldRef) { + return fieldRef.type().accept(new ConvertToColumnTypeVisitor(fieldRef.name())); + } + + private static Comparable toParquetObject(Object value) { + if (value == null) { + return null; + } + + if (value instanceof Number) { + return (Comparable) value; + } else if (value instanceof String) { + return Binary.fromString((String) value); + } else if (value instanceof BinaryString) { + return Binary.fromString(value.toString()); + } else if (value instanceof byte[]) { + return Binary.fromReusedByteArray((byte[]) value); + } + + // TODO Support Decimal and Timestamp + throw new UnsupportedOperationException(); + } + + private static class ConvertToColumnTypeVisitor + implements DataTypeVisitor> { + + private final String name; + + public ConvertToColumnTypeVisitor(String name) { + this.name = name; + } + + @Override + public Operators.Column visit(CharType charType) { + return FilterApi.binaryColumn(name); + } + + @Override + public Operators.Column visit(VarCharType varCharType) { + return FilterApi.binaryColumn(name); + } + + @Override + public Operators.Column visit(BooleanType booleanType) { + return FilterApi.booleanColumn(name); + } + + @Override + public Operators.Column visit(BinaryType binaryType) { + return FilterApi.binaryColumn(name); + } + + @Override + public Operators.Column visit(VarBinaryType varBinaryType) { + return FilterApi.binaryColumn(name); + } + + @Override + public Operators.Column visit(TinyIntType tinyIntType) { + return FilterApi.intColumn(name); + } + + @Override + public Operators.Column visit(SmallIntType smallIntType) { + return FilterApi.intColumn(name); + } + + @Override + public Operators.Column visit(IntType intType) { + return FilterApi.intColumn(name); + } + + @Override + public Operators.Column visit(BigIntType bigIntType) { + return FilterApi.longColumn(name); + } + + @Override + public Operators.Column visit(FloatType floatType) { + return FilterApi.floatColumn(name); + } + + @Override + public Operators.Column visit(DoubleType doubleType) { + return FilterApi.doubleColumn(name); + } + + @Override + public Operators.Column visit(DateType dateType) { + return FilterApi.intColumn(name); + } + + @Override + public Operators.Column visit(TimeType timeType) { + return FilterApi.intColumn(name); + } + + // TODO we can support decimal and timestamp + + @Override + public Operators.Column visit(DecimalType decimalType) { + throw new UnsupportedOperationException(); + } + + @Override + public Operators.Column visit(TimestampType timestampType) { + throw new UnsupportedOperationException(); + } + + @Override + public Operators.Column visit(LocalZonedTimestampType localZonedTimestampType) { + throw new UnsupportedOperationException(); + } + + // ===================== can not support ========================= + + @Override + public Operators.Column visit(ArrayType arrayType) { + throw new UnsupportedOperationException(); + } + + @Override + public Operators.Column visit(MultisetType multisetType) { + throw new UnsupportedOperationException(); + } + + @Override + public Operators.Column visit(MapType mapType) { + throw new UnsupportedOperationException(); + } + + @Override + public Operators.Column visit(RowType rowType) { + throw new UnsupportedOperationException(); + } + } +} diff --git a/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetFiltersTest.java b/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetFiltersTest.java new file mode 100644 index 000000000000..28b165df6c5a --- /dev/null +++ b/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetFiltersTest.java @@ -0,0 +1,111 @@ +/* + * 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.parquet; + +import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.predicate.PredicateBuilder; +import org.apache.paimon.types.BigIntType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.RowType; +import org.apache.paimon.types.VarCharType; + +import org.apache.parquet.filter2.compat.FilterCompat; +import org.apache.parquet.filter2.compat.FilterCompat.FilterPredicateCompat; +import org.apache.parquet.filter2.predicate.ParquetFilters; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.stream.Collectors; +import java.util.stream.LongStream; + +import static org.assertj.core.api.Assertions.assertThat; + +class ParquetFiltersTest { + + @Test + public void testLong() { + PredicateBuilder builder = + new PredicateBuilder( + new RowType( + Collections.singletonList( + new DataField(0, "long1", new BigIntType())))); + + test(builder.isNull(0), "eq(long1, null)", true); + + test(builder.isNotNull(0), "noteq(long1, null)", true); + + test(builder.lessThan(0, 5L), "lt(long1, 5)", true); + + test(builder.greaterThan(0, 5L), "gt(long1, 5)", true); + + test( + builder.in(0, Arrays.asList(1L, 2L, 3L)), + "or(or(eq(long1, 1), eq(long1, 2)), eq(long1, 3))", + true); + + test(builder.between(0, 1L, 3L), "and(gteq(long1, 1), lteq(long1, 3))", true); + + test( + builder.notIn(0, Arrays.asList(1L, 2L, 3L)), + "and(and(noteq(long1, 1), noteq(long1, 2)), noteq(long1, 3))", + true); + + test( + builder.in(0, LongStream.range(1L, 22L).boxed().collect(Collectors.toList())), + "", + false); + + test( + builder.notIn(0, LongStream.range(1L, 22L).boxed().collect(Collectors.toList())), + "", + false); + } + + @Test + public void testString() { + PredicateBuilder builder = + new PredicateBuilder( + new RowType( + Collections.singletonList( + new DataField(0, "string1", new VarCharType())))); + test(builder.isNull(0), "eq(string1, null)", true); + + test(builder.isNotNull(0), "noteq(string1, null)", true); + + test( + builder.in(0, Arrays.asList("1", "2", "3")), + "or(or(eq(string1, Binary{\"1\"}), eq(string1, Binary{\"2\"})), eq(string1, Binary{\"3\"}))", + true); + test( + builder.notIn(0, Arrays.asList("1", "2", "3")), + "and(and(noteq(string1, Binary{\"1\"}), noteq(string1, Binary{\"2\"})), noteq(string1, Binary{\"3\"}))", + true); + } + + private void test(Predicate predicate, String expected, boolean canPushDown) { + FilterCompat.Filter filter = ParquetFilters.convert(PredicateBuilder.splitAnd(predicate)); + if (canPushDown) { + FilterPredicateCompat compat = (FilterPredicateCompat) filter; + assertThat(compat.getFilterPredicate().toString()).isEqualTo(expected); + } else { + assertThat(filter).isEqualTo(FilterCompat.NOOP); + } + } +} diff --git a/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetReadWriteTest.java b/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetReadWriteTest.java index d56edea5959b..4457db5e81f3 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetReadWriteTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetReadWriteTest.java @@ -47,8 +47,8 @@ import org.apache.paimon.types.TimestampType; import org.apache.paimon.types.TinyIntType; import org.apache.paimon.types.VarCharType; -import org.apache.paimon.utils.InstantiationUtil; +import org.apache.parquet.filter2.compat.FilterCompat; import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; @@ -230,7 +230,8 @@ void testProjection(int rowGroupSize) throws IOException { RowType.builder() .fields(fieldTypes, new String[] {"f7", "f2", "f4"}) .build(), - 500); + 500, + FilterCompat.NOOP); AtomicInteger cnt = new AtomicInteger(0); RecordReader reader = @@ -273,7 +274,8 @@ void testProjectionReadUnknownField(int rowGroupSize) throws IOException { RowType.builder() .fields(fieldTypes, new String[] {"f7", "f2", "f4", "f99"}) .build(), - 500); + 500, + FilterCompat.NOOP); AtomicInteger cnt = new AtomicInteger(0); RecordReader reader = @@ -311,7 +313,8 @@ void testReadRowPosition() throws IOException { new ParquetReaderFactory( new Options(), RowType.builder().fields(fieldTypes, new String[] {"f7"}).build(), - batchSize); + batchSize, + FilterCompat.NOOP); AtomicInteger cnt = new AtomicInteger(0); try (RecordReader reader = @@ -360,14 +363,8 @@ private Path createTempParquetFile(File folder, List rows, int rowG } private int testReadingFile(List expected, Path path) throws IOException { - ParquetReaderFactory format = new ParquetReaderFactory(new Options(), ROW_TYPE, 500); - - // validate java serialization - try { - InstantiationUtil.clone(format); - } catch (ClassNotFoundException e) { - throw new IOException(e); - } + ParquetReaderFactory format = + new ParquetReaderFactory(new Options(), ROW_TYPE, 500, FilterCompat.NOOP); RecordReader reader = format.createReader( diff --git a/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/catalyst/Compatibility.scala b/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/catalyst/Compatibility.scala index a86c9186bb77..b552bcd1e77f 100644 --- a/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/catalyst/Compatibility.scala +++ b/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/catalyst/Compatibility.scala @@ -18,9 +18,13 @@ package org.apache.paimon.spark.catalyst -import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, Expression} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, V2WriteCommand} +import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.DataType object Compatibility { @@ -31,4 +35,19 @@ object Compatibility { DataSourceV2ScanRelation(relation, scan, output) } + def withNewQuery(o: V2WriteCommand, query: LogicalPlan): V2WriteCommand = { + o.withNewQuery(query) + } + + def castByTableInsertionTag: TreeNodeTag[Unit] = { + TreeNodeTag[Unit]("by_table_insertion") + } + + def cast( + child: Expression, + dataType: DataType, + timeZoneId: Option[String] = None, + ansiEnabled: Boolean = SQLConf.get.ansiEnabled): Cast = { + Cast(child, dataType, timeZoneId, ansiEnabled) + } } diff --git a/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/paimon/spark/catalyst/Compatibility.scala b/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/paimon/spark/catalyst/Compatibility.scala index a86c9186bb77..b552bcd1e77f 100644 --- a/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/paimon/spark/catalyst/Compatibility.scala +++ b/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/paimon/spark/catalyst/Compatibility.scala @@ -18,9 +18,13 @@ package org.apache.paimon.spark.catalyst -import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, Expression} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, V2WriteCommand} +import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.DataType object Compatibility { @@ -31,4 +35,19 @@ object Compatibility { DataSourceV2ScanRelation(relation, scan, output) } + def withNewQuery(o: V2WriteCommand, query: LogicalPlan): V2WriteCommand = { + o.withNewQuery(query) + } + + def castByTableInsertionTag: TreeNodeTag[Unit] = { + TreeNodeTag[Unit]("by_table_insertion") + } + + def cast( + child: Expression, + dataType: DataType, + timeZoneId: Option[String] = None, + ansiEnabled: Boolean = SQLConf.get.ansiEnabled): Cast = { + Cast(child, dataType, timeZoneId, ansiEnabled) + } } 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 a61642beaa91..98a920b32b96 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 @@ -19,7 +19,9 @@ package org.apache.paimon.spark; import org.apache.paimon.spark.procedure.CompactProcedure; +import org.apache.paimon.spark.procedure.CreateBranchProcedure; import org.apache.paimon.spark.procedure.CreateTagProcedure; +import org.apache.paimon.spark.procedure.DeleteBranchProcedure; import org.apache.paimon.spark.procedure.DeleteTagProcedure; import org.apache.paimon.spark.procedure.ExpireSnapshotsProcedure; import org.apache.paimon.spark.procedure.MigrateFileProcedure; @@ -54,6 +56,8 @@ private static Map> initProcedureBuilders() { procedureBuilders.put("rollback", RollbackProcedure::builder); procedureBuilders.put("create_tag", CreateTagProcedure::builder); procedureBuilders.put("delete_tag", DeleteTagProcedure::builder); + procedureBuilders.put("create_branch", CreateBranchProcedure::builder); + procedureBuilders.put("delete_branch", DeleteBranchProcedure::builder); procedureBuilders.put("compact", CompactProcedure::builder); procedureBuilders.put("migrate_table", MigrateTableProcedure::builder); procedureBuilders.put("migrate_file", MigrateFileProcedure::builder); diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkTypeUtils.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkTypeUtils.java index 850183cdb543..803cf5e54bc7 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkTypeUtils.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkTypeUtils.java @@ -51,6 +51,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; +import java.util.concurrent.atomic.AtomicInteger; /** Utils for spark {@link DataType}. */ public class SparkTypeUtils { @@ -186,7 +187,8 @@ public DataType visit(RowType rowType) { List fields = new ArrayList<>(rowType.getFieldCount()); for (DataField field : rowType.getFields()) { StructField structField = - DataTypes.createStructField(field.name(), field.type().accept(this), true); + DataTypes.createStructField( + field.name(), field.type().accept(this), field.type().isNullable()); structField = Optional.ofNullable(field.description()) .map(structField::withComment) @@ -205,34 +207,42 @@ protected DataType defaultMethod(org.apache.paimon.types.DataType dataType) { private static class SparkToPaimonTypeVisitor { static org.apache.paimon.types.DataType visit(DataType type) { - return visit(type, new SparkToPaimonTypeVisitor()); + AtomicInteger atomicInteger = new AtomicInteger(-1); + return visit(type, new SparkToPaimonTypeVisitor(), atomicInteger); } static org.apache.paimon.types.DataType visit( - DataType type, SparkToPaimonTypeVisitor visitor) { + DataType type, SparkToPaimonTypeVisitor visitor, AtomicInteger atomicInteger) { if (type instanceof StructType) { StructField[] fields = ((StructType) type).fields(); List fieldResults = new ArrayList<>(fields.length); for (StructField field : fields) { - fieldResults.add(visit(field.dataType(), visitor)); + fieldResults.add(visit(field.dataType(), visitor, atomicInteger)); } - return visitor.struct((StructType) type, fieldResults); + return visitor.struct((StructType) type, fieldResults, atomicInteger); } else if (type instanceof org.apache.spark.sql.types.MapType) { return visitor.map( (org.apache.spark.sql.types.MapType) type, - visit(((org.apache.spark.sql.types.MapType) type).keyType(), visitor), - visit(((org.apache.spark.sql.types.MapType) type).valueType(), visitor)); + visit( + ((org.apache.spark.sql.types.MapType) type).keyType(), + visitor, + atomicInteger), + visit( + ((org.apache.spark.sql.types.MapType) type).valueType(), + visitor, + atomicInteger)); } else if (type instanceof org.apache.spark.sql.types.ArrayType) { return visitor.array( (org.apache.spark.sql.types.ArrayType) type, visit( ((org.apache.spark.sql.types.ArrayType) type).elementType(), - visitor)); + visitor, + atomicInteger)); } else if (type instanceof UserDefinedType) { throw new UnsupportedOperationException("User-defined types are not supported"); @@ -243,15 +253,19 @@ static org.apache.paimon.types.DataType visit( } public org.apache.paimon.types.DataType struct( - StructType struct, List fieldResults) { + StructType struct, + List fieldResults, + AtomicInteger atomicInteger) { StructField[] fields = struct.fields(); List newFields = new ArrayList<>(fields.length); - for (int i = 0; i < fields.length; i += 1) { + for (int i = 0; i < fields.length; i++) { StructField field = fields[i]; org.apache.paimon.types.DataType fieldType = fieldResults.get(i).copy(field.nullable()); String comment = field.getComment().getOrElse(() -> null); - newFields.add(new DataField(i, field.name(), fieldType, comment)); + newFields.add( + new DataField( + atomicInteger.incrementAndGet(), field.name(), fieldType, comment)); } return new RowType(newFields); diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CreateBranchProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CreateBranchProcedure.java new file mode 100644 index 000000000000..718c5abe9f78 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CreateBranchProcedure.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.procedure; + +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.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import static org.apache.spark.sql.types.DataTypes.StringType; + +/** Spark procedure to create a branch. */ +public class CreateBranchProcedure extends BaseProcedure { + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + ProcedureParameter.required("table", StringType), + ProcedureParameter.required("branch", StringType), + ProcedureParameter.required("tag", StringType) + }; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("result", DataTypes.BooleanType, true, Metadata.empty()) + }); + + protected CreateBranchProcedure(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()); + String branch = args.getString(1); + String tag = args.getString(2); + + return modifyPaimonTable( + tableIdent, + table -> { + table.createBranch(branch, tag); + InternalRow outputRow = newInternalRow(true); + return new InternalRow[] {outputRow}; + }); + } + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + public CreateBranchProcedure doBuild() { + return new CreateBranchProcedure(tableCatalog()); + } + }; + } + + @Override + public String description() { + return "CreateBranchProcedure"; + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/DeleteBranchProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/DeleteBranchProcedure.java new file mode 100644 index 000000000000..98d2a6ada77d --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/DeleteBranchProcedure.java @@ -0,0 +1,87 @@ +/* + * 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.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.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import static org.apache.spark.sql.types.DataTypes.StringType; + +/** Spark procedure to delete a branch. */ +public class DeleteBranchProcedure extends BaseProcedure { + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + ProcedureParameter.required("table", StringType), + ProcedureParameter.required("branch", StringType) + }; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("result", DataTypes.BooleanType, true, Metadata.empty()) + }); + + protected DeleteBranchProcedure(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()); + String branch = args.getString(1); + + return modifyPaimonTable( + tableIdent, + table -> { + table.deleteBranch(branch); + InternalRow outputRow = newInternalRow(true); + return new InternalRow[] {outputRow}; + }); + } + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + public DeleteBranchProcedure doBuild() { + return new DeleteBranchProcedure(tableCatalog()); + } + }; + } + + @Override + public String description() { + return "DeleteBranchProcedure"; + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonBaseScan.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonBaseScan.scala index f6df901a4379..73a3e68f03c6 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonBaseScan.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonBaseScan.scala @@ -53,8 +53,8 @@ abstract class PaimonBaseScan( private lazy val tableSchema = SparkTypeUtils.fromPaimonRowType(tableRowType) private val (tableFields, metadataFields) = { - val requiredFieldNames = requiredSchema.fieldNames - val _tableFields = tableSchema.filter(field => requiredFieldNames.contains(field.name)) + val nameToField = tableSchema.map(field => (field.name, field)).toMap + val _tableFields = requiredSchema.flatMap(field => nameToField.get(field.name)) val _metadataFields = requiredSchema .filterNot(field => tableSchema.fieldNames.contains(field.name)) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTable.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTable.scala index 29d0cfdb936f..274619323e49 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTable.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTable.scala @@ -70,6 +70,7 @@ case class SparkTable(table: Table) override def capabilities: JSet[TableCapability] = { JEnumSet.of( + TableCapability.ACCEPT_ANY_SCHEMA, TableCapability.BATCH_READ, TableCapability.V1_BATCH_WRITE, TableCapability.OVERWRITE_BY_FILTER, diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/Compatibility.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/Compatibility.scala index a86c9186bb77..751b88f585b9 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/Compatibility.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/Compatibility.scala @@ -18,9 +18,13 @@ package org.apache.paimon.spark.catalyst -import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, EvalMode, Expression} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, V2WriteCommand} +import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.DataType object Compatibility { @@ -31,4 +35,19 @@ object Compatibility { DataSourceV2ScanRelation(relation, scan, output) } + def withNewQuery(o: V2WriteCommand, query: LogicalPlan): V2WriteCommand = { + o.withNewQuery(query) + } + + def castByTableInsertionTag: TreeNodeTag[Unit] = { + Cast.BY_TABLE_INSERTION + } + + def cast( + child: Expression, + dataType: DataType, + timeZoneId: Option[String] = None, + ansiEnabled: Boolean = SQLConf.get.ansiEnabled): Cast = { + Cast(child, dataType, timeZoneId, ansiEnabled) + } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonAnalysis.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonAnalysis.scala index 65527e499d5a..67685612664d 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonAnalysis.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonAnalysis.scala @@ -19,21 +19,37 @@ package org.apache.paimon.spark.catalyst.analysis import org.apache.paimon.spark.SparkTable +import org.apache.paimon.spark.catalyst.Compatibility import org.apache.paimon.spark.catalyst.analysis.PaimonRelation.isPaimonTable import org.apache.paimon.spark.commands.{PaimonAnalyzeTableColumnCommand, PaimonDynamicPartitionOverwriteCommand, PaimonTruncateTableCommand} import org.apache.paimon.table.FileStoreTable import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.ResolvedTable +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, Expression, NamedExpression} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType} + +import scala.collection.JavaConverters._ class PaimonAnalysis(session: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsDown { + case a @ PaimonV2WriteCommand(table, paimonTable) + if !schemaCompatible( + a.query.output.toStructType, + table.output.toStructType, + paimonTable.partitionKeys().asScala) => + val newQuery = resolveQueryColumns(a.query, table.output) + if (newQuery != a.query) { + Compatibility.withNewQuery(a, newQuery) + } else { + a + } + case o @ PaimonDynamicPartitionOverwrite(r, d) if o.resolved => PaimonDynamicPartitionOverwriteCommand(r, d, o.query, o.writeOptions, o.isByName) @@ -41,6 +57,75 @@ class PaimonAnalysis(session: SparkSession) extends Rule[LogicalPlan] { PaimonMergeIntoResolver(merge, session) } + private def schemaCompatible( + tableSchema: StructType, + dataSchema: StructType, + partitionCols: Seq[String], + parent: Array[String] = Array.empty): Boolean = { + + if (tableSchema.size != dataSchema.size) { + throw new RuntimeException("the number of data columns don't match with the table schema's.") + } + + def dataTypeCompatible(column: String, dt1: DataType, dt2: DataType): Boolean = { + (dt1, dt2) match { + case (s1: StructType, s2: StructType) => + schemaCompatible(s1, s2, partitionCols, Array(column)) + case (a1: ArrayType, a2: ArrayType) => + dataTypeCompatible(column, a1.elementType, a2.elementType) + case (m1: MapType, m2: MapType) => + dataTypeCompatible(column, m1.keyType, m2.keyType) && dataTypeCompatible( + column, + m1.valueType, + m2.valueType) + case (d1, d2) => d1 == d2 + } + } + + tableSchema.zip(dataSchema).forall { + case (f1, f2) => + checkNullability(f1, f2, partitionCols, parent) + f1.name == f2.name && dataTypeCompatible(f1.name, f1.dataType, f2.dataType) + } + } + + private def resolveQueryColumns( + query: LogicalPlan, + tableAttributes: Seq[Attribute]): LogicalPlan = { + val project = query.output.zipWithIndex.map { + case (attr, i) => + val targetAttr = tableAttributes(i) + addCastToColumn(attr, targetAttr) + } + Project(project, query) + } + + private def addCastToColumn(attr: Attribute, targetAttr: Attribute): NamedExpression = { + val expr = (attr.dataType, targetAttr.dataType) match { + case (s, t) if s == t => + attr + case _ => + cast(attr, targetAttr.dataType) + } + Alias(expr, targetAttr.name)(explicitMetadata = Option(targetAttr.metadata)) + } + + private def cast(expr: Expression, dataType: DataType): Expression = { + val cast = Compatibility.cast(expr, dataType, Option(conf.sessionLocalTimeZone)) + cast.setTagValue(Compatibility.castByTableInsertionTag, ()) + cast + } + + private def checkNullability( + input: StructField, + expected: StructField, + partitionCols: Seq[String], + parent: Array[String] = Array.empty): Unit = { + val fullColumnName = (parent ++ Array(input.name)).mkString(".") + if (!partitionCols.contains(fullColumnName) && input.nullable && !expected.nullable) { + throw new RuntimeException("Cannot write nullable values to non-null column") + } + } } case class PaimonPostHocResolutionRules(session: SparkSession) extends Rule[LogicalPlan] { @@ -78,6 +163,20 @@ case class PaimonPostHocResolutionRules(session: SparkSession) extends Rule[Logi } } +object PaimonV2WriteCommand { + def unapply(o: V2WriteCommand): Option[(DataSourceV2Relation, FileStoreTable)] = { + if (o.query.resolved) { + o.table match { + case r: DataSourceV2Relation if r.table.isInstanceOf[SparkTable] => + Some((r, r.table.asInstanceOf[SparkTable].getTable.asInstanceOf[FileStoreTable])) + case _ => None + } + } else { + None + } + } +} + object PaimonDynamicPartitionOverwrite { def unapply(o: OverwritePartitionsDynamic): Option[(DataSourceV2Relation, FileStoreTable)] = { if (o.query.resolved) { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala index 32b32875032a..bb66e38266dc 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala @@ -20,7 +20,7 @@ package org.apache.paimon.spark.catalyst.analysis import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, LogicalPlan, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction} +import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, LogicalPlan, MergeAction, MergeIntoTable, Project, UpdateAction, UpdateStarAction} /** Resolve all the expressions for MergeInto. */ object PaimonMergeIntoResolver extends PaimonMergeIntoResolverBase { @@ -30,6 +30,8 @@ object PaimonMergeIntoResolver extends PaimonMergeIntoResolverBase { target: LogicalPlan, source: LogicalPlan, resolve: (Expression, LogicalPlan) => Expression): Seq[MergeAction] = { + val fakeSource = Project(source.output, source) + def resolveMergeAction(action: MergeAction): MergeAction = { action match { case DeleteAction(condition) => @@ -47,7 +49,8 @@ object PaimonMergeIntoResolver extends PaimonMergeIntoResolverBase { case UpdateStarAction(condition) => val resolvedCond = condition.map(resolve(_, target)) val resolvedAssignments = target.output.map { - attr => Assignment(attr, resolve(UnresolvedAttribute.quotedString(attr.name), source)) + attr => + Assignment(attr, resolve(UnresolvedAttribute.quotedString(attr.name), fakeSource)) } UpdateAction(resolvedCond, resolvedAssignments) } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolverBase.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolverBase.scala index c36ee4485a81..cbd6b52c01af 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolverBase.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolverBase.scala @@ -23,7 +23,7 @@ import org.apache.paimon.spark.catalyst.analysis.expressions.ExpressionHelper import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, LogicalPlan, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction} +import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, LogicalPlan, MergeAction, MergeIntoTable, Project, UpdateAction, UpdateStarAction} trait PaimonMergeIntoResolverBase extends ExpressionHelper { @@ -36,6 +36,7 @@ trait PaimonMergeIntoResolverBase extends ExpressionHelper { val condition = merge.mergeCondition val matched = merge.matchedActions val notMatched = merge.notMatchedActions + val fakeSource = Project(source.output, source) val resolve: (Expression, LogicalPlan) => Expression = resolveExpression(spark) @@ -49,7 +50,7 @@ trait PaimonMergeIntoResolverBase extends ExpressionHelper { val resolvedAssignments = assignments.map { assignment => assignment.copy( - key = resolve(assignment.key, merge), + key = resolve(assignment.key, target), value = resolve(assignment.value, merge)) } UpdateAction(resolvedCond, resolvedAssignments) @@ -60,18 +61,19 @@ trait PaimonMergeIntoResolverBase extends ExpressionHelper { } UpdateAction(resolvedCond, resolvedAssignments) case InsertAction(condition, assignments) => - val resolvedCond = condition.map(resolve(_, source)) + val resolvedCond = condition.map(resolve(_, fakeSource)) val resolvedAssignments = assignments.map { assignment => assignment.copy( - key = resolve(assignment.key, source), - value = resolve(assignment.value, source)) + key = resolve(assignment.key, fakeSource), + value = resolve(assignment.value, fakeSource)) } InsertAction(resolvedCond, resolvedAssignments) case InsertStarAction(condition) => - val resolvedCond = condition.map(resolve(_, source)) + val resolvedCond = condition.map(resolve(_, fakeSource)) val resolvedAssignments = target.output.map { - attr => Assignment(attr, resolve(UnresolvedAttribute.quotedString(attr.name), source)) + attr => + Assignment(attr, resolve(UnresolvedAttribute.quotedString(attr.name), fakeSource)) } InsertAction(resolvedCond, resolvedAssignments) case _ => diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonProcedureResolver.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonProcedureResolver.scala index e0248f22e988..552353192349 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonProcedureResolver.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonProcedureResolver.scala @@ -19,6 +19,7 @@ package org.apache.paimon.spark.catalyst.analysis import org.apache.paimon.spark.catalog.ProcedureCatalog +import org.apache.paimon.spark.catalyst.Compatibility import org.apache.paimon.spark.catalyst.plans.logical.{PaimonCallArgument, PaimonCallCommand, PaimonCallStatement, PaimonNamedArgument, PaimonPositionalArgument} import org.apache.paimon.spark.procedure.ProcedureParameter @@ -71,7 +72,7 @@ case class PaimonProcedureResolver(sparkSession: SparkSession) s"Cannot cast $argumentType to $parameterType of ${parameter.name}.") } if (parameterType != argumentType) { - Cast(argument, parameterType) + Compatibility.cast(argument, parameterType) } else { argument } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala index 333508125dbc..c008819fb0cc 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala @@ -20,6 +20,7 @@ package org.apache.paimon.spark.catalyst.analysis.expressions import org.apache.paimon.predicate.{Predicate, PredicateBuilder} import org.apache.paimon.spark.SparkFilterConverter +import org.apache.paimon.spark.catalyst.Compatibility import org.apache.paimon.types.RowType import org.apache.spark.sql.PaimonUtils.{normalizeExprs, translateFilter} @@ -43,6 +44,12 @@ trait ExpressionHelper extends PredicateHelper { val newPlan = FakeLogicalPlan(Seq(expr), plan.children) spark.sessionState.analyzer.execute(newPlan) match { case FakeLogicalPlan(resolvedExpr, _) => + resolvedExpr.foreach { + expr => + if (!expr.resolved) { + throw new RuntimeException(s"cannot resolve ${expr.sql} from $plan") + } + } resolvedExpr.head case _ => throw new RuntimeException(s"Could not resolve expression $expr in plan: $plan") @@ -85,7 +92,7 @@ trait ExpressionHelper extends PredicateHelper { if (DataType.equalsIgnoreCaseAndNullability(fromDataType, toDataType)) { fromExpression } else { - Cast(fromExpression, toDataType, Option(SQLConf.get.sessionLocalTimeZone)) + Compatibility.cast(fromExpression, toDataType, Option(SQLConf.get.sessionLocalTimeZone)) } } } 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 f729c290c4c1..8d89af2a14de 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 @@ -27,6 +27,7 @@ import org.apache.paimon.spark.{PaimonSplitScan, SparkFilterConverter} import org.apache.paimon.spark.catalyst.Compatibility import org.apache.paimon.spark.catalyst.analysis.expressions.ExpressionHelper import org.apache.paimon.spark.commands.SparkDataFileMeta.convertToSparkDataFileMeta +import org.apache.paimon.spark.schema.PaimonMetadataColumn import org.apache.paimon.spark.schema.PaimonMetadataColumn._ import org.apache.paimon.table.sink.{CommitMessage, CommitMessageImpl} import org.apache.paimon.table.source.DataSplit @@ -37,7 +38,7 @@ import org.apache.spark.sql.{Dataset, SparkSession} import org.apache.spark.sql.PaimonUtils.createDataset import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral -import org.apache.spark.sql.catalyst.plans.logical.{Filter => FilterLogicalNode, Project} +import org.apache.spark.sql.catalyst.plans.logical.{Filter => FilterLogicalNode, LogicalPlan, Project} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.sources.{AlwaysTrue, And, EqualNullSafe, Filter} @@ -125,14 +126,7 @@ trait PaimonCommand extends WithFileStoreTable with ExpressionHelper { } val metadataCols = Seq(FILE_PATH) - val metadataProj = metadataCols.map(_.toAttribute) - val newRelation = relation.copy(output = relation.output ++ metadataProj) - val scan = PaimonSplitScan(table, candidateDataSplits.toArray, metadataCols) - val filteredRelation = Project( - metadataProj, - FilterLogicalNode( - condition, - Compatibility.createDataSourceV2ScanRelation(newRelation, scan, newRelation.output))) + val filteredRelation = createNewScanPlan(candidateDataSplits, condition, relation, metadataCols) createDataset(sparkSession, filteredRelation) .select(FILE_PATH_COLUMN) .distinct() @@ -198,18 +192,7 @@ trait PaimonCommand extends WithFileStoreTable with ExpressionHelper { val dataFileAndDeletionFile = dataFilePathToMeta.mapValues(_.toSparkDeletionFile).toArray val metadataCols = Seq(FILE_PATH, ROW_INDEX) - val metadataProj = metadataCols.map(_.toAttribute) - val scan = PaimonSplitScan(table, candidateDataSplits.toArray, metadataCols) - val filteredRelation = { - Project( - metadataProj, - FilterLogicalNode( - condition, - Compatibility.createDataSourceV2ScanRelation( - relation, - scan, - relation.output ++ metadataProj))) - } + val filteredRelation = createNewScanPlan(candidateDataSplits, condition, relation, metadataCols) val store = table.store() val fileIO = table.fileIO() @@ -248,6 +231,22 @@ trait PaimonCommand extends WithFileStoreTable with ExpressionHelper { } } + private def createNewScanPlan( + candidateDataSplits: Seq[DataSplit], + condition: Expression, + relation: DataSourceV2Relation, + metadataCols: Seq[PaimonMetadataColumn]): LogicalPlan = { + val metadataProj = metadataCols.map(_.toAttribute) + val newRelation = relation.copy(output = relation.output ++ metadataProj) + val scan = PaimonSplitScan(table, candidateDataSplits.toArray, metadataCols) + Project( + metadataProj, + FilterLogicalNode( + condition, + Compatibility.createDataSourceV2ScanRelation(newRelation, scan, newRelation.output))) + + } + protected def buildDeletedCommitMessage( deletedFiles: Array[SparkDataFileMeta]): Seq[CommitMessage] = { deletedFiles diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonSparkWriter.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonSparkWriter.scala index cc6a33cf2ee5..7be4dd80a817 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonSparkWriter.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonSparkWriter.scala @@ -238,7 +238,7 @@ case class PaimonSparkWriter(table: FileStoreTable) { sdv.bucket, DataIncrement.emptyIncrement(), CompactIncrement.emptyIncrement(), - new IndexIncrement(maintainer.prepareCommit())) + new IndexIncrement(maintainer.writeDeletionVectorsIndex())) serializer.serialize(commitMessage) } diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadITCase.java b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadITCase.java index 2dfea46c11e7..e398f13dd12b 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadITCase.java +++ b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadITCase.java @@ -249,7 +249,8 @@ public void testCreateTableAs() { .isEqualTo( String.format( "[[%sTBLPROPERTIES (\n 'path' = '%s',\n 'primary-key' = 'a')\n]]", - showCreateString("t_pk_as", "a BIGINT", "b STRING", "c STRING"), + showCreateString( + "t_pk_as", "a BIGINT NOT NULL", "b STRING", "c STRING"), new Path(warehousePath, "default.db/t_pk_as"))); List resultPk = spark.sql("SELECT * FROM t_pk_as").collectAsList(); @@ -283,8 +284,8 @@ public void testCreateTableAs() { "user_id BIGINT", "item_id BIGINT", "behavior STRING", - "dt STRING", - "hh STRING"), + "dt STRING NOT NULL", + "hh STRING NOT NULL"), new Path(warehousePath, "default.db/t_all_as"))); List resultAll = spark.sql("SELECT * FROM t_all_as").collectAsList(); assertThat(resultAll.stream().map(Row::toString)) @@ -372,7 +373,10 @@ public void testShowCreateTable() { + " 'k1' = 'v1',\n" + " 'path' = '%s',\n" + " 'primary-key' = 'a,b')\n]]", - showCreateString("tbl", "a INT COMMENT 'a comment'", "b STRING"), + showCreateString( + "tbl", + "a INT NOT NULL COMMENT 'a comment'", + "b STRING NOT NULL"), new Path(warehousePath, "default.db/tbl"))); } diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadTestBase.java b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadTestBase.java index becefb505578..ec82d8511d61 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadTestBase.java +++ b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadTestBase.java @@ -24,6 +24,7 @@ import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.FileStoreTableFactory; import org.apache.paimon.table.sink.CommitMessage; @@ -67,9 +68,15 @@ public abstract class SparkReadTestBase { @BeforeAll public static void startMetastoreAndSpark(@TempDir java.nio.file.Path tempDir) { warehousePath = new Path("file:" + tempDir.toString()); - spark = SparkSession.builder().master("local[2]").getOrCreate(); - spark.conf().set("spark.sql.catalog.paimon", SparkCatalog.class.getName()); - spark.conf().set("spark.sql.catalog.paimon.warehouse", warehousePath.toString()); + spark = + SparkSession.builder() + .master("local[2]") + .config("spark.sql.catalog.paimon", SparkCatalog.class.getName()) + .config("spark.sql.catalog.paimon.warehouse", warehousePath.toString()) + .config( + "spark.sql.extensions", + PaimonSparkSessionExtensions.class.getName()) + .getOrCreate(); spark.sql("USE paimon"); } @@ -230,6 +237,6 @@ protected String showCreateString(String table, String... fieldSpec) { // default schema protected String defaultShowCreateString(String table) { - return showCreateString(table, "a INT", "b BIGINT", "c STRING"); + return showCreateString(table, "a INT NOT NULL", "b BIGINT", "c STRING"); } } diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java index 9fa5d21ed256..136da9572b65 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java +++ b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java @@ -81,7 +81,11 @@ public void testAddColumn() { assertThat(afterAdd.toString()) .contains( showCreateString( - "testAddColumn", "a INT", "b BIGINT", "c STRING", "d STRING")); + "testAddColumn", + "a INT NOT NULL", + "b BIGINT", + "c STRING", + "d STRING")); assertThat(spark.table("testAddColumn").collectAsList().toString()) .isEqualTo("[[1,2,1,null], [5,6,3,null]]"); @@ -115,7 +119,7 @@ public void testAddColumnPosition() { showCreateString( "testAddColumnPositionFirst", "d INT", - "a INT", + "a INT NOT NULL", "b BIGINT", "c STRING")); @@ -126,7 +130,7 @@ public void testAddColumnPosition() { .contains( showCreateString( "testAddColumnPositionAfter", - "a INT", + "a INT NOT NULL", "b BIGINT", "d INT", "c STRING")); @@ -170,7 +174,9 @@ public void testRenameColumn() { spark.sql("ALTER TABLE testRenameColumn RENAME COLUMN b to bb"); List afterRename = spark.sql("SHOW CREATE TABLE testRenameColumn").collectAsList(); assertThat(afterRename.toString()) - .contains(showCreateString("testRenameColumn", "a INT", "bb BIGINT", "c STRING")); + .contains( + showCreateString( + "testRenameColumn", "a INT NOT NULL", "bb BIGINT", "c STRING")); Dataset table = spark.table("testRenameColumn"); results = table.select("bb", "c").collectAsList(); assertThat(results.toString()).isEqualTo("[[2,1], [6,3]]"); @@ -212,7 +218,7 @@ public void testDropSingleColumn() { List afterDrop = spark.sql("SHOW CREATE TABLE testDropSingleColumn").collectAsList(); assertThat(afterDrop.toString()) - .contains(showCreateString("testDropSingleColumn", "a INT", "c STRING")); + .contains(showCreateString("testDropSingleColumn", "a INT NOT NULL", "c STRING")); List results = spark.table("testDropSingleColumn").collectAsList(); assertThat(results.toString()).isEqualTo("[[1,1], [5,3]]"); @@ -228,7 +234,8 @@ public void testDropColumns() { spark.sql("ALTER TABLE testDropColumns DROP COLUMNS b,c"); List afterDrop = spark.sql("SHOW CREATE TABLE testDropColumns").collectAsList(); - assertThat(afterDrop.toString()).contains(showCreateString("testDropColumns", "a INT")); + assertThat(afterDrop.toString()) + .contains(showCreateString("testDropColumns", "a INT NOT NULL")); } @Test @@ -261,7 +268,9 @@ public void testDropPrimaryKey() { List beforeDrop = spark.sql("SHOW CREATE TABLE testDropPrimaryKey").collectAsList(); assertThat(beforeDrop.toString()) - .contains(showCreateString("testDropPrimaryKey", "a BIGINT", "b STRING")); + .contains( + showCreateString( + "testDropPrimaryKey", "a BIGINT NOT NULL", "b STRING NOT NULL")); assertThatThrownBy(() -> spark.sql("ALTER TABLE testDropPrimaryKey DROP COLUMN b")) .satisfies( @@ -277,14 +286,14 @@ public void testUpdateColumnPosition() { spark.sql("ALTER TABLE tableFirst ALTER COLUMN b FIRST"); List result = spark.sql("SHOW CREATE TABLE tableFirst").collectAsList(); assertThat(result.toString()) - .contains(showCreateString("tableFirst", "b BIGINT", "a INT", "c STRING")); + .contains(showCreateString("tableFirst", "b BIGINT", "a INT NOT NULL", "c STRING")); // move after createTable("tableAfter"); spark.sql("ALTER TABLE tableAfter ALTER COLUMN c AFTER a"); result = spark.sql("SHOW CREATE TABLE tableAfter").collectAsList(); assertThat(result.toString()) - .contains(showCreateString("tableAfter", "a INT", "c STRING", "b BIGINT")); + .contains(showCreateString("tableAfter", "a INT NOT NULL", "c STRING", "b BIGINT")); spark.sql("CREATE TABLE tableAfter1 (a INT, b BIGINT, c STRING, d DOUBLE)"); spark.sql("ALTER TABLE tableAfter1 ALTER COLUMN b AFTER c"); @@ -334,7 +343,9 @@ public void testAlterColumnType() { List afterAlter = spark.sql("SHOW CREATE TABLE testAlterColumnType").collectAsList(); assertThat(afterAlter.toString()) - .contains(showCreateString("testAlterColumnType", "a INT", "b DOUBLE", "c STRING")); + .contains( + showCreateString( + "testAlterColumnType", "a INT NOT NULL", "b DOUBLE", "c STRING")); } @Test diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTypeTest.java b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTypeTest.java index c9f82a178a56..84f509d768b6 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTypeTest.java +++ b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTypeTest.java @@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicInteger; import static org.apache.paimon.spark.SparkTypeUtils.fromPaimonRowType; +import static org.apache.paimon.spark.SparkTypeUtils.toPaimonType; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link SparkTypeUtils}. */ @@ -76,14 +77,14 @@ public void testAllTypes() { String nestedRowMapType = "StructField(locations,MapType(" + "StringType," - + "StructType(StructField(posX,DoubleType,true),StructField(posY,DoubleType,true)),true),true)"; + + "StructType(StructField(posX,DoubleType,false),StructField(posY,DoubleType,false)),true),true)"; String expected = "StructType(" - + "StructField(id,IntegerType,true)," + + "StructField(id,IntegerType,false)," + "StructField(name,StringType,true)," + "StructField(char,CharType(10),true)," + "StructField(varchar,VarcharType(10),true)," - + "StructField(salary,DoubleType,true)," + + "StructField(salary,DoubleType,false)," + nestedRowMapType + "," + "StructField(strArray,ArrayType(StringType,true),true)," @@ -102,7 +103,6 @@ public void testAllTypes() { StructType sparkType = fromPaimonRowType(ALL_TYPES); assertThat(sparkType.toString().replace(", ", ",")).isEqualTo(expected); - // Ignore the assertion below, since we force to make all the fields nullable. - // assertThat(toPaimonType(sparkType)).isEqualTo(ALL_TYPES); + assertThat(toPaimonType(sparkType)).isEqualTo(ALL_TYPES); } } diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java index b9408574ed65..6d42f614fc4c 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java +++ b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java @@ -224,18 +224,4 @@ public void testReadWriteUnawareBucketTable() { rows = spark.sql("SELECT max(bucket) FROM `T$FILES`").collectAsList(); assertThat(rows.toString()).isEqualTo("[[0]]"); } - - @Test - public void testNonnull() { - try { - spark.sql("CREATE TABLE S AS SELECT 1 as a, 2 as b, 'yann' as c"); - - spark.sql("CREATE TABLE T (a INT NOT NULL, b INT, c STRING)"); - spark.sql("INSERT INTO T SELECT * FROM S"); - List rows = spark.sql("SELECT * FROM T").collectAsList(); - assertThat(rows.toString()).isEqualTo("[[1,2,yann]]"); - } finally { - spark.sql("DROP TABLE IF EXISTS S"); - } - } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteBranchProcedureTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteBranchProcedureTest.scala new file mode 100644 index 000000000000..459ebf4abaf3 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteBranchProcedureTest.scala @@ -0,0 +1,101 @@ +/* + * 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.{Dataset, Row} +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.streaming.StreamTest + +class CreateAndDeleteBranchProcedureTest extends PaimonSparkTestBase with StreamTest { + + import testImplicits._ + test("Paimon Procedure: create and delete branch") { + failAfter(streamingTimeout) { + withTempDir { + checkpointDir => + // define a change-log table and test `forEachBatch` api + spark.sql(s""" + |CREATE TABLE T (a INT, b STRING) + |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') + |""".stripMargin) + val location = loadTable("T").location().toString + + val inputData = MemoryStream[(Int, String)] + val stream = inputData + .toDS() + .toDF("a", "b") + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .foreachBatch { + (batch: Dataset[Row], _: Long) => + batch.write.format("paimon").mode("append").save(location) + } + .start() + + val query = () => spark.sql("SELECT * FROM T ORDER BY a") + + try { + // snapshot-1 + inputData.addData((1, "a")) + stream.processAllAvailable() + checkAnswer(query(), Row(1, "a") :: Nil) + + // snapshot-2 + inputData.addData((2, "b")) + stream.processAllAvailable() + checkAnswer(query(), Row(1, "a") :: Row(2, "b") :: Nil) + + // snapshot-3 + inputData.addData((2, "b2")) + stream.processAllAvailable() + checkAnswer(query(), Row(1, "a") :: Row(2, "b2") :: Nil) + + // create tag + checkAnswer( + spark.sql( + "CALL paimon.sys.create_tag(table => 'test.T', tag => 'test_tag', snapshot => 2)"), + Row(true) :: Nil) + checkAnswer( + spark.sql("SELECT tag_name FROM paimon.test.`T$tags`"), + Row("test_tag") :: Nil) + + // create branch + checkAnswer( + spark.sql( + "CALL paimon.sys.create_branch(table => 'test.T', branch => 'test_branch', tag => 'test_tag')"), + Row(true) :: Nil) + val table = loadTable("T") + val branchManager = table.branchManager() + assert(branchManager.branchExists("test_branch")) + + // delete branch + checkAnswer( + spark.sql( + "CALL paimon.sys.delete_branch(table => 'test.T', branch => 'test_branch')"), + Row(true) :: Nil) + assert(!branchManager.branchExists("test_branch")) + } finally { + stream.stop() + } + } + } + } +} diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala index f62b87a60188..80118a1b6243 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala @@ -27,6 +27,36 @@ abstract class DDLTestBase extends PaimonSparkTestBase { import testImplicits._ + test("Paimon DDL: create table with not null") { + withTable("T") { + sql(""" + |CREATE TABLE T (id INT NOT NULL, name STRING) + |""".stripMargin) + + val exception = intercept[RuntimeException] { + sql(""" + |INSERT INTO T VALUES (1, "a"), (2, "b"), (null, "c") + |""".stripMargin) + } + Assertions.assertTrue( + exception.getMessage().contains("Cannot write nullable values to non-null column")) + + sql(""" + |INSERT INTO T VALUES (1, "a"), (2, "b"), (3, null) + |""".stripMargin) + + checkAnswer( + sql("SELECT * FROM T ORDER BY id"), + Seq((1, "a"), (2, "b"), (3, null)).toDF() + ) + + val schema = spark.table("T").schema + Assertions.assertEquals(schema.size, 2) + Assertions.assertFalse(schema("id").nullable) + Assertions.assertTrue(schema("name").nullable) + } + } + test("Paimon DDL: Create Table As Select") { withTable("source", "t1", "t2") { Seq((1L, "x1", "2023"), (2L, "x2", "2023")) @@ -86,9 +116,9 @@ abstract class DDLTestBase extends PaimonSparkTestBase { } } - test("Paimon DDL: create table with char/varchar/string") { - Seq("orc", "avro").foreach( - format => { + fileFormats.foreach { + format => + test(s"Paimon DDL: create table with char/varchar/string, file.format: $format") { withTable("paimon_tbl") { spark.sql( s""" @@ -116,7 +146,7 @@ abstract class DDLTestBase extends PaimonSparkTestBase { ) // check select - if (format == "orc" && !gteqSpark3_4) { + if (!gteqSpark3_4) { // Orc reader will right trim the char type, e.g. "Friday " => "Friday" (see orc's `CharTreeReader`) // and Spark has a conf `spark.sql.readSideCharPadding` to auto padding char only since 3.4 (default true) // So when using orc with Spark3.4-, here will return "Friday" @@ -149,6 +179,6 @@ abstract class DDLTestBase extends PaimonSparkTestBase { Row("Friday") :: Nil ) } - }) + } } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTest.scala index 1cef45cf94b4..9ad1f4523884 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTest.scala @@ -23,7 +23,7 @@ import org.apache.paimon.spark.PaimonSparkTestBase import org.apache.spark.sql.Row import org.apache.spark.sql.types._ -import java.sql.Date +import java.sql.{Date, Timestamp} abstract class InsertOverwriteTableTestBase extends PaimonSparkTestBase { @@ -355,4 +355,61 @@ abstract class InsertOverwriteTableTestBase extends PaimonSparkTestBase { spark.sql("INSERT OVERWRITE T partition (dt='2024-04-18') values(1)") checkAnswer(spark.sql("SELECT * FROM T"), Row("1", Date.valueOf("2024-04-18"))) } + + test("Paimon Insert: all data types") { + spark.sql(""" + |CREATE TABLE T ( + |id bigint, name string, birth date, age int, marital boolean, + |height float, weight double, + |interests array, scores map, avg_score decimal(5, 2), + |address struct, + |create_time timestamp + |) + |""".stripMargin) + + spark.sql( + """ + |INSERT INTO T + |SELECT 1L, "yann", TO_DATE('1990-01-01', 'yyyy-MM-dd'), 32, true, 123.4F, 56.7D, + |array("abc", "def"), map("math", 90D, "history", 60D), 75.000, struct("Zhejiang", "Hangzhou", "Xihu"), + |TO_TIMESTAMP('2024-07-01 16:00:00', 'yyyy-MM-dd kk:mm:ss') + |UNION ALL + |SELECT 2L, "mai", TO_DATE('2021-06-01', 'yyyy-MM-dd'), 3, false, 98.7F, 12.3D, + |array("def", "xyz"), null, null, struct("Zhejiang", "Hangzhou", "Xihu"), + |TO_TIMESTAMP('2024-07-01 16:00:00', 'yyyy-MM-dd kk:mm:ss') + |; + |""".stripMargin) + + checkAnswer( + spark.sql("SELECT * FROM T ORDER BY id"), + Row( + 1L, + "yann", + Date.valueOf("1990-01-01"), + 32, + true, + 123.4f, + 56.7d, + Array("abc", "def"), + Map("math" -> 90d, "history" -> 60d), + BigDecimal.apply(75.00), + Row("Zhejiang", "Hangzhou", "Xihu"), + Timestamp.valueOf("2024-07-01 16:00:00") + ) :: + Row( + 2L, + "mai", + Date.valueOf("2021-06-01"), + 3, + false, + 98.7f, + 12.3d, + Array("def", "xyz"), + null, + null, + Row("Zhejiang", "Hangzhou", "Xihu"), + Timestamp.valueOf("2024-07-01 16:00:00") + ) :: Nil + ) + } } diff --git a/pom.xml b/pom.xml index 393c899f9f98..c2bf9e90cc41 100644 --- a/pom.xml +++ b/pom.xml @@ -77,7 +77,7 @@ under the License. 30.1.1-jre 2.9.3 4.1.100.Final - 4.8 + 4.9.3 2.8.5 2.12.15 2.12 @@ -101,7 +101,7 @@ under the License. **/*Test.* 1.19 - 1.19.0 + 1.19.1 3.0.1-1.18 @@ -114,7 +114,6 @@ under the License. 3.2.3 3.2.2 2.1.0 - 4.8 1.15.3 128m