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:"none": No changelog file. "input": Double write to a changelog file when flushing memory table, the changelog is from input. "full-compaction": Generate changelog files with each full compaction. "lookup": Generate changelog files through 'lookup' before committing the data writing.
+
+ 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