From bf4e8a224919f7007835d81b89e3169592cd86c1 Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Fri, 15 Mar 2024 12:01:42 +0800 Subject: [PATCH 01/79] [hotfix] Fix unstable testRandomCdcEventsUnawareBucket (#3014) --- .../cdc/FlinkCdcSyncDatabaseSinkITCase.java | 36 +++++++------ .../sink/cdc/FlinkCdcSyncTableSinkITCase.java | 27 +++++++--- .../paimon/flink/sink/cdc/TestTable.java | 50 ++++++------------- 3 files changed, 56 insertions(+), 57 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java index 167d56b4781a..d3369cef3344 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogUtils; +import org.apache.paimon.data.InternalRow; import org.apache.paimon.flink.FlinkCatalogFactory; import org.apache.paimon.flink.util.AbstractTestBase; import org.apache.paimon.fs.FileIO; @@ -28,12 +29,15 @@ import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.options.MemorySize; import org.apache.paimon.options.Options; +import org.apache.paimon.reader.RecordReaderIterator; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.SchemaUtils; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.table.source.TableScan; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.FailingFileIO; import org.apache.paimon.utils.TraceableFileIO; @@ -41,7 +45,6 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.io.TempDir; @@ -64,23 +67,20 @@ public class FlinkCdcSyncDatabaseSinkITCase extends AbstractTestBase { @TempDir java.nio.file.Path tempDir; - @Disabled @Test @Timeout(120) public void testRandomCdcEvents() throws Exception { innerTestRandomCdcEvents(() -> ThreadLocalRandom.current().nextInt(5) + 1, false); } - @Disabled @Test - @Timeout(180) + @Timeout(120) public void testRandomCdcEventsDynamicBucket() throws Exception { innerTestRandomCdcEvents(() -> -1, false); } - @Disabled @Test - @Timeout(180) + @Timeout(120) public void testRandomCdcEventsUnawareBucket() throws Exception { innerTestRandomCdcEvents(() -> -1, true); } @@ -177,17 +177,21 @@ private void innerTestRandomCdcEvents(Supplier bucket, boolean unawareB // enable failure when running jobs if needed FailingFileIO.reset(failingName, 2, 10000); - if (unawareBucketMode) { - // there's a compact operator which won't terminate - env.executeAsync(); - } else { - env.execute(); - } + env.execute(); // no failure when checking results FailingFileIO.reset(failingName, 0, 1); - for (int i = 0; i < fileStoreTables.size(); i++) { - testTables.get(i).assertResult(fileStoreTables.get(i)); + for (int i = 0; i < numTables; i++) { + FileStoreTable table = fileStoreTables.get(i).copyWithLatestSchema(); + SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); + TableSchema schema = schemaManager.latest().get(); + + ReadBuilder readBuilder = table.newReadBuilder(); + TableScan.Plan plan = readBuilder.newScan().plan(); + try (RecordReaderIterator it = + new RecordReaderIterator<>(readBuilder.newRead().createReader(plan))) { + testTables.get(i).assertResult(schema, it); + } } } @@ -204,6 +208,10 @@ private FileStoreTable createFileStoreTable( conf.set(CoreOptions.DYNAMIC_BUCKET_TARGET_ROW_NUM, 100L); conf.set(CoreOptions.WRITE_BUFFER_SIZE, new MemorySize(4096 * 3)); conf.set(CoreOptions.PAGE_SIZE, new MemorySize(4096)); + // disable compaction for unaware bucket mode to avoid instability + if (primaryKeys.isEmpty() && numBucket == -1) { + conf.set(CoreOptions.WRITE_ONLY, true); + } TableSchema tableSchema = SchemaUtils.forceCommit( diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java index a475617410b2..57a7604c00bb 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogUtils; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.data.InternalRow; import org.apache.paimon.flink.FlinkCatalogFactory; import org.apache.paimon.flink.util.AbstractTestBase; import org.apache.paimon.fs.FileIO; @@ -29,12 +30,15 @@ import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.options.MemorySize; import org.apache.paimon.options.Options; +import org.apache.paimon.reader.RecordReaderIterator; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.SchemaUtils; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.table.source.TableScan; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.FailingFileIO; import org.apache.paimon.utils.TraceableFileIO; @@ -168,18 +172,21 @@ private void innerTestRandomCdcEvents( // enable failure when running jobs if needed FailingFileIO.reset(failingName, 10, 10000); - - if (unawareBucketMode) { - // there's a compact operator which won't terminate - env.executeAsync(); - } else { - env.execute(); - } + env.execute(); // no failure when checking results FailingFileIO.reset(failingName, 0, 1); - testTable.assertResult(table); + table = table.copyWithLatestSchema(); + SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); + TableSchema schema = schemaManager.latest().get(); + + ReadBuilder readBuilder = table.newReadBuilder(); + TableScan.Plan plan = readBuilder.newScan().plan(); + try (RecordReaderIterator it = + new RecordReaderIterator<>(readBuilder.newRead().createReader(plan))) { + testTable.assertResult(schema, it); + } } private FileStoreTable createFileStoreTable( @@ -195,6 +202,10 @@ private FileStoreTable createFileStoreTable( conf.set(CoreOptions.DYNAMIC_BUCKET_TARGET_ROW_NUM, 100L); conf.set(CoreOptions.WRITE_BUFFER_SIZE, new MemorySize(4096 * 3)); conf.set(CoreOptions.PAGE_SIZE, new MemorySize(4096)); + // disable compaction for unaware bucket mode to avoid instability + if (primaryKeys.isEmpty() && numBucket == -1) { + conf.set(CoreOptions.WRITE_ONLY, true); + } TableSchema tableSchema = SchemaUtils.forceCommit( diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestTable.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestTable.java index beb8786d140d..525a05096942 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestTable.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestTable.java @@ -19,11 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.reader.RecordReaderIterator; import org.apache.paimon.schema.TableSchema; -import org.apache.paimon.table.FileStoreTable; -import org.apache.paimon.table.source.ReadBuilder; -import org.apache.paimon.table.source.TableScan; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; @@ -33,6 +29,7 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -188,40 +185,23 @@ public Queue events() { return events; } - public void assertResult(FileStoreTable table) throws Exception { + public void assertResult(TableSchema schema, Iterator it) { Map> actual = new HashMap<>(); - while (true) { - actual.clear(); - table = table.copyWithLatestSchema(); - TableSchema schema = table.schema(); - ReadBuilder readBuilder = table.newReadBuilder(); - TableScan.Plan plan = readBuilder.newScan().plan(); - - try (RecordReaderIterator it = - new RecordReaderIterator<>(readBuilder.newRead().createReader(plan))) { - while (it.hasNext()) { - InternalRow row = it.next(); - Map fields = new HashMap<>(); - for (int i = 0; i < schema.fieldNames().size(); i++) { - if (!row.isNullAt(i)) { - fields.put( - schema.fieldNames().get(i), - String.valueOf( - schema.fields().get(i).type().equals(DataTypes.BIGINT()) - ? row.getLong(i) - : row.getInt(i))); - } - } - actual.put(Integer.valueOf(fields.get("k")), fields); + while (it.hasNext()) { + InternalRow row = it.next(); + Map fields = new HashMap<>(); + for (int i = 0; i < schema.fieldNames().size(); i++) { + if (!row.isNullAt(i)) { + fields.put( + schema.fieldNames().get(i), + String.valueOf( + schema.fields().get(i).type().equals(DataTypes.BIGINT()) + ? row.getLong(i) + : row.getInt(i))); } } - - if (actual.size() == expected.size()) { - assertThat(actual).isEqualTo(expected); - break; - } - - Thread.sleep(500); + actual.put(Integer.valueOf(fields.get("k")), fields); } + assertThat(actual).isEqualTo(expected); } } From 2798db60d3d663c60d1832011ad06086505d5df7 Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Fri, 15 Mar 2024 13:22:48 +0800 Subject: [PATCH 02/79] [flink] Refactor: flink tests use common CatalogITCaseBase (#3016) --- paimon-flink/paimon-flink-1.15/pom.xml | 63 +------- .../paimon/flink/BatchFileStoreITCase.java | 2 +- .../paimon/flink/CatalogITCaseBase.java | 147 ------------------ .../flink/ContinuousFileStoreITCase.java | 2 +- .../paimon/flink/FlinkActionITCase.java | 2 +- .../apache/paimon/flink/LookupJoinITCase.java | 2 +- paimon-flink/paimon-flink-1.16/pom.xml | 15 +- .../paimon/flink/CatalogITCaseBase.java | 120 -------------- .../paimon/flink/SchemaChangeITCase.java | 2 +- paimon-flink/paimon-flink-cdc/pom.xml | 64 -------- paimon-flink/paimon-flink-common/pom.xml | 69 -------- paimon-flink/pom.xml | 64 +++++--- 12 files changed, 64 insertions(+), 488 deletions(-) delete mode 100644 paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java delete mode 100644 paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java diff --git a/paimon-flink/paimon-flink-1.15/pom.xml b/paimon-flink/paimon-flink-1.15/pom.xml index 18c4e0e8ef1c..3694bfb9a76f 100644 --- a/paimon-flink/paimon-flink-1.15/pom.xml +++ b/paimon-flink/paimon-flink-1.15/pom.xml @@ -71,21 +71,16 @@ under the License. - org.apache.flink - flink-test-utils - ${flink.version} + org.apache.paimon + paimon-flink-common + ${project.version} + test-jar test - - - log4j - log4j - - org.apache.flink - flink-table-planner_${scala.binary.version} + flink-test-utils ${flink.version} test @@ -94,14 +89,14 @@ under the License. org.apache.flink flink-table-planner_${scala.binary.version} ${flink.version} - test-jar test org.apache.flink - flink-connector-base + flink-table-planner_${scala.binary.version} ${flink.version} + test-jar test @@ -112,50 +107,6 @@ under the License. test - - org.apache.hadoop - hadoop-hdfs-client - ${hadoop.version} - provided - - - - org.apache.hadoop - hadoop-mapreduce-client-core - ${hadoop.version} - test - - - org.apache.curator - curator-test - - - org.apache.hadoop - hadoop-yarn-common - - - org.apache.avro - avro - - - org.codehaus.jackson - jackson-core-asl - - - org.slf4j - slf4j-log4j12 - - - jdk.tools - jdk.tools - - - com.google.protobuf - protobuf-java - - - - com.ververica frocksdbjni 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 5d61204e5b21..abf2564a1b5b 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 @@ -21,7 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.flink.types.Row; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Collections; import java.util.List; diff --git a/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java b/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java deleted file mode 100644 index d14e0649508f..000000000000 --- a/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink; - -import org.apache.paimon.Snapshot; -import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.fs.Path; -import org.apache.paimon.fs.local.LocalFileIO; -import org.apache.paimon.table.Table; -import org.apache.paimon.utils.SnapshotManager; - -import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; - -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.config.ExecutionConfigOptions; -import org.apache.flink.table.api.internal.TableEnvironmentImpl; -import org.apache.flink.table.delegation.Parser; -import org.apache.flink.table.operations.Operation; -import org.apache.flink.table.operations.ddl.CreateCatalogOperation; -import org.apache.flink.test.util.AbstractTestBase; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.junit.Before; - -import javax.annotation.Nullable; - -import java.io.File; -import java.io.IOException; -import java.time.Duration; -import java.util.Collections; -import java.util.List; - -import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL; - -/** ITCase for catalog. */ -public abstract class CatalogITCaseBase extends AbstractTestBase { - - protected TableEnvironment tEnv; - protected TableEnvironment sEnv; - protected String path; - - @Before - public void before() throws IOException { - tEnv = TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); - String catalog = "PAIMON"; - path = getTempDirPath("paimon"); - tEnv.executeSql( - String.format( - "CREATE CATALOG %s WITH (" + "'type'='paimon', 'warehouse'='%s')", - catalog, path)); - tEnv.useCatalog(catalog); - - sEnv = TableEnvironment.create(EnvironmentSettings.newInstance().inStreamingMode().build()); - sEnv.getConfig().getConfiguration().set(CHECKPOINTING_INTERVAL, Duration.ofMillis(100)); - sEnv.registerCatalog(catalog, tEnv.getCatalog(catalog).get()); - sEnv.useCatalog(catalog); - - setParallelism(defaultParallelism()); - prepareEnv(); - } - - private void prepareEnv() { - Parser parser = ((TableEnvironmentImpl) tEnv).getParser(); - for (String ddl : ddl()) { - tEnv.executeSql(ddl); - List operations = parser.parse(ddl); - if (operations.size() == 1) { - Operation operation = operations.get(0); - if (operation instanceof CreateCatalogOperation) { - String name = ((CreateCatalogOperation) operation).getCatalogName(); - sEnv.registerCatalog(name, tEnv.getCatalog(name).orElse(null)); - } - } - } - } - - protected void setParallelism(int parallelism) { - tEnv.getConfig() - .getConfiguration() - .set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, parallelism); - sEnv.getConfig() - .getConfiguration() - .set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, parallelism); - } - - protected int defaultParallelism() { - return 2; - } - - protected List ddl() { - return Collections.emptyList(); - } - - protected List batchSql(String query, Object... args) { - return sql(query, args); - } - - protected List sql(String query, Object... args) { - try (CloseableIterator iter = tEnv.executeSql(String.format(query, args)).collect()) { - return ImmutableList.copyOf(iter); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - protected CloseableIterator streamSqlIter(String query, Object... args) { - return sEnv.executeSql(String.format(query, args)).collect(); - } - - protected Table paimonTable(String tableName) - throws org.apache.paimon.catalog.Catalog.TableNotExistException { - FlinkCatalog flinkCatalog = (FlinkCatalog) tEnv.getCatalog(tEnv.getCurrentCatalog()).get(); - org.apache.paimon.catalog.Catalog catalog = flinkCatalog.catalog(); - return catalog.getTable(Identifier.create(tEnv.getCurrentDatabase(), tableName)); - } - - protected Path getTableDirectory(String tableName) { - return new Path( - new File(path, String.format("%s.db/%s", tEnv.getCurrentDatabase(), tableName)) - .toString()); - } - - @Nullable - protected Snapshot findLatestSnapshot(String tableName) { - SnapshotManager snapshotManager = - new SnapshotManager(LocalFileIO.create(), getTableDirectory(tableName)); - Long id = snapshotManager.latestSnapshotId(); - return id == null ? null : snapshotManager.snapshot(id); - } -} diff --git a/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java b/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java index 0e14aaeebe39..e1c62e28f599 100644 --- a/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java +++ b/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java @@ -26,7 +26,7 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; import org.apache.flink.types.Row; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; diff --git a/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/FlinkActionITCase.java b/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/FlinkActionITCase.java index 22df2a57acd0..2d743218d40f 100644 --- a/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/FlinkActionITCase.java +++ b/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/FlinkActionITCase.java @@ -22,7 +22,7 @@ import org.apache.paimon.utils.BlockingIterator; import org.apache.flink.types.Row; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Collections; import java.util.List; diff --git a/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/LookupJoinITCase.java b/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/LookupJoinITCase.java index f9f2a10b6583..e78208fb3be4 100644 --- a/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/LookupJoinITCase.java +++ b/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/LookupJoinITCase.java @@ -21,7 +21,7 @@ import org.apache.paimon.utils.BlockingIterator; import org.apache.flink.types.Row; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.List; diff --git a/paimon-flink/paimon-flink-1.16/pom.xml b/paimon-flink/paimon-flink-1.16/pom.xml index fdb7ebb5721a..9aaa19228ed5 100644 --- a/paimon-flink/paimon-flink-1.16/pom.xml +++ b/paimon-flink/paimon-flink-1.16/pom.xml @@ -72,23 +72,24 @@ under the License. - org.apache.flink - flink-test-utils - ${flink.version} + org.apache.paimon + paimon-flink-common + ${project.version} + test-jar test org.apache.flink - flink-table-planner_${scala.binary.version} + flink-test-utils ${flink.version} test - org.apache.hadoop - hadoop-hdfs-client - ${hadoop.version} + org.apache.flink + flink-table-planner_${scala.binary.version} + ${flink.version} test diff --git a/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java b/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java deleted file mode 100644 index 54f686190333..000000000000 --- a/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink; - -import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; - -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.config.ExecutionConfigOptions; -import org.apache.flink.table.api.internal.TableEnvironmentImpl; -import org.apache.flink.table.catalog.Catalog; -import org.apache.flink.table.catalog.CatalogBaseTable; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; -import org.apache.flink.table.delegation.Parser; -import org.apache.flink.table.operations.Operation; -import org.apache.flink.table.operations.ddl.CreateCatalogOperation; -import org.apache.flink.test.util.AbstractTestBase; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.junit.Before; - -import java.io.IOException; -import java.time.Duration; -import java.util.Collections; -import java.util.List; - -import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL; - -/** ITCase for catalog. */ -public abstract class CatalogITCaseBase extends AbstractTestBase { - - protected TableEnvironment tEnv; - protected TableEnvironment sEnv; - protected String path; - - @Before - public void before() throws IOException { - tEnv = TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); - String catalog = "PAIMON"; - path = getTempDirPath("paimon"); - tEnv.executeSql( - String.format( - "CREATE CATALOG %s WITH (" + "'type'='paimon', 'warehouse'='%s')", - catalog, path)); - tEnv.useCatalog(catalog); - - sEnv = TableEnvironment.create(EnvironmentSettings.newInstance().inStreamingMode().build()); - sEnv.getConfig().getConfiguration().set(CHECKPOINTING_INTERVAL, Duration.ofMillis(100)); - sEnv.registerCatalog(catalog, tEnv.getCatalog(catalog).get()); - sEnv.useCatalog(catalog); - - setParallelism(defaultParallelism()); - prepareEnv(); - } - - private void prepareEnv() { - Parser parser = ((TableEnvironmentImpl) tEnv).getParser(); - for (String ddl : ddl()) { - tEnv.executeSql(ddl); - List operations = parser.parse(ddl); - if (operations.size() == 1) { - Operation operation = operations.get(0); - if (operation instanceof CreateCatalogOperation) { - String name = ((CreateCatalogOperation) operation).getCatalogName(); - sEnv.registerCatalog(name, tEnv.getCatalog(name).orElse(null)); - } - } - } - } - - protected void setParallelism(int parallelism) { - tEnv.getConfig() - .getConfiguration() - .set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, parallelism); - sEnv.getConfig() - .getConfiguration() - .set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, parallelism); - } - - protected int defaultParallelism() { - return 2; - } - - protected List ddl() { - return Collections.emptyList(); - } - - protected List sql(String query, Object... args) { - try (CloseableIterator iter = tEnv.executeSql(String.format(query, args)).collect()) { - return ImmutableList.copyOf(iter); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - protected CatalogTable table(String tableName) throws TableNotExistException { - Catalog catalog = tEnv.getCatalog(tEnv.getCurrentCatalog()).get(); - CatalogBaseTable table = - catalog.getTable(new ObjectPath(catalog.getDefaultDatabase(), tableName)); - return (CatalogTable) table; - } -} diff --git a/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java b/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java index 663319ec989a..0335b9bef05c 100644 --- a/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java +++ b/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java @@ -18,7 +18,7 @@ package org.apache.paimon.flink; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Map; diff --git a/paimon-flink/paimon-flink-cdc/pom.xml b/paimon-flink/paimon-flink-cdc/pom.xml index dab19aec2309..c7f4dcbc96f5 100644 --- a/paimon-flink/paimon-flink-cdc/pom.xml +++ b/paimon-flink/paimon-flink-cdc/pom.xml @@ -66,20 +66,6 @@ under the License. provided - - org.apache.flink - flink-table-common - ${flink.version} - provided - - - - org.apache.flink - flink-streaming-java - ${flink.version} - provided - - org.apache.flink flink-table-runtime @@ -175,20 +161,6 @@ under the License. test - - org.apache.flink - flink-table-api-java - ${flink.version} - test - - - - org.apache.flink - flink-table-api-java-bridge - ${flink.version} - test - - org.apache.flink flink-table-planner_${scala.binary.version} @@ -251,42 +223,6 @@ under the License. - - org.apache.hadoop - hadoop-common - ${hadoop.version} - test - - - org.apache.avro - avro - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - jdk.tools - jdk.tools - - - com.google.protobuf - protobuf-java - - - - - - org.apache.hadoop - hadoop-hdfs-client - ${hadoop.version} - test - - org.testcontainers mysql diff --git a/paimon-flink/paimon-flink-common/pom.xml b/paimon-flink/paimon-flink-common/pom.xml index c4bc2a1d1fd9..efca0b1bde64 100644 --- a/paimon-flink/paimon-flink-common/pom.xml +++ b/paimon-flink/paimon-flink-common/pom.xml @@ -52,13 +52,6 @@ under the License. provided - - org.apache.flink - flink-table-common - ${flink.version} - provided - - org.apache.flink flink-table-runtime @@ -66,31 +59,6 @@ under the License. provided - - org.apache.hadoop - hadoop-hdfs - ${hadoop.version} - provided - - - jdk.tools - jdk.tools - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - com.google.protobuf - protobuf-java - - - - @@ -101,43 +69,6 @@ under the License. test - - org.apache.hadoop - hadoop-mapreduce-client-core - ${hadoop.version} - test - - - org.apache.curator - curator-test - - - org.apache.hadoop - hadoop-yarn-common - - - org.apache.avro - avro - - - org.codehaus.jackson - jackson-core-asl - - - org.slf4j - slf4j-log4j12 - - - jdk.tools - jdk.tools - - - com.google.protobuf - protobuf-java - - - - org.apache.flink flink-connector-test-utils diff --git a/paimon-flink/pom.xml b/paimon-flink/pom.xml index e81d4be8793e..30ef22d8c573 100644 --- a/paimon-flink/pom.xml +++ b/paimon-flink/pom.xml @@ -62,6 +62,13 @@ under the License. ${project.version} + + org.apache.hadoop + hadoop-hdfs-client + ${hadoop.version} + provided + + org.apache.hadoop hadoop-common @@ -93,6 +100,43 @@ under the License. + + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop.version} + test + + + org.apache.curator + curator-test + + + org.apache.hadoop + hadoop-yarn-common + + + org.apache.avro + avro + + + org.codehaus.jackson + jackson-core-asl + + + org.slf4j + slf4j-log4j12 + + + jdk.tools + jdk.tools + + + com.google.protobuf + protobuf-java + + + + org.apache.paimon paimon-core @@ -107,26 +151,6 @@ under the License. ${project.version} test - - - junit - junit - ${junit4.version} - test - - - - org.junit.vintage - junit-vintage-engine - ${junit5.version} - - - junit - junit - - - test - From 4ec5806428a8d3ddbb0ba8521b8971f0e9de7b17 Mon Sep 17 00:00:00 2001 From: Xiangyu Feng Date: Thu, 14 Mar 2024 16:58:14 +0800 Subject: [PATCH 03/79] [core] Support compiled class cache in CodeGenUtils This closes #2976 --- .../apache/paimon/codegen/CodeGenUtils.java | 196 +++++++++++++++-- .../paimon/codegen/CodeGenUtilsTest.java | 207 ++++++++++++++++++ 2 files changed, 388 insertions(+), 15 deletions(-) create mode 100644 paimon-core/src/test/java/org/apache/paimon/codegen/CodeGenUtilsTest.java diff --git a/paimon-core/src/main/java/org/apache/paimon/codegen/CodeGenUtils.java b/paimon-core/src/main/java/org/apache/paimon/codegen/CodeGenUtils.java index 57abd915a96d..290a10b6fe5b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/codegen/CodeGenUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/codegen/CodeGenUtils.java @@ -21,13 +21,29 @@ import org.apache.paimon.data.BinaryRow; import org.apache.paimon.types.DataType; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.Pair; +import org.apache.paimon.shade.guava30.com.google.common.cache.Cache; +import org.apache.paimon.shade.guava30.com.google.common.cache.CacheBuilder; + +import java.time.Duration; +import java.util.Arrays; import java.util.List; +import java.util.Objects; import java.util.stream.IntStream; /** Utils for code generations. */ public class CodeGenUtils { + static final Cache, Object[]>> COMPILED_CLASS_CACHE = + CacheBuilder.newBuilder() + // assume the table schema will stay the same for a period of time + .expireAfterAccess(Duration.ofMinutes(30)) + // estimated cache size + .maximumSize(300) + .softValues() + .build(); + public static final Projection EMPTY_PROJECTION = input -> BinaryRow.EMPTY_ROW; public static Projection newProjection(RowType inputType, List fields) { @@ -40,17 +56,121 @@ public static Projection newProjection(RowType inputType, int[] mapping) { if (mapping.length == 0) { return EMPTY_PROJECTION; } + String className = "Projection"; + ClassKey classKey = + new ClassKey(Projection.class, className, inputType.getFieldTypes(), mapping); - return CodeGenLoader.getCodeGenerator() - .generateProjection("Projection", inputType, mapping) - .newInstance(CodeGenUtils.class.getClassLoader()); + try { + Pair, Object[]> classPair = + COMPILED_CLASS_CACHE.get( + classKey, + () -> { + GeneratedClass generatedClass = + CodeGenLoader.getCodeGenerator() + .generateProjection(className, inputType, mapping); + return Pair.of( + generatedClass.compile(CodeGenUtils.class.getClassLoader()), + generatedClass.getReferences()); + }); + + return (Projection) + classPair + .getLeft() + .getConstructor(Object[].class) + .newInstance(new Object[] {classPair.getRight()}); + } catch (Exception e) { + throw new RuntimeException( + "Could not instantiate generated class '" + className + "'", e); + } } public static NormalizedKeyComputer newNormalizedKeyComputer( List inputTypes, int[] sortFields, String name) { - return CodeGenLoader.getCodeGenerator() - .generateNormalizedKeyComputer(inputTypes, sortFields, name) - .newInstance(CodeGenUtils.class.getClassLoader()); + ClassKey classKey = new ClassKey(NormalizedKeyComputer.class, name, inputTypes, sortFields); + + try { + Pair, Object[]> classPair = + COMPILED_CLASS_CACHE.get( + classKey, + () -> { + GeneratedClass generatedClass = + CodeGenLoader.getCodeGenerator() + .generateNormalizedKeyComputer( + inputTypes, sortFields, name); + return Pair.of( + generatedClass.compile(CodeGenUtils.class.getClassLoader()), + generatedClass.getReferences()); + }); + + return (NormalizedKeyComputer) + classPair + .getLeft() + .getConstructor(Object[].class) + .newInstance(new Object[] {classPair.getRight()}); + } catch (Exception e) { + throw new RuntimeException("Could not instantiate generated class '" + name + "'", e); + } + } + + public static RecordEqualiser newRecordEqualiser(List inputTypes, String name) { + ClassKey classKey = + new ClassKey( + RecordEqualiser.class, + name, + inputTypes, + IntStream.range(0, inputTypes.size()).toArray()); + + try { + Pair, Object[]> classPair = + COMPILED_CLASS_CACHE.get( + classKey, + () -> { + GeneratedClass generatedClass = + generateRecordEqualiser(inputTypes, name); + return Pair.of( + generatedClass.compile(CodeGenUtils.class.getClassLoader()), + generatedClass.getReferences()); + }); + + return (RecordEqualiser) + classPair + .getLeft() + .getConstructor(Object[].class) + .newInstance(new Object[] {classPair.getRight()}); + } catch (Exception e) { + throw new RuntimeException("Could not instantiate generated class '" + name + "'", e); + } + } + + public static RecordComparator newRecordComparator(List inputTypes, String name) { + return newRecordComparator( + inputTypes, IntStream.range(0, inputTypes.size()).toArray(), name); + } + + public static RecordComparator newRecordComparator( + List inputTypes, int[] sortFields, String name) { + ClassKey classKey = new ClassKey(RecordComparator.class, name, inputTypes, sortFields); + + try { + Pair, Object[]> classPair = + COMPILED_CLASS_CACHE.get( + classKey, + () -> { + GeneratedClass generatedClass = + generateRecordComparator(inputTypes, sortFields, name); + return Pair.of( + generatedClass.compile(CodeGenUtils.class.getClassLoader()), + generatedClass.getReferences()); + }); + + return (RecordComparator) + classPair + .getLeft() + .getConstructor(Object[].class) + .newInstance(new Object[] {classPair.getRight()}); + } catch (Exception e) { + throw new RuntimeException("Could not instantiate generated class '" + name + "'", e); + } } public static GeneratedClass generateRecordComparator( @@ -64,15 +184,61 @@ public static GeneratedClass generateRecordEqualiser( return CodeGenLoader.getCodeGenerator().generateRecordEqualiser(fieldTypes, name); } - public static RecordComparator newRecordComparator( - List inputTypes, int[] sortFields, String name) { - return generateRecordComparator(inputTypes, sortFields, name) - .newInstance(CodeGenUtils.class.getClassLoader()); - } + /** Class to use as key for the {@link #COMPILED_CLASS_CACHE}. */ + public static class ClassKey { - public static RecordComparator newRecordComparator(List inputTypes, String name) { - return generateRecordComparator( - inputTypes, IntStream.range(0, inputTypes.size()).toArray(), name) - .newInstance(CodeGenUtils.class.getClassLoader()); + private final Class classType; + + private final String className; + + private final List fields; + + private final int[] fieldsIndex; + + public ClassKey( + Class classType, String className, List fields, int[] fieldsIndex) { + this.classType = classType; + this.className = className; + this.fields = fields; + this.fieldsIndex = fieldsIndex; + } + + public Class getClassType() { + return classType; + } + + public String getClassName() { + return className; + } + + public List getFields() { + return fields; + } + + public int[] getFieldsIndex() { + return fieldsIndex; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ClassKey classKey = (ClassKey) o; + return Objects.equals(classType, classKey.classType) + && Objects.equals(className, classKey.className) + && Objects.equals(fields, classKey.fields) + && Arrays.equals(fieldsIndex, classKey.fieldsIndex); + } + + @Override + public int hashCode() { + int result = Objects.hash(classType, className, fields); + result = 31 * result + Arrays.hashCode(fieldsIndex); + return result; + } } } diff --git a/paimon-core/src/test/java/org/apache/paimon/codegen/CodeGenUtilsTest.java b/paimon-core/src/test/java/org/apache/paimon/codegen/CodeGenUtilsTest.java new file mode 100644 index 000000000000..207aa4b7b1a6 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/codegen/CodeGenUtilsTest.java @@ -0,0 +1,207 @@ +/* + * 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.codegen; + +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DoubleType; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.RowType; +import org.apache.paimon.types.VarCharType; +import org.apache.paimon.utils.Pair; + +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.IntStream; + +import static org.apache.paimon.codegen.CodeGenUtils.ClassKey; +import static org.assertj.core.api.Assertions.assertThat; + +class CodeGenUtilsTest { + + @BeforeAll + public static void before() { + // cleanup cached class before tests + CodeGenUtils.COMPILED_CLASS_CACHE.invalidateAll(); + } + + @Test + public void testProjectionCodegenCache() { + String name = "Projection"; + List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); + List dataTypes2 = Arrays.asList(new VarCharType(1), new IntType()); + int[] fieldIndexes1 = new int[] {0, 1}; + int[] fieldIndexes2 = new int[] {0, 1}; + + Projection projection = + CodeGenUtils.newProjection( + RowType.builder().fields(dataTypes1).build(), fieldIndexes1); + + ClassKey classKey = new ClassKey(Projection.class, name, dataTypes2, fieldIndexes2); + Pair, Object[]> classPair = + CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); + + assertThat(classPair).isNotNull(); + assertThat(projection.getClass()).isEqualTo(classPair.getLeft()); + } + + @Test + public void testProjectionCodegenCacheMiss() { + String name = "Projection"; + List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); + List dataTypes2 = + Arrays.asList(new VarCharType(1), new IntType(), new DoubleType()); + int[] fieldIndexes1 = new int[] {0, 1}; + int[] fieldIndexes2 = new int[] {0, 1, 2}; + + CodeGenUtils.newProjection(RowType.builder().fields(dataTypes1).build(), fieldIndexes1); + + ClassKey classKey = new ClassKey(Projection.class, name, dataTypes2, fieldIndexes2); + Pair, Object[]> classPair = + CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); + + assertThat(classPair).isNull(); + } + + @Test + public void testNormalizedKeyComputerCodegenCache() { + String name1 = "NormalizedKeyComputer"; + String name2 = "NormalizedKeyComputer"; + List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); + List dataTypes2 = Arrays.asList(new VarCharType(1), new IntType()); + int[] fieldIndexes1 = new int[] {0, 1}; + int[] fieldIndexes2 = new int[] {0, 1}; + + NormalizedKeyComputer normalizedKeyComputer = + CodeGenUtils.newNormalizedKeyComputer(dataTypes1, fieldIndexes1, name1); + + ClassKey classKey = + new ClassKey(NormalizedKeyComputer.class, name2, dataTypes2, fieldIndexes2); + Pair, Object[]> classPair = + CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); + + assertThat(classPair).isNotNull(); + assertThat(normalizedKeyComputer.getClass()).isEqualTo(classPair.getLeft()); + } + + @Test + public void testNormalizedKeyComputerCodegenCacheMiss() { + String name1 = "NormalizedKeyComputer"; + String name2 = "NormalizedKeyComputer"; + List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); + List dataTypes2 = + Arrays.asList(new VarCharType(1), new IntType(), new DoubleType()); + int[] fieldIndexes1 = new int[] {0, 1}; + int[] fieldIndexes2 = new int[] {0, 1, 2}; + + CodeGenUtils.newNormalizedKeyComputer(dataTypes1, fieldIndexes1, name1); + + ClassKey classKey = + new ClassKey(NormalizedKeyComputer.class, name2, dataTypes2, fieldIndexes2); + Pair, Object[]> classPair = + CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); + + assertThat(classPair).isNull(); + } + + @Test + public void testRecordComparatorCodegenCache() { + String name1 = "RecordComparator"; + String name2 = "RecordComparator"; + List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); + List dataTypes2 = Arrays.asList(new VarCharType(1), new IntType()); + int[] fieldIndexes1 = new int[] {0, 1}; + int[] fieldIndexes2 = new int[] {0, 1}; + + RecordComparator recordComparator = + CodeGenUtils.newRecordComparator(dataTypes1, fieldIndexes1, name1); + + ClassKey classKey = new ClassKey(RecordComparator.class, name2, dataTypes2, fieldIndexes2); + Pair, Object[]> classPair = + CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); + + assertThat(classPair).isNotNull(); + assertThat(recordComparator.getClass()).isEqualTo(classPair.getLeft()); + } + + @Test + public void testRecordComparatorCodegenCacheMiss() { + String name1 = "RecordComparator"; + String name2 = "RecordComparator"; + List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); + List dataTypes2 = + Arrays.asList(new VarCharType(1), new IntType(), new DoubleType()); + int[] fieldIndexes1 = new int[] {0, 1}; + int[] fieldIndexes2 = new int[] {0, 1, 2}; + + CodeGenUtils.newRecordComparator(dataTypes1, fieldIndexes1, name1); + + ClassKey classKey = new ClassKey(RecordComparator.class, name2, dataTypes2, fieldIndexes2); + Pair, Object[]> classPair = + CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); + + assertThat(classPair).isNull(); + } + + @Test + public void testRecordEqualiserCodegenCache() { + String name1 = "RecordEqualiser"; + String name2 = "RecordEqualiser"; + List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); + List dataTypes2 = Arrays.asList(new VarCharType(1), new IntType()); + + RecordEqualiser recordEqualiser = CodeGenUtils.newRecordEqualiser(dataTypes1, name1); + + ClassKey classKey = + new ClassKey( + RecordEqualiser.class, + name2, + dataTypes2, + IntStream.range(0, dataTypes2.size()).toArray()); + Pair, Object[]> classPair = + CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); + + assertThat(classPair).isNotNull(); + assertThat(recordEqualiser.getClass()).isEqualTo(classPair.getLeft()); + } + + @Test + public void testRecordEqualiserCodegenCacheMiss() { + String name1 = "RecordEqualiser"; + String name2 = "RecordEqualiser"; + List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); + List dataTypes2 = + Arrays.asList(new VarCharType(1), new IntType(), new DoubleType()); + + CodeGenUtils.newRecordEqualiser(dataTypes1, name1); + + ClassKey classKey = + new ClassKey( + RecordEqualiser.class, + name2, + dataTypes2, + IntStream.range(0, dataTypes2.size()).toArray()); + Pair, Object[]> classPair = + CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); + + assertThat(classPair).isNull(); + } +} From 6d114192b0115a224cb22b8fdebc595bb43693a7 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Fri, 15 Mar 2024 14:18:43 +0800 Subject: [PATCH 04/79] [core] Optimize compiled class cache in CodeGenUtils --- .../paimon/codegen/CodeGeneratorImpl.java | 19 +- .../apache/paimon/codegen/CodeGenerator.java | 9 +- .../apache/paimon/codegen/GeneratedClass.java | 14 +- .../apache/paimon/codegen/CodeGenUtils.java | 175 +++++----------- .../mergetree/SortBufferWriteBuffer.java | 5 +- .../compact/aggregate/FieldCollectAgg.java | 7 +- .../aggregate/FieldNestedUpdateAgg.java | 10 +- .../paimon/sort/BinaryExternalSortBuffer.java | 6 +- .../source/snapshot/SnapshotReaderImpl.java | 3 +- .../paimon/utils/KeyComparatorSupplier.java | 18 +- .../utils/UserDefinedSeqComparator.java | 3 +- .../paimon/utils/ValueEqualiserSupplier.java | 13 +- .../paimon/codegen/CodeGenUtilsTest.java | 195 +++++------------- .../flink/lookup/DynamicPartitionLoader.java | 3 +- 14 files changed, 145 insertions(+), 335 deletions(-) diff --git a/paimon-codegen/src/main/java/org/apache/paimon/codegen/CodeGeneratorImpl.java b/paimon-codegen/src/main/java/org/apache/paimon/codegen/CodeGeneratorImpl.java index c4189542219e..b8efa170deae 100644 --- a/paimon-codegen/src/main/java/org/apache/paimon/codegen/CodeGeneratorImpl.java +++ b/paimon-codegen/src/main/java/org/apache/paimon/codegen/CodeGeneratorImpl.java @@ -28,37 +28,34 @@ public class CodeGeneratorImpl implements CodeGenerator { @Override - public GeneratedClass generateProjection( - String name, RowType inputType, int[] inputMapping) { + public GeneratedClass generateProjection(RowType inputType, int[] inputMapping) { RowType outputType = TypeUtils.project(inputType, inputMapping); return ProjectionCodeGenerator.generateProjection( - new CodeGeneratorContext(), name, inputType, outputType, inputMapping); + new CodeGeneratorContext(), "Projection", inputType, outputType, inputMapping); } @Override public GeneratedClass generateNormalizedKeyComputer( - List inputTypes, int[] sortFields, String name) { + List inputTypes, int[] sortFields) { return new SortCodeGenerator( RowType.builder().fields(inputTypes).build(), getAscendingSortSpec(sortFields)) - .generateNormalizedKeyComputer(name); + .generateNormalizedKeyComputer("NormalizedKeyComputer"); } @Override public GeneratedClass generateRecordComparator( - List inputTypes, int[] sortFields, String name) { + List inputTypes, int[] sortFields) { return ComparatorCodeGenerator.gen( - name, + "RecordComparator", RowType.builder().fields(inputTypes).build(), getAscendingSortSpec(sortFields)); } - /** Generate a {@link RecordEqualiser}. */ @Override - public GeneratedClass generateRecordEqualiser( - List fieldTypes, String name) { + public GeneratedClass generateRecordEqualiser(List fieldTypes) { return new EqualiserCodeGenerator(RowType.builder().fields(fieldTypes).build()) - .generateRecordEqualiser(name); + .generateRecordEqualiser("RecordEqualiser"); } private SortSpec getAscendingSortSpec(int[] sortFields) { diff --git a/paimon-common/src/main/java/org/apache/paimon/codegen/CodeGenerator.java b/paimon-common/src/main/java/org/apache/paimon/codegen/CodeGenerator.java index 842098dbcbed..a9b48b70b58e 100644 --- a/paimon-common/src/main/java/org/apache/paimon/codegen/CodeGenerator.java +++ b/paimon-common/src/main/java/org/apache/paimon/codegen/CodeGenerator.java @@ -26,8 +26,7 @@ /** {@link GeneratedClass} generator. */ public interface CodeGenerator { - GeneratedClass generateProjection( - String name, RowType inputType, int[] inputMapping); + GeneratedClass generateProjection(RowType inputType, int[] inputMapping); /** * Generate a {@link NormalizedKeyComputer}. @@ -37,7 +36,7 @@ GeneratedClass generateProjection( * second field, then the third field and so on. All fields are compared in ascending order. */ GeneratedClass generateNormalizedKeyComputer( - List inputTypes, int[] sortFields, String name); + List inputTypes, int[] sortFields); /** * Generate a {@link RecordComparator}. @@ -47,7 +46,7 @@ GeneratedClass generateNormalizedKeyComputer( * second field, then the third field and so on. All fields are compared in ascending order. */ GeneratedClass generateRecordComparator( - List inputTypes, int[] sortFields, String name); + List inputTypes, int[] sortFields); /** * Generate a {@link RecordEqualiser}. @@ -56,5 +55,5 @@ GeneratedClass generateRecordComparator( * compared by the first field, then the second field, then the third field and so on. All * * fields are compared in ascending order. */ - GeneratedClass generateRecordEqualiser(List fieldTypes, String name); + GeneratedClass generateRecordEqualiser(List fieldTypes); } diff --git a/paimon-common/src/main/java/org/apache/paimon/codegen/GeneratedClass.java b/paimon-common/src/main/java/org/apache/paimon/codegen/GeneratedClass.java index 97d63923084e..edfe380e8db8 100644 --- a/paimon-common/src/main/java/org/apache/paimon/codegen/GeneratedClass.java +++ b/paimon-common/src/main/java/org/apache/paimon/codegen/GeneratedClass.java @@ -72,13 +72,15 @@ public T newInstance(ClassLoader classLoader) { } } - @SuppressWarnings("unchecked") - public T newInstance(ClassLoader classLoader, Object... args) { + /** Create a new instance of this generated class. */ + public static T newInstance(Class clazz, Object[] references) { try { - return (T) compile(classLoader).getConstructors()[0].newInstance(args); - } catch (Exception e) { - throw new RuntimeException( - "Could not instantiate generated class '" + className + "'", e); + return clazz.getConstructor(Object[].class) + // Because Constructor.newInstance(Object... initargs), we need to load + // references into a new Object[], otherwise it cannot be compiled. + .newInstance(new Object[] {references}); + } catch (Throwable e) { + throw new RuntimeException("Could not instantiate generated class '" + clazz + "'", e); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/codegen/CodeGenUtils.java b/paimon-core/src/main/java/org/apache/paimon/codegen/CodeGenUtils.java index 290a10b6fe5b..7b7e992c704d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/codegen/CodeGenUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/codegen/CodeGenUtils.java @@ -30,12 +30,15 @@ import java.util.Arrays; import java.util.List; import java.util.Objects; +import java.util.function.Supplier; import java.util.stream.IntStream; +import static org.apache.paimon.codegen.CodeGenLoader.getCodeGenerator; + /** Utils for code generations. */ public class CodeGenUtils { - static final Cache, Object[]>> COMPILED_CLASS_CACHE = + private static final Cache, Object[]>> COMPILED_CLASS_CACHE = CacheBuilder.newBuilder() // assume the table schema will stay the same for a period of time .expireAfterAccess(Duration.ofMinutes(30)) @@ -56,169 +59,84 @@ public static Projection newProjection(RowType inputType, int[] mapping) { if (mapping.length == 0) { return EMPTY_PROJECTION; } - String className = "Projection"; - ClassKey classKey = - new ClassKey(Projection.class, className, inputType.getFieldTypes(), mapping); - - try { - Pair, Object[]> classPair = - COMPILED_CLASS_CACHE.get( - classKey, - () -> { - GeneratedClass generatedClass = - CodeGenLoader.getCodeGenerator() - .generateProjection(className, inputType, mapping); - return Pair.of( - generatedClass.compile(CodeGenUtils.class.getClassLoader()), - generatedClass.getReferences()); - }); - return (Projection) - classPair - .getLeft() - .getConstructor(Object[].class) - .newInstance(new Object[] {classPair.getRight()}); - } catch (Exception e) { - throw new RuntimeException( - "Could not instantiate generated class '" + className + "'", e); - } + return generate( + Projection.class, + inputType.getFieldTypes(), + mapping, + () -> getCodeGenerator().generateProjection(inputType, mapping)); } public static NormalizedKeyComputer newNormalizedKeyComputer( - List inputTypes, int[] sortFields, String name) { - ClassKey classKey = new ClassKey(NormalizedKeyComputer.class, name, inputTypes, sortFields); - - try { - Pair, Object[]> classPair = - COMPILED_CLASS_CACHE.get( - classKey, - () -> { - GeneratedClass generatedClass = - CodeGenLoader.getCodeGenerator() - .generateNormalizedKeyComputer( - inputTypes, sortFields, name); - return Pair.of( - generatedClass.compile(CodeGenUtils.class.getClassLoader()), - generatedClass.getReferences()); - }); - - return (NormalizedKeyComputer) - classPair - .getLeft() - .getConstructor(Object[].class) - .newInstance(new Object[] {classPair.getRight()}); - } catch (Exception e) { - throw new RuntimeException("Could not instantiate generated class '" + name + "'", e); - } + List inputTypes, int[] sortFields) { + return generate( + NormalizedKeyComputer.class, + inputTypes, + sortFields, + () -> getCodeGenerator().generateNormalizedKeyComputer(inputTypes, sortFields)); } - public static RecordEqualiser newRecordEqualiser(List inputTypes, String name) { - ClassKey classKey = - new ClassKey( - RecordEqualiser.class, - name, - inputTypes, - IntStream.range(0, inputTypes.size()).toArray()); - - try { - Pair, Object[]> classPair = - COMPILED_CLASS_CACHE.get( - classKey, - () -> { - GeneratedClass generatedClass = - generateRecordEqualiser(inputTypes, name); - return Pair.of( - generatedClass.compile(CodeGenUtils.class.getClassLoader()), - generatedClass.getReferences()); - }); + public static RecordComparator newRecordComparator(List inputTypes) { + return newRecordComparator(inputTypes, IntStream.range(0, inputTypes.size()).toArray()); + } - return (RecordEqualiser) - classPair - .getLeft() - .getConstructor(Object[].class) - .newInstance(new Object[] {classPair.getRight()}); - } catch (Exception e) { - throw new RuntimeException("Could not instantiate generated class '" + name + "'", e); - } + public static RecordComparator newRecordComparator( + List inputTypes, int[] sortFields) { + return generate( + RecordComparator.class, + inputTypes, + sortFields, + () -> getCodeGenerator().generateRecordComparator(inputTypes, sortFields)); } - public static RecordComparator newRecordComparator(List inputTypes, String name) { - return newRecordComparator( - inputTypes, IntStream.range(0, inputTypes.size()).toArray(), name); + public static RecordEqualiser newRecordEqualiser(List fieldTypes) { + return generate( + RecordEqualiser.class, + fieldTypes, + IntStream.range(0, fieldTypes.size()).toArray(), + () -> getCodeGenerator().generateRecordEqualiser(fieldTypes)); } - public static RecordComparator newRecordComparator( - List inputTypes, int[] sortFields, String name) { - ClassKey classKey = new ClassKey(RecordComparator.class, name, inputTypes, sortFields); + private static T generate( + Class classType, + List fields, + int[] fieldsIndex, + Supplier> supplier) { + ClassKey classKey = new ClassKey(classType, fields, fieldsIndex); try { - Pair, Object[]> classPair = + Pair, Object[]> result = COMPILED_CLASS_CACHE.get( classKey, () -> { - GeneratedClass generatedClass = - generateRecordComparator(inputTypes, sortFields, name); + GeneratedClass generatedClass = supplier.get(); return Pair.of( generatedClass.compile(CodeGenUtils.class.getClassLoader()), generatedClass.getReferences()); }); - return (RecordComparator) - classPair - .getLeft() - .getConstructor(Object[].class) - .newInstance(new Object[] {classPair.getRight()}); + //noinspection unchecked + return (T) GeneratedClass.newInstance(result.getLeft(), result.getRight()); } catch (Exception e) { - throw new RuntimeException("Could not instantiate generated class '" + name + "'", e); + throw new RuntimeException( + "Could not instantiate generated class '" + classType + "'", e); } } - public static GeneratedClass generateRecordComparator( - List inputTypes, int[] sortFields, String name) { - return CodeGenLoader.getCodeGenerator() - .generateRecordComparator(inputTypes, sortFields, name); - } - - public static GeneratedClass generateRecordEqualiser( - List fieldTypes, String name) { - return CodeGenLoader.getCodeGenerator().generateRecordEqualiser(fieldTypes, name); - } - - /** Class to use as key for the {@link #COMPILED_CLASS_CACHE}. */ - public static class ClassKey { + private static class ClassKey { private final Class classType; - private final String className; - private final List fields; private final int[] fieldsIndex; - public ClassKey( - Class classType, String className, List fields, int[] fieldsIndex) { + public ClassKey(Class classType, List fields, int[] fieldsIndex) { this.classType = classType; - this.className = className; this.fields = fields; this.fieldsIndex = fieldsIndex; } - public Class getClassType() { - return classType; - } - - public String getClassName() { - return className; - } - - public List getFields() { - return fields; - } - - public int[] getFieldsIndex() { - return fieldsIndex; - } - @Override public boolean equals(Object o) { if (this == o) { @@ -229,14 +147,13 @@ public boolean equals(Object o) { } ClassKey classKey = (ClassKey) o; return Objects.equals(classType, classKey.classType) - && Objects.equals(className, classKey.className) && Objects.equals(fields, classKey.fields) && Arrays.equals(fieldsIndex, classKey.fieldsIndex); } @Override public int hashCode() { - int result = Objects.hash(classType, className, fields); + int result = Objects.hash(classType, fields); result = 31 * result + Arrays.hashCode(fieldsIndex); return result; } diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/SortBufferWriteBuffer.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/SortBufferWriteBuffer.java index f0e9fcf07cf8..072c8a0508c6 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/SortBufferWriteBuffer.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/SortBufferWriteBuffer.java @@ -96,10 +96,9 @@ public SortBufferWriteBuffer( fieldTypes.addAll(valueType.getFieldTypes()); NormalizedKeyComputer normalizedKeyComputer = - CodeGenUtils.newNormalizedKeyComputer( - fieldTypes, sortFieldArray, "MemTableKeyComputer"); + CodeGenUtils.newNormalizedKeyComputer(fieldTypes, sortFieldArray); RecordComparator keyComparator = - CodeGenUtils.newRecordComparator(fieldTypes, sortFieldArray, "MemTableComparator"); + CodeGenUtils.newRecordComparator(fieldTypes, sortFieldArray); if (memoryPool.freePages() < 3) { throw new IllegalArgumentException( diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldCollectAgg.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldCollectAgg.java index d335b327a342..b153287041f9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldCollectAgg.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldCollectAgg.java @@ -18,7 +18,6 @@ package org.apache.paimon.mergetree.compact.aggregate; -import org.apache.paimon.codegen.CodeGenUtils; import org.apache.paimon.codegen.RecordEqualiser; import org.apache.paimon.data.GenericArray; import org.apache.paimon.data.GenericRow; @@ -39,6 +38,8 @@ import java.util.List; import java.util.function.BiFunction; +import static org.apache.paimon.codegen.CodeGenUtils.newRecordEqualiser; + /** Collect elements into an ARRAY. */ public class FieldCollectAgg extends FieldAggregator { @@ -63,9 +64,7 @@ public FieldCollectAgg(ArrayType dataType, boolean distinct) { elementType instanceof RowType ? ((RowType) elementType).getFieldTypes() : Collections.singletonList(elementType); - RecordEqualiser elementEqualiser = - CodeGenUtils.generateRecordEqualiser(fieldTypes, "elementEqualiser") - .newInstance(FieldCollectAgg.class.getClassLoader()); + RecordEqualiser elementEqualiser = newRecordEqualiser(fieldTypes); this.equaliser = (o1, o2) -> { InternalRow row1, row2; diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldNestedUpdateAgg.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldNestedUpdateAgg.java index c3a459550bf5..5e0149345eea 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldNestedUpdateAgg.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldNestedUpdateAgg.java @@ -18,7 +18,6 @@ package org.apache.paimon.mergetree.compact.aggregate; -import org.apache.paimon.codegen.CodeGenUtils; import org.apache.paimon.codegen.Projection; import org.apache.paimon.codegen.RecordEqualiser; import org.apache.paimon.data.BinaryRow; @@ -35,6 +34,8 @@ import java.util.List; import java.util.Map; +import static org.apache.paimon.codegen.CodeGenUtils.newProjection; +import static org.apache.paimon.codegen.CodeGenUtils.newRecordEqualiser; import static org.apache.paimon.utils.Preconditions.checkNotNull; /** @@ -56,12 +57,9 @@ public FieldNestedUpdateAgg(ArrayType dataType, List nestedKey) { this.nestedFields = nestedType.getFieldCount(); if (nestedKey.isEmpty()) { this.keyProjection = null; - this.elementEqualiser = - CodeGenUtils.generateRecordEqualiser( - nestedType.getFieldTypes(), "elementEqualiser") - .newInstance(FieldNestedUpdateAgg.class.getClassLoader()); + this.elementEqualiser = newRecordEqualiser(nestedType.getFieldTypes()); } else { - this.keyProjection = CodeGenUtils.newProjection(nestedType, nestedKey); + this.keyProjection = newProjection(nestedType, nestedKey); this.elementEqualiser = null; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/sort/BinaryExternalSortBuffer.java b/paimon-core/src/main/java/org/apache/paimon/sort/BinaryExternalSortBuffer.java index 4de25a4ea9cd..f7cf9092859d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/sort/BinaryExternalSortBuffer.java +++ b/paimon-core/src/main/java/org/apache/paimon/sort/BinaryExternalSortBuffer.java @@ -113,12 +113,10 @@ public static BinaryExternalSortBuffer create( MemorySegmentPool pool, int maxNumFileHandles, String compression) { - RecordComparator comparator = - newRecordComparator(rowType.getFieldTypes(), keyFields, "ExternalSort_comparator"); + RecordComparator comparator = newRecordComparator(rowType.getFieldTypes(), keyFields); BinaryInMemorySortBuffer sortBuffer = BinaryInMemorySortBuffer.createBuffer( - newNormalizedKeyComputer( - rowType.getFieldTypes(), keyFields, "ExternalSort_normalized_key"), + newNormalizedKeyComputer(rowType.getFieldTypes(), keyFields), new InternalRowSerializer(rowType), comparator, pool); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java index 9c8126ffe3a1..06836d481aa5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java @@ -402,8 +402,7 @@ private RecordComparator partitionComparator() { if (lazyPartitionComparator == null) { lazyPartitionComparator = CodeGenUtils.newRecordComparator( - tableSchema.logicalPartitionType().getFieldTypes(), - "PartitionComparator"); + tableSchema.logicalPartitionType().getFieldTypes()); } return lazyPartitionComparator; } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/KeyComparatorSupplier.java b/paimon-core/src/main/java/org/apache/paimon/utils/KeyComparatorSupplier.java index 30d3ab32874e..6f90cef01a20 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/KeyComparatorSupplier.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/KeyComparatorSupplier.java @@ -18,33 +18,33 @@ package org.apache.paimon.utils; -import org.apache.paimon.codegen.CodeGenUtils; -import org.apache.paimon.codegen.GeneratedClass; import org.apache.paimon.codegen.RecordComparator; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.types.DataType; import org.apache.paimon.types.RowType; import java.util.Comparator; +import java.util.List; import java.util.function.Supplier; import java.util.stream.IntStream; +import static org.apache.paimon.codegen.CodeGenUtils.newRecordComparator; + /** A {@link Supplier} that returns the comparator for the file store key. */ public class KeyComparatorSupplier implements SerializableSupplier> { private static final long serialVersionUID = 1L; - private final GeneratedClass genRecordComparator; + private final List inputTypes; + private final int[] sortFields; public KeyComparatorSupplier(RowType keyType) { - genRecordComparator = - CodeGenUtils.generateRecordComparator( - keyType.getFieldTypes(), - IntStream.range(0, keyType.getFieldCount()).toArray(), - "KeyComparator"); + this.inputTypes = keyType.getFieldTypes(); + this.sortFields = IntStream.range(0, keyType.getFieldCount()).toArray(); } @Override public RecordComparator get() { - return genRecordComparator.newInstance(KeyComparatorSupplier.class.getClassLoader()); + return newRecordComparator(inputTypes, sortFields); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/UserDefinedSeqComparator.java b/paimon-core/src/main/java/org/apache/paimon/utils/UserDefinedSeqComparator.java index bab0ba377ddc..35fa7a66d775 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/UserDefinedSeqComparator.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/UserDefinedSeqComparator.java @@ -63,8 +63,7 @@ public static UserDefinedSeqComparator create(RowType rowType, List sequ List fieldNames = rowType.getFieldNames(); int[] fields = sequenceFields.stream().mapToInt(fieldNames::indexOf).toArray(); RecordComparator comparator = - CodeGenUtils.newRecordComparator( - rowType.getFieldTypes(), fields, "UserDefinedSeqComparator"); + CodeGenUtils.newRecordComparator(rowType.getFieldTypes(), fields); return new UserDefinedSeqComparator(fields, comparator); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/ValueEqualiserSupplier.java b/paimon-core/src/main/java/org/apache/paimon/utils/ValueEqualiserSupplier.java index 0878c18158a4..6de3ded2f2e4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/ValueEqualiserSupplier.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/ValueEqualiserSupplier.java @@ -18,27 +18,28 @@ package org.apache.paimon.utils; -import org.apache.paimon.codegen.CodeGenUtils; -import org.apache.paimon.codegen.GeneratedClass; import org.apache.paimon.codegen.RecordEqualiser; +import org.apache.paimon.types.DataType; import org.apache.paimon.types.RowType; +import java.util.List; import java.util.function.Supplier; +import static org.apache.paimon.codegen.CodeGenUtils.newRecordEqualiser; + /** A {@link Supplier} that returns the equaliser for the file store value. */ public class ValueEqualiserSupplier implements SerializableSupplier { private static final long serialVersionUID = 1L; - private final GeneratedClass genRecordEqualiser; + private final List fieldTypes; public ValueEqualiserSupplier(RowType keyType) { - genRecordEqualiser = - CodeGenUtils.generateRecordEqualiser(keyType.getFieldTypes(), "valueEqualiser"); + this.fieldTypes = keyType.getFieldTypes(); } @Override public RecordEqualiser get() { - return genRecordEqualiser.newInstance(ValueEqualiserSupplier.class.getClassLoader()); + return newRecordEqualiser(fieldTypes); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/codegen/CodeGenUtilsTest.java b/paimon-core/src/test/java/org/apache/paimon/codegen/CodeGenUtilsTest.java index 207aa4b7b1a6..b7e416af137b 100644 --- a/paimon-core/src/test/java/org/apache/paimon/codegen/CodeGenUtilsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/codegen/CodeGenUtilsTest.java @@ -18,190 +18,93 @@ package org.apache.paimon.codegen; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DoubleType; -import org.apache.paimon.types.IntType; import org.apache.paimon.types.RowType; -import org.apache.paimon.types.VarCharType; -import org.apache.paimon.utils.Pair; -import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import java.util.Arrays; -import java.util.List; -import java.util.stream.IntStream; - -import static org.apache.paimon.codegen.CodeGenUtils.ClassKey; +import java.util.function.Supplier; + +import static org.apache.paimon.codegen.CodeGenUtils.newNormalizedKeyComputer; +import static org.apache.paimon.codegen.CodeGenUtils.newProjection; +import static org.apache.paimon.codegen.CodeGenUtils.newRecordComparator; +import static org.apache.paimon.codegen.CodeGenUtils.newRecordEqualiser; +import static org.apache.paimon.types.DataTypes.DOUBLE; +import static org.apache.paimon.types.DataTypes.INT; +import static org.apache.paimon.types.DataTypes.STRING; import static org.assertj.core.api.Assertions.assertThat; class CodeGenUtilsTest { - @BeforeAll - public static void before() { - // cleanup cached class before tests - CodeGenUtils.COMPILED_CLASS_CACHE.invalidateAll(); - } - @Test public void testProjectionCodegenCache() { - String name = "Projection"; - List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); - List dataTypes2 = Arrays.asList(new VarCharType(1), new IntType()); - int[] fieldIndexes1 = new int[] {0, 1}; - int[] fieldIndexes2 = new int[] {0, 1}; - - Projection projection = - CodeGenUtils.newProjection( - RowType.builder().fields(dataTypes1).build(), fieldIndexes1); - - ClassKey classKey = new ClassKey(Projection.class, name, dataTypes2, fieldIndexes2); - Pair, Object[]> classPair = - CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); - - assertThat(classPair).isNotNull(); - assertThat(projection.getClass()).isEqualTo(classPair.getLeft()); + assertClassEquals( + () -> + newProjection( + RowType.builder().fields(STRING(), INT()).build(), + new int[] {0, 1})); } @Test public void testProjectionCodegenCacheMiss() { - String name = "Projection"; - List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); - List dataTypes2 = - Arrays.asList(new VarCharType(1), new IntType(), new DoubleType()); - int[] fieldIndexes1 = new int[] {0, 1}; - int[] fieldIndexes2 = new int[] {0, 1, 2}; - - CodeGenUtils.newProjection(RowType.builder().fields(dataTypes1).build(), fieldIndexes1); - - ClassKey classKey = new ClassKey(Projection.class, name, dataTypes2, fieldIndexes2); - Pair, Object[]> classPair = - CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); - - assertThat(classPair).isNull(); + assertClassNotEquals( + newProjection(RowType.builder().fields(STRING(), INT()).build(), new int[] {0, 1}), + newProjection( + RowType.builder().fields(STRING(), INT(), DOUBLE()).build(), + new int[] {0, 1, 2})); } @Test public void testNormalizedKeyComputerCodegenCache() { - String name1 = "NormalizedKeyComputer"; - String name2 = "NormalizedKeyComputer"; - List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); - List dataTypes2 = Arrays.asList(new VarCharType(1), new IntType()); - int[] fieldIndexes1 = new int[] {0, 1}; - int[] fieldIndexes2 = new int[] {0, 1}; - - NormalizedKeyComputer normalizedKeyComputer = - CodeGenUtils.newNormalizedKeyComputer(dataTypes1, fieldIndexes1, name1); - - ClassKey classKey = - new ClassKey(NormalizedKeyComputer.class, name2, dataTypes2, fieldIndexes2); - Pair, Object[]> classPair = - CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); - - assertThat(classPair).isNotNull(); - assertThat(normalizedKeyComputer.getClass()).isEqualTo(classPair.getLeft()); + assertClassEquals( + () -> newNormalizedKeyComputer(Arrays.asList(STRING(), INT()), new int[] {0, 1})); } @Test public void testNormalizedKeyComputerCodegenCacheMiss() { - String name1 = "NormalizedKeyComputer"; - String name2 = "NormalizedKeyComputer"; - List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); - List dataTypes2 = - Arrays.asList(new VarCharType(1), new IntType(), new DoubleType()); - int[] fieldIndexes1 = new int[] {0, 1}; - int[] fieldIndexes2 = new int[] {0, 1, 2}; - - CodeGenUtils.newNormalizedKeyComputer(dataTypes1, fieldIndexes1, name1); - - ClassKey classKey = - new ClassKey(NormalizedKeyComputer.class, name2, dataTypes2, fieldIndexes2); - Pair, Object[]> classPair = - CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); - - assertThat(classPair).isNull(); + assertClassNotEquals( + newNormalizedKeyComputer(Arrays.asList(STRING(), INT()), new int[] {0, 1}), + newNormalizedKeyComputer( + Arrays.asList(STRING(), INT(), DOUBLE()), new int[] {0, 1, 2})); } @Test public void testRecordComparatorCodegenCache() { - String name1 = "RecordComparator"; - String name2 = "RecordComparator"; - List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); - List dataTypes2 = Arrays.asList(new VarCharType(1), new IntType()); - int[] fieldIndexes1 = new int[] {0, 1}; - int[] fieldIndexes2 = new int[] {0, 1}; - - RecordComparator recordComparator = - CodeGenUtils.newRecordComparator(dataTypes1, fieldIndexes1, name1); - - ClassKey classKey = new ClassKey(RecordComparator.class, name2, dataTypes2, fieldIndexes2); - Pair, Object[]> classPair = - CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); - - assertThat(classPair).isNotNull(); - assertThat(recordComparator.getClass()).isEqualTo(classPair.getLeft()); + assertClassEquals( + () -> newRecordComparator(Arrays.asList(STRING(), INT()), new int[] {0, 1})); } @Test public void testRecordComparatorCodegenCacheMiss() { - String name1 = "RecordComparator"; - String name2 = "RecordComparator"; - List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); - List dataTypes2 = - Arrays.asList(new VarCharType(1), new IntType(), new DoubleType()); - int[] fieldIndexes1 = new int[] {0, 1}; - int[] fieldIndexes2 = new int[] {0, 1, 2}; - - CodeGenUtils.newRecordComparator(dataTypes1, fieldIndexes1, name1); - - ClassKey classKey = new ClassKey(RecordComparator.class, name2, dataTypes2, fieldIndexes2); - Pair, Object[]> classPair = - CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); - - assertThat(classPair).isNull(); + assertClassNotEquals( + newRecordComparator(Arrays.asList(STRING(), INT()), new int[] {0, 1}), + newRecordComparator(Arrays.asList(STRING(), INT(), DOUBLE()), new int[] {0, 1, 2})); } @Test public void testRecordEqualiserCodegenCache() { - String name1 = "RecordEqualiser"; - String name2 = "RecordEqualiser"; - List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); - List dataTypes2 = Arrays.asList(new VarCharType(1), new IntType()); - - RecordEqualiser recordEqualiser = CodeGenUtils.newRecordEqualiser(dataTypes1, name1); - - ClassKey classKey = - new ClassKey( - RecordEqualiser.class, - name2, - dataTypes2, - IntStream.range(0, dataTypes2.size()).toArray()); - Pair, Object[]> classPair = - CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); - - assertThat(classPair).isNotNull(); - assertThat(recordEqualiser.getClass()).isEqualTo(classPair.getLeft()); + assertClassEquals(() -> newRecordEqualiser(Arrays.asList(STRING(), INT()))); } @Test public void testRecordEqualiserCodegenCacheMiss() { - String name1 = "RecordEqualiser"; - String name2 = "RecordEqualiser"; - List dataTypes1 = Arrays.asList(new VarCharType(1), new IntType()); - List dataTypes2 = - Arrays.asList(new VarCharType(1), new IntType(), new DoubleType()); - - CodeGenUtils.newRecordEqualiser(dataTypes1, name1); - - ClassKey classKey = - new ClassKey( - RecordEqualiser.class, - name2, - dataTypes2, - IntStream.range(0, dataTypes2.size()).toArray()); - Pair, Object[]> classPair = - CodeGenUtils.COMPILED_CLASS_CACHE.getIfPresent(classKey); - - assertThat(classPair).isNull(); + assertClassNotEquals( + newRecordEqualiser(Arrays.asList(STRING(), INT())), + newRecordEqualiser(Arrays.asList(STRING(), INT(), DOUBLE()))); + } + + @Test + public void testHybridNotEqual() { + assertClassNotEquals( + newRecordComparator(Arrays.asList(STRING(), INT()), new int[] {0, 1}), + newNormalizedKeyComputer(Arrays.asList(STRING(), INT()), new int[] {0, 1})); + } + + private void assertClassEquals(Supplier supplier) { + assertThat(supplier.get().getClass()).isEqualTo(supplier.get().getClass()); + } + + private void assertClassNotEquals(Object o1, Object o2) { + assertThat(o1.getClass()).isNotEqualTo(o2.getClass()); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLoader.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLoader.java index 26c172621ac4..3e420633223e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLoader.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLoader.java @@ -63,8 +63,7 @@ private DynamicPartitionLoader(Table table, Duration refreshInterval) { public void open() { this.scan = table.newReadBuilder().newScan(); RowType partitionType = table.rowType().project(table.partitionKeys()); - this.comparator = - CodeGenUtils.newRecordComparator(partitionType.getFieldTypes(), "Partition"); + this.comparator = CodeGenUtils.newRecordComparator(partitionType.getFieldTypes()); } public void addJoinKeys(List joinKeys) { From 7c8e42b10d2749d04b7994a1c1e270c16135d2d9 Mon Sep 17 00:00:00 2001 From: YeJunHao <41894543+leaves12138@users.noreply.github.com> Date: Fri, 15 Mar 2024 15:49:45 +0800 Subject: [PATCH 05/79] [core] AvroBulkFormat should not get file-size by file-io repeatly (#3019) --- .../org/apache/paimon/format/avro/AvroBulkFormat.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) 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 1c1cfddf1fd2..da60eedda113 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 @@ -69,18 +69,18 @@ private class AvroReader implements RecordReader { private AvroReader(FileIO fileIO, Path path) throws IOException { this.fileIO = fileIO; - this.reader = createReaderFromPath(path); - this.reader.sync(0); this.end = fileIO.getFileSize(path); + this.reader = createReaderFromPath(path, end); + this.reader.sync(0); this.pool = new Pool<>(1); this.pool.add(new Object()); } - private DataFileReader createReaderFromPath(Path path) throws IOException { + private DataFileReader createReaderFromPath(Path path, long fileSize) + throws IOException { DatumReader datumReader = new AvroRowDatumReader(projectedRowType); SeekableInput in = - new SeekableInputStreamWrapper( - fileIO.newInputStream(path), fileIO.getFileSize(path)); + new SeekableInputStreamWrapper(fileIO.newInputStream(path), fileSize); try { return (DataFileReader) DataFileReader.openReader(in, datumReader); } catch (Throwable e) { From 8338cad5afb6f5b1d225791d7d6939f622bcb4bb Mon Sep 17 00:00:00 2001 From: wgcn <1026688210@qq.com> Date: Fri, 15 Mar 2024 16:10:37 +0800 Subject: [PATCH 06/79] [hive] Cache TableSchema into Configuration to avoid loading read scheme file in PaimonSerDe (#2946) --- .../apache/paimon/utils/JsonSerdeUtil.java | 8 +++++++ .../org/apache/paimon/hive/HiveSchema.java | 2 +- .../org/apache/paimon/hive/PaimonSerDe.java | 20 +++++++++++++--- .../paimon/hive/PaimonStorageHandler.java | 15 +++++++++--- .../paimon/hive/HiveTableSchemaTest.java | 23 +++++++++++++++++++ 5 files changed, 61 insertions(+), 7 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java b/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java index d45a9336847d..676276a30e58 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/JsonSerdeUtil.java @@ -120,6 +120,14 @@ public static T getNodeAs( fieldName, clazz.getName(), node.getClass().getName())); } + public static T fromJson(String json, TypeReference typeReference) { + try { + return OBJECT_MAPPER_INSTANCE.readValue(json, typeReference); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + public static T fromJson(String json, Class clazz) { try { return OBJECT_MAPPER_INSTANCE.reader().readValue(json, clazz); diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/HiveSchema.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/HiveSchema.java index dc6fc99219f8..f637651413ed 100644 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/HiveSchema.java +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/HiveSchema.java @@ -67,7 +67,7 @@ public class HiveSchema { private static final Logger LOG = LoggerFactory.getLogger(HiveSchema.class); private final RowType rowType; - private HiveSchema(RowType rowType) { + HiveSchema(RowType rowType) { this.rowType = rowType; } diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonSerDe.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonSerDe.java index 2934b7e77d9f..fe6a31b53c29 100644 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonSerDe.java +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonSerDe.java @@ -19,8 +19,12 @@ package org.apache.paimon.hive; import org.apache.paimon.hive.objectinspector.PaimonInternalRowObjectInspector; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.JsonSerdeUtil; import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.type.TypeReference; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.serde2.AbstractSerDe; @@ -32,6 +36,7 @@ import javax.annotation.Nullable; +import java.util.List; import java.util.Map; import java.util.Properties; @@ -53,11 +58,20 @@ public class PaimonSerDe extends AbstractSerDe { @Override public void initialize(@Nullable Configuration configuration, Properties properties) throws SerDeException { - HiveSchema schema = HiveSchema.extract(configuration, properties); - this.tableSchema = schema; + String dataFieldStr = properties.getProperty(PaimonStorageHandler.PAIMON_TABLE_FIELDS); + if (dataFieldStr != null) { + List dataFields = + JsonSerdeUtil.fromJson(dataFieldStr, new TypeReference>() {}); + this.tableSchema = new HiveSchema(new RowType(dataFields)); + } else { + this.tableSchema = HiveSchema.extract(configuration, properties); + } + inspector = new PaimonInternalRowObjectInspector( - schema.fieldNames(), schema.fieldTypes(), schema.fieldComments()); + tableSchema.fieldNames(), + tableSchema.fieldTypes(), + tableSchema.fieldComments()); } @Override diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonStorageHandler.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonStorageHandler.java index b410833a9b11..5987fd0c9dfb 100644 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonStorageHandler.java +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonStorageHandler.java @@ -21,6 +21,7 @@ import org.apache.paimon.hive.mapred.PaimonInputFormat; import org.apache.paimon.hive.mapred.PaimonOutputCommitter; import org.apache.paimon.hive.mapred.PaimonOutputFormat; +import org.apache.paimon.utils.JsonSerdeUtil; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.HiveMetaHook; @@ -46,6 +47,8 @@ public class PaimonStorageHandler implements HiveStoragePredicateHandler, HiveSt private static final String MAPRED_OUTPUT_COMMITTER = "mapred.output.committer.class"; private static final String PAIMON_WRITE = "paimon.write"; + public static final String PAIMON_TABLE_FIELDS = "paimon.table.fields"; + private Configuration conf; @Override @@ -76,9 +79,15 @@ public HiveAuthorizationProvider getAuthorizationProvider() throws HiveException @Override public void configureInputJobProperties(TableDesc tableDesc, Map map) { Properties properties = tableDesc.getProperties(); - map.put( - LocationKeyExtractor.INTERNAL_LOCATION, - LocationKeyExtractor.getPaimonLocation(conf, properties)); + String paimonLocation = LocationKeyExtractor.getPaimonLocation(conf, properties); + map.put(LocationKeyExtractor.INTERNAL_LOCATION, paimonLocation); + String dataFieldJsonStr = getDataFieldsJsonStr(properties); + tableDesc.getProperties().put(PAIMON_TABLE_FIELDS, dataFieldJsonStr); + } + + static String getDataFieldsJsonStr(Properties properties) { + HiveSchema hiveSchema = HiveSchema.extract(null, properties); + return JsonSerdeUtil.toJson(hiveSchema.fields()); } public void configureInputJobCredentials(TableDesc tableDesc, Map map) {} diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveTableSchemaTest.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveTableSchemaTest.java index 326667a47619..07cd00c8e67e 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveTableSchemaTest.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveTableSchemaTest.java @@ -25,7 +25,11 @@ import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.JsonSerdeUtil; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.type.TypeReference; + +import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -33,6 +37,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Properties; import static org.assertj.core.api.Assertions.assertThat; @@ -342,4 +347,22 @@ private Properties createTableWithExistsDDL() { properties.setProperty("location", tempDir.toString()); return properties; } + + @Test + public void testReadHiveSchemaFromProperties() throws Exception { + createSchema(); + // cache the TableSchema to properties + Properties properties = new Properties(); + properties.put(hive_metastoreConstants.META_TABLE_LOCATION, tempDir.toString()); + + HiveSchema hiveSchema = HiveSchema.extract(null, properties); + + List dataFields = hiveSchema.fields(); + String dataFieldStr = JsonSerdeUtil.toJson(dataFields); + + List dataFieldsDeserialized = + JsonSerdeUtil.fromJson(dataFieldStr, new TypeReference>() {}); + HiveSchema newHiveSchema = new HiveSchema(new RowType(dataFieldsDeserialized)); + assertThat(newHiveSchema).usingRecursiveComparison().isEqualTo(hiveSchema); + } } From 8a0aec65ee5c91857fd15d254a34ad0b9daab8f7 Mon Sep 17 00:00:00 2001 From: Fang Yong Date: Fri, 15 Mar 2024 16:27:09 +0800 Subject: [PATCH 07/79] [core] Support configuring lock in paimon catalog (#2933) * [core] Support configuring lock in paimon catalog --- .../generated/catalog_configuration.html | 8 ++- .../apache/paimon/options/CatalogOptions.java | 8 ++- .../paimon/catalog/AbstractCatalog.java | 35 ++++++++++ .../paimon/catalog/FileSystemCatalog.java | 30 ++++++--- .../org/apache/paimon/jdbc/JdbcCatalog.java | 15 ++--- .../paimon/jdbc/JdbcCatalogFactory.java | 12 +++- .../apache/paimon/jdbc/JdbcCatalogLock.java | 40 ++++++------ .../org.apache.paimon.factories.Factory | 1 + .../apache/paimon/jdbc/JdbcCatalogTest.java | 1 + .../paimon/flink/FileSystemCatalogITCase.java | 65 ++++++++++++++++++- .../org.apache.paimon.factories.Factory | 5 +- .../org/apache/paimon/hive/HiveCatalog.java | 27 ++++---- .../apache/paimon/hive/HiveCatalogLock.java | 14 ++-- .../org.apache.paimon.factories.Factory | 3 + 14 files changed, 197 insertions(+), 67 deletions(-) diff --git a/docs/layouts/shortcodes/generated/catalog_configuration.html b/docs/layouts/shortcodes/generated/catalog_configuration.html index e685559447e2..cab6e731e851 100644 --- a/docs/layouts/shortcodes/generated/catalog_configuration.html +++ b/docs/layouts/shortcodes/generated/catalog_configuration.html @@ -62,11 +62,17 @@ Boolean Enable Catalog Lock. + +
lock.type
+ (none) + String + The Lock Type for Catalog, such as 'hive', 'zookeeper'. +
metastore
"filesystem" String - Metastore of paimon catalog, supports filesystem、hive and jdbc. + Metastore of paimon catalog, supports filesystem, hive and jdbc.
table.type
diff --git a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java index 42cd9e418844..f00a35a75094 100644 --- a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java @@ -40,7 +40,7 @@ public class CatalogOptions { .stringType() .defaultValue("filesystem") .withDescription( - "Metastore of paimon catalog, supports filesystem、hive and jdbc."); + "Metastore of paimon catalog, supports filesystem, hive and jdbc."); public static final ConfigOption URI = ConfigOptions.key("uri") @@ -60,6 +60,12 @@ public class CatalogOptions { .defaultValue(false) .withDescription("Enable Catalog Lock."); + public static final ConfigOption LOCK_TYPE = + ConfigOptions.key("lock.type") + .stringType() + .noDefaultValue() + .withDescription("The Lock Type for Catalog, such as 'hive', 'zookeeper'."); + public static final ConfigOption LOCK_CHECK_MAX_SLEEP = key("lock-check-max-sleep") .durationType() diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index 0dbbb1f40302..c69a72b0db6a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -46,10 +46,13 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.UUID; import java.util.stream.Collectors; import static org.apache.paimon.options.CatalogOptions.LINEAGE_META; +import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED; +import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE; import static org.apache.paimon.options.OptionsUtils.convertToPropertiesPrefixKey; import static org.apache.paimon.utils.Preconditions.checkArgument; @@ -80,6 +83,30 @@ protected AbstractCatalog(FileIO fileIO, Options options) { this.tableDefaultOptions = convertToPropertiesPrefixKey(options.toMap(), TABLE_DEFAULT_OPTION_PREFIX); this.catalogOptions = options; + + if (lockEnabled()) { + checkArgument(options.contains(LOCK_TYPE), "No lock type when lock is enabled."); + } + } + + @Override + public Optional lockFactory() { + return lockEnabled() + ? Optional.of( + FactoryUtil.discoverFactory( + AbstractCatalog.class.getClassLoader(), + CatalogLock.LockFactory.class, + catalogOptions.get(LOCK_TYPE))) + : Optional.empty(); + } + + @Override + public Optional lockContext() { + return Optional.of(new OptionLockContext(catalogOptions)); + } + + protected boolean lockEnabled() { + return catalogOptions.get(LOCK_ENABLED); } @Override @@ -465,4 +492,12 @@ private void validateAutoCreateClose(Map options) { "The value of %s property should be %s.", CoreOptions.AUTO_CREATE.key(), Boolean.FALSE)); } + + static class OptionLockContext implements CatalogLock.LockContext { + private final Options catalogOptions; + + public OptionLockContext(Options catalogOptions) { + this.catalogOptions = catalogOptions; + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java index 0c01e9cb7a71..e71c92dc4007 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java @@ -21,6 +21,7 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; +import org.apache.paimon.operation.Lock; import org.apache.paimon.options.Options; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; @@ -34,7 +35,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.Callable; import static org.apache.paimon.catalog.FileSystemCatalogOptions.CASE_SENSITIVE; @@ -56,11 +56,6 @@ public FileSystemCatalog(FileIO fileIO, Path warehouse, Options options) { this.warehouse = warehouse; } - @Override - public Optional lockFactory() { - return Optional.empty(); - } - @Override public List listDatabases() { List databases = new ArrayList<>(); @@ -128,8 +123,7 @@ private boolean tableExists(Path tablePath) { @Override public TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException { - Path path = getDataTableLocation(identifier); - return new SchemaManager(fileIO, path) + return schemaManager(identifier) .latest() .orElseThrow(() -> new TableNotExistException(identifier)); } @@ -142,8 +136,24 @@ protected void dropTableImpl(Identifier identifier) { @Override public void createTableImpl(Identifier identifier, Schema schema) { + uncheck(() -> schemaManager(identifier).createTable(schema)); + } + + private SchemaManager schemaManager(Identifier identifier) { Path path = getDataTableLocation(identifier); - uncheck(() -> new SchemaManager(fileIO, path).createTable(schema)); + CatalogLock catalogLock = + lockFactory() + .map( + fac -> + fac.create( + lockContext() + .orElseThrow( + () -> + new RuntimeException( + "No lock context when lock is enabled.")))) + .orElse(null); + return new SchemaManager(fileIO, path) + .withLock(catalogLock == null ? null : Lock.fromCatalog(catalogLock, identifier)); } @Override @@ -156,7 +166,7 @@ public void renameTableImpl(Identifier fromTable, Identifier toTable) { @Override protected void alterTableImpl(Identifier identifier, List changes) throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException { - new SchemaManager(fileIO, getDataTableLocation(identifier)).commitChanges(changes); + schemaManager(identifier).commitChanges(changes); } private static T uncheck(Callable callable) { diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java index 61dc5959c5b5..689a93ee91f7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java @@ -26,6 +26,7 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.operation.Lock; import org.apache.paimon.options.CatalogOptions; +import org.apache.paimon.options.Options; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.SchemaManager; @@ -55,7 +56,6 @@ import static org.apache.paimon.jdbc.JdbcUtils.execute; import static org.apache.paimon.jdbc.JdbcUtils.insertProperties; import static org.apache.paimon.jdbc.JdbcUtils.updateTable; -import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED; /* This file is based on source code from the Iceberg Project (http://iceberg.apache.org/), licensed by the Apache * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for @@ -76,7 +76,7 @@ public class JdbcCatalog extends AbstractCatalog { protected JdbcCatalog( FileIO fileIO, String catalogKey, Map config, String warehouse) { - super(fileIO); + super(fileIO, Options.fromMap(config)); this.catalogKey = catalogKey; this.options = config; this.warehouse = warehouse; @@ -347,15 +347,8 @@ public boolean caseSensitive() { } @Override - public Optional lockFactory() { - return lockEnabled() - ? Optional.of(JdbcCatalogLock.createFactory(connections, catalogKey, options)) - : Optional.empty(); - } - - private boolean lockEnabled() { - return Boolean.parseBoolean( - options.getOrDefault(LOCK_ENABLED.key(), LOCK_ENABLED.defaultValue().toString())); + public Optional lockContext() { + return Optional.of(new JdbcCatalogLock.JdbcLockContext(connections, catalogKey, options)); } private Lock lock(Identifier identifier) { diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogFactory.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogFactory.java index ff438a8c8f85..5e605923206b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogFactory.java @@ -23,6 +23,10 @@ import org.apache.paimon.catalog.CatalogFactory; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.options.Options; + +import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED; +import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE; /** Factory to create {@link JdbcCatalog}. */ public class JdbcCatalogFactory implements CatalogFactory { @@ -36,7 +40,13 @@ public String identifier() { @Override public Catalog create(FileIO fileIO, Path warehouse, CatalogContext context) { - String catalogKey = context.options().get(JdbcCatalogOptions.CATALOG_KEY); + Options options = context.options(); + String catalogKey = options.get(JdbcCatalogOptions.CATALOG_KEY); + if (options.get(LOCK_ENABLED)) { + if (!options.getOptional(LOCK_TYPE).isPresent()) { + options.set(LOCK_TYPE, JdbcCatalogLock.JdbcCatalogLockFactory.IDENTIFIER); + } + } return new JdbcCatalog(fileIO, catalogKey, context.options().toMap(), warehouse.toString()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java index 94287cb6e0a0..85f15f7f9d53 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java @@ -86,26 +86,11 @@ public void close() throws IOException { // Do nothing } - /** Create a jdbc lock factory. */ - public static LockFactory createFactory( - JdbcClientPool connections, String catalogName, Map conf) { - return new JdbcCatalogLockFactory(connections, catalogName, conf); - } - - private static class JdbcCatalogLockFactory implements LockFactory { + /** Jdbc catalog lock factory. */ + public static class JdbcCatalogLockFactory implements LockFactory { private static final long serialVersionUID = 1L; - private static final String IDENTIFIER = "jdbc"; - private final JdbcClientPool connections; - private final String catalogName; - private final Map conf; - - public JdbcCatalogLockFactory( - JdbcClientPool connections, String catalogName, Map conf) { - this.connections = connections; - this.catalogName = catalogName; - this.conf = conf; - } + public static final String IDENTIFIER = "jdbc"; @Override public String identifier() { @@ -114,8 +99,25 @@ public String identifier() { @Override public CatalogLock create(LockContext context) { + JdbcLockContext lockContext = (JdbcLockContext) context; return new JdbcCatalogLock( - connections, catalogName, checkMaxSleep(conf), acquireTimeout(conf)); + lockContext.connections, + lockContext.catalogName, + checkMaxSleep(lockContext.conf), + acquireTimeout(lockContext.conf)); + } + } + + static class JdbcLockContext implements LockContext { + private final JdbcClientPool connections; + private final String catalogName; + private final Map conf; + + public JdbcLockContext( + JdbcClientPool connections, String catalogName, Map conf) { + this.connections = connections; + this.catalogName = catalogName; + this.conf = conf; } } diff --git a/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index cc2b3f063138..34de4106bada 100644 --- a/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -15,3 +15,4 @@ org.apache.paimon.catalog.FileSystemCatalogFactory org.apache.paimon.jdbc.JdbcCatalogFactory +org.apache.paimon.jdbc.JdbcCatalogLock$JdbcCatalogLockFactory diff --git a/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java index 5cc79fc85da3..d03c64bd825e 100644 --- a/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java @@ -55,6 +55,7 @@ private JdbcCatalog initCatalog(Map props) { properties.put(JdbcCatalog.PROPERTY_PREFIX + "password", "password"); properties.put(CatalogOptions.WAREHOUSE.key(), warehouse); properties.put(CatalogOptions.LOCK_ENABLED.key(), "true"); + properties.put(CatalogOptions.LOCK_TYPE.key(), "jdbc"); properties.putAll(props); JdbcCatalog catalog = new JdbcCatalog(fileIO, "test-jdbc-catalog", properties, warehouse); assertThat(catalog.warehouse()).isEqualTo(warehouse); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java index 71f93553d49d..b68d65dd2251 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java @@ -20,6 +20,7 @@ import org.apache.paimon.catalog.AbstractCatalog; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogLock; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.util.AbstractTestBase; import org.apache.paimon.fs.Path; @@ -35,15 +36,19 @@ import org.junit.jupiter.api.Test; import java.io.File; +import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicInteger; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; /** ITCase for {@link FlinkCatalog}. */ public class FileSystemCatalogITCase extends AbstractTestBase { + private static final AtomicInteger LOCK_COUNT = new AtomicInteger(0); private static final String DB_NAME = "default"; @@ -113,7 +118,7 @@ public void testCatalogOptionsInheritAndOverride() throws Exception { + "'table-default.opt2'='value2', " + "'table-default.opt3'='value3', " + "'fs.allow-hadoop-fallback'='false'," - + "'lock.enabled'='true'" + + "'lock.enabled'='false'" + ")", path)); tEnv.useCatalog("fs_with_options"); @@ -146,6 +151,39 @@ public void testCatalogOptionsInheritAndOverride() throws Exception { assertThat(tableOptions).doesNotContainKey("lock.enabled"); } + @Test + void testCatalogWithLockForSchema() throws Exception { + LOCK_COUNT.set(0); + assertThatThrownBy( + () -> + tEnv.executeSql( + String.format( + "CREATE CATALOG fs_with_lock WITH (" + + "'type'='paimon', " + + "'warehouse'='%s', " + + "'lock.enabled'='true'" + + ")", + path)) + .await()) + .hasRootCauseMessage("No lock type when lock is enabled."); + tEnv.executeSql( + String.format( + "CREATE CATALOG fs_with_lock WITH (" + + "'type'='paimon', " + + "'warehouse'='%s', " + + "'lock.enabled'='true'," + + "'lock.type'='DUMMY'" + + ")", + path)) + .await(); + tEnv.useCatalog("fs_with_lock"); + tEnv.executeSql("CREATE TABLE table1 (a STRING, b STRING, c STRING)").await(); + tEnv.executeSql("CREATE TABLE table2 (a STRING, b STRING, c STRING)").await(); + tEnv.executeSql("CREATE TABLE table3 (a STRING, b STRING, c STRING)").await(); + tEnv.executeSql("DROP TABLE table3").await(); + assertThat(LOCK_COUNT.get()).isEqualTo(3); + } + private void innerTestWriteRead() throws Exception { tEnv.executeSql("INSERT INTO T VALUES ('1', '2', '3'), ('4', '5', '6')").await(); BlockingIterator iterator = @@ -163,4 +201,29 @@ private List collect(String sql) throws Exception { } return result; } + + /** Lock factory for file system catalog. */ + public static class FileSystemCatalogDummyLockFactory implements CatalogLock.LockFactory { + private static final String IDENTIFIER = "DUMMY"; + + @Override + public String identifier() { + return IDENTIFIER; + } + + @Override + public CatalogLock create(CatalogLock.LockContext context) { + return new CatalogLock() { + @Override + public T runWithLock(String database, String table, Callable callable) + throws Exception { + LOCK_COUNT.incrementAndGet(); + return callable.call(); + } + + @Override + public void close() throws IOException {} + }; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory index 22e88ba48420..fcb6fe982943 100644 --- a/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -16,4 +16,7 @@ org.apache.paimon.flink.FlinkCatalogTest$TestingLogSoreRegisterFactory # Lineage meta factory -org.apache.paimon.flink.FlinkLineageITCase$TestingMemoryLineageMetaFactory \ No newline at end of file +org.apache.paimon.flink.FlinkLineageITCase$TestingMemoryLineageMetaFactory + +# Catalog lock factory +org.apache.paimon.flink.FileSystemCatalogITCase$FileSystemCatalogDummyLockFactory \ No newline at end of file diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index 024857976888..589e920370e0 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -77,6 +77,7 @@ import java.util.stream.Collectors; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREWAREHOUSE; +import static org.apache.paimon.hive.HiveCatalogLock.LOCK_IDENTIFIER; import static org.apache.paimon.hive.HiveCatalogLock.acquireTimeout; import static org.apache.paimon.hive.HiveCatalogLock.checkMaxSleep; import static org.apache.paimon.hive.HiveCatalogOptions.HADOOP_CONF_DIR; @@ -84,6 +85,7 @@ import static org.apache.paimon.hive.HiveCatalogOptions.IDENTIFIER; import static org.apache.paimon.hive.HiveCatalogOptions.LOCATION_IN_PROPERTIES; import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED; +import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE; import static org.apache.paimon.options.CatalogOptions.TABLE_TYPE; import static org.apache.paimon.options.OptionsUtils.convertToPropertiesPrefixKey; import static org.apache.paimon.utils.Preconditions.checkArgument; @@ -146,11 +148,6 @@ public HiveCatalog( this.client = createClient(hiveConf, clientClassName); } - @Override - public Optional lockFactory() { - return lockEnabled() ? Optional.of(HiveCatalogLock.createFactory()) : Optional.empty(); - } - @Override public Optional lockContext() { return Optional.of( @@ -158,11 +155,6 @@ public Optional lockContext() { new SerializableHiveConf(hiveConf), clientClassName)); } - private boolean lockEnabled() { - return Boolean.parseBoolean( - hiveConf.get(LOCK_ENABLED.key(), LOCK_ENABLED.defaultValue().toString())); - } - @Override public Optional metastoreClientFactory(Identifier identifier) { try { @@ -670,7 +662,8 @@ public static boolean isEmbeddedMetastore(HiveConf hiveConf) { public static Catalog createHiveCatalog(CatalogContext context) { HiveConf hiveConf = createHiveConf(context); - String warehouseStr = context.options().get(CatalogOptions.WAREHOUSE); + Options options = context.options(); + String warehouseStr = options.get(CatalogOptions.WAREHOUSE); if (warehouseStr == null) { warehouseStr = hiveConf.get(METASTOREWAREHOUSE.varname, METASTOREWAREHOUSE.defaultStrVal); @@ -687,11 +680,19 @@ public static Catalog createHiveCatalog(CatalogContext context) { } catch (IOException e) { throw new UncheckedIOException(e); } + + /** Hive catalog only support hive lock. */ + if (options.getOptional(LOCK_ENABLED).orElse(false)) { + Optional lockType = options.getOptional(LOCK_TYPE); + if (!lockType.isPresent()) { + options.set(LOCK_TYPE, LOCK_IDENTIFIER); + } + } return new HiveCatalog( fileIO, hiveConf, - context.options().get(HiveCatalogFactory.METASTORE_CLIENT_CLASS), - context.options(), + options.get(HiveCatalogFactory.METASTORE_CLIENT_CLASS), + options, warehouse.toUri().toString()); } diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLock.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLock.java index 1635d80960f1..c49cd020c654 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLock.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLock.java @@ -44,6 +44,8 @@ /** Hive {@link CatalogLock}. */ public class HiveCatalogLock implements CatalogLock { + static final String LOCK_IDENTIFIER = "hive"; + private final IMetaStoreClient client; private final long checkMaxSleep; private final long acquireTimeout; @@ -112,17 +114,11 @@ public void close() { this.client.close(); } - /** Create a hive lock factory. */ - public static LockFactory createFactory() { - return new HiveCatalogLockFactory(); - } - - private static class HiveCatalogLockFactory implements LockFactory { + /** Catalog lock factory for hive. */ + public static class HiveCatalogLockFactory implements LockFactory { private static final long serialVersionUID = 1L; - private static final String IDENTIFIER = "hive"; - @Override public CatalogLock create(LockContext context) { checkArgument(context instanceof HiveLockContext); @@ -136,7 +132,7 @@ public CatalogLock create(LockContext context) { @Override public String identifier() { - return IDENTIFIER; + return LOCK_IDENTIFIER; } } diff --git a/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index dcc7e6554426..d4af13cc08e6 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -14,3 +14,6 @@ # limitations under the License. org.apache.paimon.hive.HiveCatalogFactory + +# Hive catalog lock factory +org.apache.paimon.hive.HiveCatalogLock$HiveCatalogLockFactory From 35acc4c2de17c6b36b985b73fdfe28b922971bec Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Fri, 15 Mar 2024 18:28:04 +0800 Subject: [PATCH 08/79] [codegen] EqualiserCodeGenerator supports ARRAY (#3023) --- .../codegen/EqualiserCodeGenerator.scala | 18 ++---------- .../paimon/codegen/ScalarOperatorGens.scala | 26 +++++++++++++++++ .../flink/FullCompactionFileStoreITCase.java | 29 +++++++++++++++++++ 3 files changed, 58 insertions(+), 15 deletions(-) diff --git a/paimon-codegen/src/main/scala/org/apache/paimon/codegen/EqualiserCodeGenerator.scala b/paimon-codegen/src/main/scala/org/apache/paimon/codegen/EqualiserCodeGenerator.scala index b76f170a6bf0..41c7427af433 100644 --- a/paimon-codegen/src/main/scala/org/apache/paimon/codegen/EqualiserCodeGenerator.scala +++ b/paimon-codegen/src/main/scala/org/apache/paimon/codegen/EqualiserCodeGenerator.scala @@ -19,9 +19,9 @@ package org.apache.paimon.codegen import org.apache.paimon.codegen.GenerateUtils._ -import org.apache.paimon.codegen.ScalarOperatorGens.generateEquals +import org.apache.paimon.codegen.ScalarOperatorGens.{generateEquals, generateRowEqualiser} import org.apache.paimon.types.{BooleanType, DataType, RowType} -import org.apache.paimon.types.DataTypeChecks.{getFieldTypes, isCompositeType} +import org.apache.paimon.types.DataTypeChecks.isCompositeType import org.apache.paimon.types.DataTypeRoot._ import org.apache.paimon.utils.TypeUtils.isPrimitive @@ -136,19 +136,7 @@ class EqualiserCodeGenerator(fieldTypes: Array[DataType]) { if (isInternalPrimitive(fieldType)) { ("", s"$leftFieldTerm == $rightFieldTerm") } else if (isCompositeType(fieldType)) { - val equaliserGenerator = - new EqualiserCodeGenerator(getFieldTypes(fieldType).asScala.toArray) - val generatedEqualiser = equaliserGenerator.generateRecordEqualiser("fieldGeneratedEqualiser") - val generatedEqualiserTerm = - ctx.addReusableObject(generatedEqualiser, "fieldGeneratedEqualiser") - val equaliserTypeTerm = classOf[RecordEqualiser].getCanonicalName - val equaliserTerm = newName("equaliser") - ctx.addReusableMember(s"private $equaliserTypeTerm $equaliserTerm = null;") - ctx.addReusableInitStatement( - s""" - |$equaliserTerm = ($equaliserTypeTerm) - | $generatedEqualiserTerm.newInstance(this.getClass().getClassLoader()); - |""".stripMargin) + val equaliserTerm = generateRowEqualiser(ctx, fieldType) ("", s"$equaliserTerm.equals($leftFieldTerm, $rightFieldTerm)") } else { val left = GeneratedExpression(leftFieldTerm, leftNullTerm, "", fieldType) diff --git a/paimon-codegen/src/main/scala/org/apache/paimon/codegen/ScalarOperatorGens.scala b/paimon-codegen/src/main/scala/org/apache/paimon/codegen/ScalarOperatorGens.scala index 841eea331e99..5dfa4bff6835 100644 --- a/paimon-codegen/src/main/scala/org/apache/paimon/codegen/ScalarOperatorGens.scala +++ b/paimon-codegen/src/main/scala/org/apache/paimon/codegen/ScalarOperatorGens.scala @@ -21,10 +21,13 @@ package org.apache.paimon.codegen import org.apache.paimon.codegen.GenerateUtils._ import org.apache.paimon.data.serializer.InternalMapSerializer import org.apache.paimon.types._ +import org.apache.paimon.types.DataTypeChecks.{getFieldTypes, isCompositeType} import org.apache.paimon.utils.InternalRowUtils import org.apache.paimon.utils.TypeCheckUtils._ import org.apache.paimon.utils.TypeUtils.isInteroperable +import scala.collection.JavaConverters._ + /** * Utilities to generate SQL scalar operators, e.g. arithmetic operator, compare operator, equal * operator, etc. @@ -78,6 +81,10 @@ object ScalarOperatorGens { // comparable types of same type else if (isComparable(left.resultType) && canEqual) { generateComparison(ctx, "==", left, right, resultType) + } else if (isCompositeType(left.resultType) && canEqual) { + val equaliserTerm = generateRowEqualiser(ctx, left.resultType) + generateOperatorIfNotNull(ctx, resultType, left, right)( + (leftTerm, rightTerm) => s"$equaliserTerm.equals($leftTerm, $rightTerm)") } // non comparable types else { @@ -95,6 +102,25 @@ object ScalarOperatorGens { } } + /** Generates [[RecordEqualiser]] code for row and return equaliser name. */ + def generateRowEqualiser(ctx: CodeGeneratorContext, fieldType: DataType): String = { + val equaliserGenerator = + new EqualiserCodeGenerator(getFieldTypes(fieldType).asScala.toArray) + val generatedEqualiser = + equaliserGenerator.generateRecordEqualiser("fieldGeneratedEqualiser") + val generatedEqualiserTerm = + ctx.addReusableObject(generatedEqualiser, "fieldGeneratedEqualiser") + val equaliserTypeTerm = classOf[RecordEqualiser].getCanonicalName + val equaliserTerm = newName("equaliser") + ctx.addReusableMember(s"private $equaliserTypeTerm $equaliserTerm = null;") + ctx.addReusableInitStatement( + s""" + |$equaliserTerm = ($equaliserTypeTerm) + | $generatedEqualiserTerm.newInstance(this.getClass().getClassLoader()); + |""".stripMargin) + equaliserTerm + } + /** Generates comparison code for numeric types and comparable types of same type. */ def generateComparison( ctx: CodeGeneratorContext, diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FullCompactionFileStoreITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FullCompactionFileStoreITCase.java index e23dff160d76..a40587782b1f 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FullCompactionFileStoreITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FullCompactionFileStoreITCase.java @@ -164,4 +164,33 @@ public void testUpdateAuditLog() throws Exception { assertThat(sql("SELECT * FROM %s$audit_log", table)) .containsExactlyInAnyOrder(Row.ofKind(RowKind.INSERT, "+I", "1", "4", "5")); } + + @Test + public void testRowDeduplicateWithArrayRow() throws Exception { + String table = "T_ARRAY_ROW"; + tEnv.executeSql( + "CREATE TABLE IF NOT EXISTS " + + table + + "(" + + "ID INT PRIMARY KEY NOT ENFORCED,\n" + + "NAMES ARRAY>\n" + + ") WITH (" + + "'changelog-producer'='full-compaction'," + + "'changelog-producer.compaction-interval' = '1s'," + + "'changelog-producer.row-deduplicate' = 'true')"); + BlockingIterator iterator = + BlockingIterator.of(streamSqlIter("SELECT * FROM %s", table)); + + sql("INSERT INTO %s VALUES (1, ARRAY[('a','mark1')]);", table); + assertThat(iterator.collect(1).stream().map(Row::toString).collect(Collectors.toList())) + .containsExactlyInAnyOrder("+I[1, [+I[a, mark1]]]"); + + sql("INSERT INTO %s VALUES (1, ARRAY[('b', 'mark2')])", table); + assertThat(iterator.collect(2).stream().map(Row::toString).collect(Collectors.toList())) + .containsExactly("-U[1, [+I[a, mark1]]]", "+U[1, [+I[b, mark2]]]"); + + sql("INSERT INTO %s VALUES (1, ARRAY[('b', 'mark2')]), (2, ARRAY[('c', 'mark3')])", table); + assertThat(iterator.collect(1).stream().map(Row::toString).collect(Collectors.toList())) + .containsExactly("+I[2, [+I[c, mark3]]]"); + } } From 6e1b9891cda1080db8915209a810b6689dfd3f11 Mon Sep 17 00:00:00 2001 From: Xiaojian Sun Date: Mon, 18 Mar 2024 09:56:12 +0800 Subject: [PATCH 09/79] [core] Unified Naming of JdbcCatalog Fields (#3025) --- .../org/apache/paimon/jdbc/JdbcCatalogLock.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java index 85f15f7f9d53..d135a6caf22b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java @@ -35,22 +35,22 @@ public class JdbcCatalogLock implements CatalogLock { private final JdbcClientPool connections; private final long checkMaxSleep; private final long acquireTimeout; - private final String catalogName; + private final String catalogKey; public JdbcCatalogLock( JdbcClientPool connections, - String catalogName, + String catalogKey, long checkMaxSleep, long acquireTimeout) { this.connections = connections; this.checkMaxSleep = checkMaxSleep; this.acquireTimeout = acquireTimeout; - this.catalogName = catalogName; + this.catalogKey = catalogKey; } @Override public T runWithLock(String database, String table, Callable callable) throws Exception { - String lockUniqueName = String.format("%s.%s.%s", catalogName, database, table); + String lockUniqueName = String.format("%s.%s.%s", catalogKey, database, table); lock(lockUniqueName); try { return callable.call(); @@ -102,7 +102,7 @@ public CatalogLock create(LockContext context) { JdbcLockContext lockContext = (JdbcLockContext) context; return new JdbcCatalogLock( lockContext.connections, - lockContext.catalogName, + lockContext.catalogKey, checkMaxSleep(lockContext.conf), acquireTimeout(lockContext.conf)); } @@ -110,13 +110,13 @@ public CatalogLock create(LockContext context) { static class JdbcLockContext implements LockContext { private final JdbcClientPool connections; - private final String catalogName; + private final String catalogKey; private final Map conf; public JdbcLockContext( - JdbcClientPool connections, String catalogName, Map conf) { + JdbcClientPool connections, String catalogKey, Map conf) { this.connections = connections; - this.catalogName = catalogName; + this.catalogKey = catalogKey; this.conf = conf; } } From f2c605ecceffa58aeb70d65fc0374e7f7f8a07de Mon Sep 17 00:00:00 2001 From: Kerwin <37063904+zhuangchong@users.noreply.github.com> Date: Mon, 18 Mar 2024 09:57:22 +0800 Subject: [PATCH 10/79] [hotfix] upgrade json-path and avro version (#2987) --- paimon-flink/paimon-flink-cdc/pom.xml | 2 +- paimon-format/src/main/resources/META-INF/NOTICE | 6 +++--- pom.xml | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/pom.xml b/paimon-flink/paimon-flink-cdc/pom.xml index c7f4dcbc96f5..1b18833a466d 100644 --- a/paimon-flink/paimon-flink-cdc/pom.xml +++ b/paimon-flink/paimon-flink-cdc/pom.xml @@ -39,7 +39,7 @@ under the License. 2.4.1 1.11.1 2.2.0 - 2.8.0 + 2.9.0 1.19.1 4.0.0-1.17 diff --git a/paimon-format/src/main/resources/META-INF/NOTICE b/paimon-format/src/main/resources/META-INF/NOTICE index ecf4f31dfae4..f6e607ef3502 100644 --- a/paimon-format/src/main/resources/META-INF/NOTICE +++ b/paimon-format/src/main/resources/META-INF/NOTICE @@ -13,9 +13,9 @@ This project bundles the following dependencies under the Apache Software Licens - commons-lang:commons-lang:2.6 - org.apache.commons:commons-lang3:3.12.0 -- org.apache.avro:avro:1.11.1 -- com.fasterxml.jackson.core:jackson-core:2.12.7 -- com.fasterxml.jackson.core:jackson-databind:2.12.7 +- org.apache.avro:avro:1.11.3 +- com.fasterxml.jackson.core:jackson-core:2.14.2 +- com.fasterxml.jackson.core:jackson-databind:2.14.2 - com.fasterxml.jackson.core:jackson-annotations:2.14.2 - org.apache.commons:commons-compress:1.4.1 diff --git a/pom.xml b/pom.xml index dc4f354578c7..238398e20b4e 100644 --- a/pom.xml +++ b/pom.xml @@ -110,7 +110,7 @@ under the License. 2.3.1 1.3.9 2.4.9 - 1.11.1 + 1.11.3 3.2.3 3.2.2 2.1.0 From 17268f0608f0de62764cd171d32da09765b1b31e Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Mon, 18 Mar 2024 10:00:16 +0800 Subject: [PATCH 11/79] [core] Dv table supports value filter pushdown (#3024) --- .../org/apache/paimon/KeyValueFileStore.java | 3 +- .../operation/KeyValueFileStoreScan.java | 27 ++++++-- .../table/PrimaryKeyFileStoreTableTest.java | 64 +++++++++++++++++++ 3 files changed, 87 insertions(+), 7 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java index e9b4cc5fd00e..a44c91ba54c5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java @@ -228,7 +228,8 @@ public void pushdown(Predicate keyFilter) { options.bucket(), forWrite, options.scanManifestParallelism(), - branchName); + branchName, + options.deletionVectorsEnabled()); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java index 90d2c1b184e4..8e37d1e4fac1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java @@ -43,6 +43,7 @@ public class KeyValueFileStoreScan extends AbstractFileStoreScan { private Predicate keyFilter; private Predicate valueFilter; + private final boolean deletionVectorsEnabled; public KeyValueFileStoreScan( RowType partitionType, @@ -56,7 +57,8 @@ public KeyValueFileStoreScan( int numOfBuckets, boolean checkNumOfBuckets, Integer scanManifestParallelism, - String branchName) { + String branchName, + boolean deletionVectorsEnabled) { super( partitionType, bucketFilter, @@ -74,6 +76,7 @@ public KeyValueFileStoreScan( this.fieldValueStatsConverters = new FieldStatsConverters( sid -> keyValueFieldsExtractor.valueFields(scanTableSchema(sid)), schemaId); + this.deletionVectorsEnabled = deletionVectorsEnabled; } public KeyValueFileStoreScan withKeyFilter(Predicate predicate) { @@ -90,14 +93,26 @@ public KeyValueFileStoreScan withValueFilter(Predicate predicate) { /** Note: Keep this thread-safe. */ @Override protected boolean filterByStats(ManifestEntry entry) { - if (keyFilter == null) { + Predicate filter = null; + FieldStatsArraySerializer serializer = null; + BinaryTableStats stats = null; + if (deletionVectorsEnabled && entry.level() > 0 && valueFilter != null) { + filter = valueFilter; + serializer = fieldValueStatsConverters.getOrCreate(entry.file().schemaId()); + stats = entry.file().valueStats(); + } + + if (filter == null && keyFilter != null) { + filter = keyFilter; + serializer = fieldKeyStatsConverters.getOrCreate(entry.file().schemaId()); + stats = entry.file().keyStats(); + } + + if (filter == null) { return true; } - FieldStatsArraySerializer serializer = - fieldKeyStatsConverters.getOrCreate(entry.file().schemaId()); - BinaryTableStats stats = entry.file().keyStats(); - return keyFilter.test( + return filter.test( entry.file().rowCount(), serializer.evolution(stats.minValues()), serializer.evolution(stats.maxValues()), diff --git a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java index 68adf63fb04d..c245a2ce7220 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java @@ -80,6 +80,7 @@ import static org.apache.paimon.CoreOptions.BUCKET; import static org.apache.paimon.CoreOptions.CHANGELOG_PRODUCER; import static org.apache.paimon.CoreOptions.ChangelogProducer.LOOKUP; +import static org.apache.paimon.CoreOptions.DELETION_VECTORS_ENABLED; import static org.apache.paimon.CoreOptions.TARGET_FILE_SIZE; import static org.apache.paimon.Snapshot.CommitKind.COMPACT; import static org.apache.paimon.data.DataFormatTestUtil.internalRowToString; @@ -1257,6 +1258,69 @@ public void testReadOptimizedTable() throws Exception { commit.close(); } + @Test + public void testReadDeletionVectorTable() throws Exception { + FileStoreTable table = + createFileStoreTable( + options -> { + // let level has many files + options.set(TARGET_FILE_SIZE, new MemorySize(1)); + options.set(DELETION_VECTORS_ENABLED, true); + }); + StreamTableWrite write = table.newWrite(commitUser); + IOManager ioManager = IOManager.create(tablePath.toString()); + write.withIOManager(ioManager); + StreamTableCommit commit = table.newCommit(commitUser); + + write.write(rowDataWithKind(RowKind.INSERT, 1, 10, 100L)); + write.write(rowDataWithKind(RowKind.INSERT, 2, 20, 100L)); + commit.commit(0, write.prepareCommit(true, 0)); + write.write(rowDataWithKind(RowKind.INSERT, 1, 20, 200L)); + commit.commit(1, write.prepareCommit(true, 1)); + write.write(rowDataWithKind(RowKind.INSERT, 1, 10, 110L)); + commit.commit(2, write.prepareCommit(true, 2)); + + // test result + Function rowDataToString = + row -> + internalRowToString( + row, + DataTypes.ROW( + DataTypes.INT(), DataTypes.INT(), DataTypes.BIGINT())); + List result = + getResult(table.newRead(), table.newScan().plan().splits(), rowDataToString); + assertThat(result) + .containsExactlyInAnyOrder("+I[1, 10, 110]", "+I[1, 20, 200]", "+I[2, 20, 100]"); + + // file layout + // pt 1 + // level 4 (1, 10, 110L) + // level 5 (1, 10, 100L), (1, 20, 200L) + // pt 2 + // level 5 (2, 20, 100L) + + // test filter on dv table + // with key filter pt = 1 + PredicateBuilder builder = new PredicateBuilder(ROW_TYPE); + Predicate filter = builder.equal(0, 1); + int files = + table.newScan().withFilter(filter).plan().splits().stream() + .mapToInt(split -> ((DataSplit) split).dataFiles().size()) + .sum(); + assertThat(files).isEqualTo(3); + + // with key filter pt = 1 and value filter idx2 = 110L + filter = and(filter, builder.equal(2, 110L)); + files = + table.newScan().withFilter(filter).plan().splits().stream() + .mapToInt(split -> ((DataSplit) split).dataFiles().size()) + .sum(); + assertThat(files).isEqualTo(1); + + write.close(); + commit.close(); + } + @Test public void testTableQueryForLookup() throws Exception { FileStoreTable table = From 48c239de8f00f54d0908a400954f3e870fde2bfc Mon Sep 17 00:00:00 2001 From: HZY <48040205+Alibaba-HZY@users.noreply.github.com> Date: Mon, 18 Mar 2024 10:08:48 +0800 Subject: [PATCH 12/79] [hive] Fix hive writer in tez-mr (#2954) --- .../apache/paimon/hive/HiveWriteITCase.java | 21 +++ .../hive/mapred/PaimonOutputCommitter.java | 15 +- .../hive/mapred/PaimonOutputFormat.java | 3 +- .../apache/paimon/hive/mapred/TezUtil.java | 136 ++++++++++++++++++ 4 files changed, 169 insertions(+), 6 deletions(-) create mode 100644 paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/TezUtil.java diff --git a/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/HiveWriteITCase.java b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/HiveWriteITCase.java index 4751b075ac30..7513621a855b 100644 --- a/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/HiveWriteITCase.java +++ b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/HiveWriteITCase.java @@ -170,6 +170,27 @@ public void testInsert() throws Exception { assertThat(select).isEqualTo(Arrays.asList("1\t2\t3\tHello", "4\t5\t6\tFine")); } + @Test + public void testHiveCreateAndHiveWrite() throws Exception { + List emptyData = Collections.emptyList(); + + hiveShell.execute( + "CREATE TABLE paimon_table (\n" + + " `a` STRING comment '',\n" + + " `b` STRING comment '',\n" + + " `c` STRING comment ''\n" + + ") \n" + + "STORED BY 'org.apache.paimon.hive.PaimonStorageHandler'\n" + + "TBLPROPERTIES (\n" + + " 'primary-key' = 'a',\n" + + " 'bucket' = '1',\n" + + " 'bucket_key' = 'a'\n" + + ");"); + hiveShell.execute("insert into paimon_table values (2,3,'Hello'),(5,6,'Fine')"); + List select = hiveShell.executeQuery("select * from paimon_table"); + assertThat(select).containsExactly("2\t3\tHello", "5\t6\tFine"); + } + @Test public void testInsertTimestampAndDate() throws Exception { List emptyData = Collections.emptyList(); diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/PaimonOutputCommitter.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/PaimonOutputCommitter.java index 122c3f766d05..94bc4a675ae8 100644 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/PaimonOutputCommitter.java +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/PaimonOutputCommitter.java @@ -71,8 +71,9 @@ public boolean needsTaskCommit(TaskAttemptContext taskAttemptContext) throws IOE } @Override - public void commitTask(TaskAttemptContext taskAttemptContext) throws IOException { - + public void commitTask(TaskAttemptContext originalContext) throws IOException { + TaskAttemptContext taskAttemptContext = + TezUtil.enrichContextWithAttemptWrapper(originalContext); TaskAttemptID attemptID = taskAttemptContext.getTaskAttemptID(); JobConf jobConf = taskAttemptContext.getJobConf(); FileStoreTable table = createFileStoreTable(jobConf); @@ -117,7 +118,9 @@ public void commitTask(TaskAttemptContext taskAttemptContext) throws IOException } @Override - public void abortTask(TaskAttemptContext taskAttemptContext) throws IOException { + public void abortTask(TaskAttemptContext originalContext) throws IOException { + TaskAttemptContext taskAttemptContext = + TezUtil.enrichContextWithAttemptWrapper(originalContext); Map writers = PaimonRecordWriter.removeWriters(taskAttemptContext.getTaskAttemptID()); @@ -130,7 +133,8 @@ public void abortTask(TaskAttemptContext taskAttemptContext) throws IOException } @Override - public void commitJob(JobContext jobContext) throws IOException { + public void commitJob(JobContext originalContext) throws IOException { + JobContext jobContext = TezUtil.enrichContextWithVertexId(originalContext); JobConf jobConf = jobContext.getJobConf(); long startTime = System.currentTimeMillis(); @@ -161,7 +165,8 @@ public void commitJob(JobContext jobContext) throws IOException { } @Override - public void abortJob(JobContext jobContext, int status) throws IOException { + public void abortJob(JobContext originalContext, int status) throws IOException { + JobContext jobContext = TezUtil.enrichContextWithVertexId(originalContext); FileStoreTable table = createFileStoreTable(jobContext.getJobConf()); if (table != null) { diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/PaimonOutputFormat.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/PaimonOutputFormat.java index ccfd7bcdfcd6..ef1ee687ecb0 100644 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/PaimonOutputFormat.java +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/PaimonOutputFormat.java @@ -73,7 +73,8 @@ public FileSinkOperator.RecordWriter getHiveRecordWriter( } private static PaimonRecordWriter writer(JobConf jobConf) { - TaskAttemptID taskAttemptID = TaskAttemptID.forName(jobConf.get(TASK_ATTEMPT_ID_KEY)); + TaskAttemptID taskAttemptID = TezUtil.taskAttemptWrapper(jobConf); + FileStoreTable table = createFileStoreTable(jobConf); // force write-only = true Map newOptions = diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/TezUtil.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/TezUtil.java new file mode 100644 index 000000000000..d4d7c7cd9c71 --- /dev/null +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/TezUtil.java @@ -0,0 +1,136 @@ +/* + * 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.hive.mapred; + +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.JobContext; +import org.apache.hadoop.mapred.JobContextImpl; +import org.apache.hadoop.mapred.TaskAttemptContext; +import org.apache.hadoop.mapred.TaskAttemptContextImpl; +import org.apache.hadoop.mapred.TaskAttemptID; +import org.apache.hadoop.mapreduce.JobID; + +import java.util.Objects; + +/** + * Utility class to enrich the JobContext and TaskAttemptContext with the vertex id. copied form + * iceberg. + */ +public class TezUtil { + + private static final String TASK_ATTEMPT_ID_KEY = "mapred.task.id"; + // TezProcessor (Hive) propagates the vertex id under this key - available during Task commit + // phase + private static final String TEZ_VERTEX_ID_HIVE = "hive.tez.vertex.index"; + // MROutputCommitter (Tez) propagates the vertex id under this key - available during DAG/Vertex + // commit phase + private static final String TEZ_VERTEX_ID_DAG = "mapreduce.task.vertex.id"; + + /** + * If the Tez vertex id is present in config, creates a new jobContext by appending the Tez + * vertex id to the jobID. For the rationale behind this enrichment, please refer to point #1 in + * the docs of {@link TaskAttemptWrapper}. + * + * @param jobContext original jobContext to be enriched + * @return enriched jobContext + */ + public static JobContext enrichContextWithVertexId(JobContext jobContext) { + String vertexId = jobContext.getJobConf().get(TEZ_VERTEX_ID_DAG); + if (vertexId != null) { + JobID jobID = getJobIDWithVertexAppended(jobContext.getJobID(), vertexId); + return new JobContextImpl(jobContext.getJobConf(), jobID, jobContext.getProgressible()); + } else { + return jobContext; + } + } + + /** + * Creates a new taskAttemptContext by replacing the taskAttemptID with a wrapped object. For + * the rationale behind this enrichment, please refer to point #2 in the docs of {@link + * TaskAttemptWrapper}. + * + * @param taskAttemptContext original taskAttemptContext to be enriched + * @return enriched taskAttemptContext + */ + public static TaskAttemptContext enrichContextWithAttemptWrapper( + TaskAttemptContext taskAttemptContext) { + TaskAttemptID wrapped = TezUtil.taskAttemptWrapper(taskAttemptContext.getTaskAttemptID()); + return new TaskAttemptContextImpl(taskAttemptContext.getJobConf(), wrapped); + } + + public static TaskAttemptID taskAttemptWrapper(TaskAttemptID attemptID) { + return new TaskAttemptWrapper(attemptID, ""); + } + + public static TaskAttemptID taskAttemptWrapper(JobConf jc) { + return new TaskAttemptWrapper( + TaskAttemptID.forName(jc.get(TASK_ATTEMPT_ID_KEY)), jc.get(TEZ_VERTEX_ID_HIVE)); + } + + private static JobID getJobIDWithVertexAppended(JobID jobID, String vertexId) { + if (vertexId != null && !vertexId.isEmpty()) { + return new JobID(jobID.getJtIdentifier() + vertexId, jobID.getId()); + } else { + return jobID; + } + } + + private TezUtil() {} + + /** + * Subclasses {@link TaskAttemptID}. It has two main purposes: 1. Provide a way to append an + * optional vertex id to the Job ID. This is needed because there is a discrepancy between how + * the attempt ID is constructed in the {@link org.apache.tez.mapreduce.output.MROutput} (with + * vertex ID appended to the end of the Job ID) and how it's available in the mapper (without + * vertex ID) which creates and caches the HiveIcebergRecordWriter object. 2. Redefine the + * equals/hashcode provided by TaskAttemptID so that task type (map or reduce) does not count, + * and therefore the mapper and reducer threads can use the same attempt ID-based key to + * retrieve the cached HiveIcebergRecordWriter object. + */ + private static class TaskAttemptWrapper extends TaskAttemptID { + + TaskAttemptWrapper(TaskAttemptID attemptID, String vertexId) { + super( + getJobIDWithVertexAppended(attemptID.getJobID(), vertexId).getJtIdentifier(), + attemptID.getJobID().getId(), + attemptID.getTaskType(), + attemptID.getTaskID().getId(), + attemptID.getId()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TaskAttemptWrapper that = (TaskAttemptWrapper) o; + return getId() == that.getId() + && getTaskID().getId() == that.getTaskID().getId() + && Objects.equals(getJobID(), that.getJobID()); + } + + @Override + public int hashCode() { + return Objects.hash(getId(), getTaskID().getId(), getJobID()); + } + } +} From f33f932e52bdc01230b2a1e5aea5c3cfbb381224 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Mon, 18 Mar 2024 10:12:41 +0800 Subject: [PATCH 13/79] [hive] Add license for TezUtil --- LICENSE | 1 + .../java/org/apache/paimon/hive/mapred/TezUtil.java | 11 ++++++----- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/LICENSE b/LICENSE index 97c849402f1d..eb317b6613a4 100644 --- a/LICENSE +++ b/LICENSE @@ -242,6 +242,7 @@ paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java paimon-core/src/main/java/org/apache/paimon/utils/ZOrderByteUtils.java paimon-core/src/test/java/org/apache/paimon/utils/TestZOrderByteUtil.java paimon-hive/paimon-hive-common/src/test/java/org/apache/paimon/hive/TestHiveMetastore.java +paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/TezUtil.java paimon-spark/paimon-spark-common/src/main/antlr4/org.apache.spark.sql.catalyst.parser.extensions/PaimonSqlExtensions.g4 paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/analysis/CoerceArguments.scala diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/TezUtil.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/TezUtil.java index d4d7c7cd9c71..99a52798041a 100644 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/TezUtil.java +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/TezUtil.java @@ -28,10 +28,11 @@ import java.util.Objects; -/** - * Utility class to enrich the JobContext and TaskAttemptContext with the vertex id. copied form - * iceberg. - */ +/* This file is based on source code from the Iceberg Project (http://iceberg.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** Utility class to enrich the JobContext and TaskAttemptContext with the vertex id. */ public class TezUtil { private static final String TASK_ATTEMPT_ID_KEY = "mapred.task.id"; @@ -96,7 +97,7 @@ private TezUtil() {} /** * Subclasses {@link TaskAttemptID}. It has two main purposes: 1. Provide a way to append an * optional vertex id to the Job ID. This is needed because there is a discrepancy between how - * the attempt ID is constructed in the {@link org.apache.tez.mapreduce.output.MROutput} (with + * the attempt ID is constructed in the {@code org.apache.tez.mapreduce.output.MROutput} (with * vertex ID appended to the end of the Job ID) and how it's available in the mapper (without * vertex ID) which creates and caches the HiveIcebergRecordWriter object. 2. Redefine the * equals/hashcode provided by TaskAttemptID so that task type (map or reduce) does not count, From abaa59f460268bc35151170fd48381fb0ddc54ae Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Mon, 18 Mar 2024 10:17:31 +0800 Subject: [PATCH 14/79] [core] Reduce schema read for readers (#3021) --- .../org/apache/paimon/AbstractFileStore.java | 7 +++-- .../apache/paimon/AppendOnlyFileStore.java | 11 +++---- .../org/apache/paimon/KeyValueFileStore.java | 14 ++++----- .../paimon/io/KeyValueFileReaderFactory.java | 25 ++++++++-------- .../operation/AbstractFileStoreScan.java | 6 +++- .../operation/AppendOnlyFileStoreRead.java | 15 ++++++---- .../operation/AppendOnlyFileStoreScan.java | 6 ++-- .../operation/KeyValueFileStoreRead.java | 6 ++-- .../operation/KeyValueFileStoreScan.java | 10 +++++-- .../operation/KeyValueFileStoreWrite.java | 7 +++-- .../table/AppendOnlyFileStoreTable.java | 2 +- .../table/PrimaryKeyFileStoreTable.java | 2 +- .../java/org/apache/paimon/TestFileStore.java | 29 +++++++++++++++---- .../paimon/io/KeyValueFileReadWriteTest.java | 2 +- .../paimon/mergetree/ContainsLevelsTest.java | 2 +- .../paimon/mergetree/LookupLevelsTest.java | 2 +- .../paimon/mergetree/MergeTreeTestBase.java | 2 +- .../paimon/operation/FileDeletionTest.java | 19 +++++++----- .../paimon/operation/FileStoreCommitTest.java | 23 ++++++++------- .../operation/FileStoreExpireTestBase.java | 3 +- .../operation/KeyValueFileStoreReadTest.java | 8 +++-- .../operation/KeyValueFileStoreScanTest.java | 3 +- .../source/TestChangelogDataReadWrite.java | 11 ++++--- 23 files changed, 129 insertions(+), 86 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java index 6413cd088ede..87cc4e65c544 100644 --- a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java @@ -34,6 +34,7 @@ import org.apache.paimon.operation.TagDeletion; import org.apache.paimon.options.MemorySize; import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; import org.apache.paimon.service.ServiceManager; import org.apache.paimon.stats.StatsFile; import org.apache.paimon.stats.StatsFileHandler; @@ -65,7 +66,7 @@ public abstract class AbstractFileStore implements FileStore { protected final FileIO fileIO; protected final SchemaManager schemaManager; - protected final long schemaId; + protected final TableSchema schema; protected final CoreOptions options; protected final RowType partitionType; private final CatalogEnvironment catalogEnvironment; @@ -75,13 +76,13 @@ public abstract class AbstractFileStore implements FileStore { public AbstractFileStore( FileIO fileIO, SchemaManager schemaManager, - long schemaId, + TableSchema schema, CoreOptions options, RowType partitionType, CatalogEnvironment catalogEnvironment) { this.fileIO = fileIO; this.schemaManager = schemaManager; - this.schemaId = schemaId; + this.schema = schema; this.options = options; this.partitionType = partitionType; this.catalogEnvironment = catalogEnvironment; diff --git a/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java b/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java index 8be8f817841d..0d546e2154d5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java @@ -28,6 +28,7 @@ import org.apache.paimon.operation.ScanBucketFilter; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.CatalogEnvironment; import org.apache.paimon.types.RowType; @@ -50,14 +51,14 @@ public class AppendOnlyFileStore extends AbstractFileStore { public AppendOnlyFileStore( FileIO fileIO, SchemaManager schemaManager, - long schemaId, + TableSchema schema, CoreOptions options, RowType partitionType, RowType bucketKeyType, RowType rowType, String tableName, CatalogEnvironment catalogEnvironment) { - super(fileIO, schemaManager, schemaId, options, partitionType, catalogEnvironment); + super(fileIO, schemaManager, schema, options, partitionType, catalogEnvironment); this.bucketKeyType = bucketKeyType; this.rowType = rowType; this.tableName = tableName; @@ -82,7 +83,7 @@ public AppendOnlyFileStoreRead newRead() { return new AppendOnlyFileStoreRead( fileIO, schemaManager, - schemaId, + schema, rowType, FileFormatDiscover.of(options), pathFactory()); @@ -99,7 +100,7 @@ public AppendOnlyFileStoreWrite newWrite( return new AppendOnlyFileStoreWrite( fileIO, newRead(), - schemaId, + schema.id(), commitUser, rowType, pathFactory(), @@ -138,7 +139,7 @@ public void pushdown(Predicate predicate) { bucketFilter, snapshotManager(), schemaManager, - schemaId, + schema, manifestFileFactory(forWrite), manifestListFactory(forWrite), options.bucket(), diff --git a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java index a44c91ba54c5..d80bd6d3979d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java @@ -35,6 +35,7 @@ import org.apache.paimon.predicate.Predicate; import org.apache.paimon.schema.KeyValueFieldsExtractor; import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.CatalogEnvironment; import org.apache.paimon.types.RowType; @@ -75,7 +76,7 @@ public class KeyValueFileStore extends AbstractFileStore { public KeyValueFileStore( FileIO fileIO, SchemaManager schemaManager, - long schemaId, + TableSchema schema, boolean crossPartitionUpdate, CoreOptions options, RowType partitionType, @@ -86,7 +87,7 @@ public KeyValueFileStore( MergeFunctionFactory mfFactory, String tableName, CatalogEnvironment catalogEnvironment) { - super(fileIO, schemaManager, schemaId, options, partitionType, catalogEnvironment); + super(fileIO, schemaManager, schema, options, partitionType, catalogEnvironment); this.crossPartitionUpdate = crossPartitionUpdate; this.bucketKeyType = bucketKeyType; this.keyType = keyType; @@ -121,8 +122,7 @@ public KeyValueFileStoreScan newScan(String branchName) { public KeyValueFileStoreRead newRead() { return new KeyValueFileStoreRead( options, - schemaManager, - schemaId, + schema, keyType, valueType, newKeyComparator(), @@ -134,7 +134,7 @@ public KeyValueFileReaderFactory.Builder newReaderFactoryBuilder() { return KeyValueFileReaderFactory.builder( fileIO, schemaManager, - schemaId, + schema, keyType, valueType, FileFormatDiscover.of(options), @@ -162,7 +162,7 @@ public KeyValueFileStoreWrite newWrite(String commitUser, ManifestCacheFilter ma return new KeyValueFileStoreWrite( fileIO, schemaManager, - schemaId, + schema, commitUser, keyType, valueType, @@ -221,7 +221,7 @@ public void pushdown(Predicate keyFilter) { bucketFilter, snapshotManager(), schemaManager, - schemaId, + schema, keyValueFieldsExtractor, manifestFileFactory(forWrite), manifestListFactory(forWrite), diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java index 63fef31fc142..cc7534e9aed6 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java @@ -31,6 +31,7 @@ import org.apache.paimon.reader.RecordReader; import org.apache.paimon.schema.KeyValueFieldsExtractor; import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.AsyncRecordReader; import org.apache.paimon.utils.BulkFormatMapping; @@ -52,7 +53,7 @@ public class KeyValueFileReaderFactory { private final FileIO fileIO; private final SchemaManager schemaManager; - private final long schemaId; + private final TableSchema schema; private final RowType keyType; private final RowType valueType; @@ -67,7 +68,7 @@ public class KeyValueFileReaderFactory { private KeyValueFileReaderFactory( FileIO fileIO, SchemaManager schemaManager, - long schemaId, + TableSchema schema, RowType keyType, RowType valueType, BulkFormatMapping.BulkFormatMappingBuilder bulkFormatMappingBuilder, @@ -77,7 +78,7 @@ private KeyValueFileReaderFactory( DeletionVector.Factory dvFactory) { this.fileIO = fileIO; this.schemaManager = schemaManager; - this.schemaId = schemaId; + this.schema = schema; this.keyType = keyType; this.valueType = valueType; this.bulkFormatMappingBuilder = bulkFormatMappingBuilder; @@ -110,8 +111,8 @@ private RecordReader createRecordReader( () -> bulkFormatMappingBuilder.build( formatIdentifier, - schemaManager.schema(this.schemaId), - schemaManager.schema(schemaId)); + schema, + schemaId == schema.id() ? schema : schemaManager.schema(schemaId)); BulkFormatMapping bulkFormatMapping = reuseFormat @@ -141,7 +142,7 @@ private RecordReader createRecordReader( public static Builder builder( FileIO fileIO, SchemaManager schemaManager, - long schemaId, + TableSchema schema, RowType keyType, RowType valueType, FileFormatDiscover formatDiscover, @@ -151,7 +152,7 @@ public static Builder builder( return new Builder( fileIO, schemaManager, - schemaId, + schema, keyType, valueType, formatDiscover, @@ -165,7 +166,7 @@ public static class Builder { private final FileIO fileIO; private final SchemaManager schemaManager; - private final long schemaId; + private final TableSchema schema; private final RowType keyType; private final RowType valueType; private final FileFormatDiscover formatDiscover; @@ -182,7 +183,7 @@ public static class Builder { private Builder( FileIO fileIO, SchemaManager schemaManager, - long schemaId, + TableSchema schema, RowType keyType, RowType valueType, FileFormatDiscover formatDiscover, @@ -191,7 +192,7 @@ private Builder( CoreOptions options) { this.fileIO = fileIO; this.schemaManager = schemaManager; - this.schemaId = schemaId; + this.schema = schema; this.keyType = keyType; this.valueType = valueType; this.formatDiscover = formatDiscover; @@ -209,7 +210,7 @@ public Builder copyWithoutProjection() { return new Builder( fileIO, schemaManager, - schemaId, + schema, keyType, valueType, formatDiscover, @@ -255,7 +256,7 @@ public KeyValueFileReaderFactory build( return new KeyValueFileReaderFactory( fileIO, schemaManager, - schemaId, + schema, projectedKeyType, projectedValueType, BulkFormatMapping.newBuilder( diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java index e6c95e8854b5..52983f4b6e2b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java @@ -72,6 +72,7 @@ public abstract class AbstractFileStoreScan implements FileStoreScan { private final ConcurrentMap tableSchemas; private final SchemaManager schemaManager; + private final TableSchema schema; protected final ScanBucketFilter bucketKeyFilter; private final String branchName; @@ -91,6 +92,7 @@ public AbstractFileStoreScan( ScanBucketFilter bucketKeyFilter, SnapshotManager snapshotManager, SchemaManager schemaManager, + TableSchema schema, ManifestFile.Factory manifestFileFactory, ManifestList.Factory manifestListFactory, int numOfBuckets, @@ -101,6 +103,7 @@ public AbstractFileStoreScan( this.bucketKeyFilter = bucketKeyFilter; this.snapshotManager = snapshotManager; this.schemaManager = schemaManager; + this.schema = schema; this.manifestFileFactory = manifestFileFactory; this.manifestList = manifestListFactory.create(); this.numOfBuckets = numOfBuckets; @@ -407,7 +410,8 @@ private List readManifests(Snapshot snapshot) { /** Note: Keep this thread-safe. */ protected TableSchema scanTableSchema(long id) { - return tableSchemas.computeIfAbsent(id, key -> schemaManager.schema(id)); + return tableSchemas.computeIfAbsent( + id, key -> key == schema.id() ? schema : schemaManager.schema(id)); } /** Note: Keep this thread-safe. */ diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java index c06cce45899a..8363c297ad7b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java @@ -62,7 +62,7 @@ public class AppendOnlyFileStoreRead implements FileStoreRead { private final FileIO fileIO; private final SchemaManager schemaManager; - private final long schemaId; + private final TableSchema schema; private final FileFormatDiscover formatDiscover; private final FileStorePathFactory pathFactory; private final Map bulkFormatMappings; @@ -74,13 +74,13 @@ public class AppendOnlyFileStoreRead implements FileStoreRead { public AppendOnlyFileStoreRead( FileIO fileIO, SchemaManager schemaManager, - long schemaId, + TableSchema schema, RowType rowType, FileFormatDiscover formatDiscover, FileStorePathFactory pathFactory) { this.fileIO = fileIO; this.schemaManager = schemaManager; - this.schemaId = schemaId; + this.schema = schema; this.formatDiscover = formatDiscover; this.pathFactory = pathFactory; this.bulkFormatMappings = new HashMap<>(); @@ -113,8 +113,11 @@ public RecordReader createReader(DataSplit split) throws IOExceptio bulkFormatMappings.computeIfAbsent( new FormatKey(file.schemaId(), formatIdentifier), key -> { - TableSchema tableSchema = schemaManager.schema(this.schemaId); - TableSchema dataSchema = schemaManager.schema(key.schemaId); + TableSchema tableSchema = schema; + TableSchema dataSchema = + key.schemaId == schema.id() + ? schema + : schemaManager.schema(key.schemaId); // projection to data schema int[][] dataProjection = @@ -131,7 +134,7 @@ public RecordReader createReader(DataSplit split) throws IOExceptio dataSchema.fields()); List dataFilters = - this.schemaId == key.schemaId + this.schema.id() == key.schemaId ? filters : SchemaEvolutionUtil.createDataFilters( tableSchema.fields(), diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java index 2cec4e064975..866c87d75f66 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java @@ -24,6 +24,7 @@ import org.apache.paimon.manifest.ManifestList; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; import org.apache.paimon.stats.BinaryTableStats; import org.apache.paimon.stats.FieldStatsArraySerializer; import org.apache.paimon.stats.FieldStatsConverters; @@ -44,7 +45,7 @@ public AppendOnlyFileStoreScan( ScanBucketFilter bucketFilter, SnapshotManager snapshotManager, SchemaManager schemaManager, - long schemaId, + TableSchema schema, ManifestFile.Factory manifestFileFactory, ManifestList.Factory manifestListFactory, int numOfBuckets, @@ -56,6 +57,7 @@ public AppendOnlyFileStoreScan( bucketFilter, snapshotManager, schemaManager, + schema, manifestFileFactory, manifestListFactory, numOfBuckets, @@ -63,7 +65,7 @@ public AppendOnlyFileStoreScan( scanManifestParallelism, branchName); this.fieldStatsConverters = - new FieldStatsConverters(sid -> scanTableSchema(sid).fields(), schemaId); + new FieldStatsConverters(sid -> scanTableSchema(sid).fields(), schema.id()); } public AppendOnlyFileStoreScan withFilter(Predicate predicate) { diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java index db4c0ac926ca..0e115fdddc87 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java @@ -41,7 +41,6 @@ import org.apache.paimon.mergetree.compact.ReducerMergeFunctionWrapper; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.reader.RecordReader; -import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.DeletionFile; @@ -89,14 +88,13 @@ public class KeyValueFileStoreRead implements FileStoreRead { public KeyValueFileStoreRead( CoreOptions options, - SchemaManager schemaManager, - long schemaId, + TableSchema schema, RowType keyType, RowType valueType, Comparator keyComparator, MergeFunctionFactory mfFactory, KeyValueFileReaderFactory.Builder readerFactoryBuilder) { - this.tableSchema = schemaManager.schema(schemaId); + this.tableSchema = schema; this.readerFactoryBuilder = readerFactoryBuilder; this.fileIO = readerFactoryBuilder.fileIO(); this.keyComparator = keyComparator; diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java index 8e37d1e4fac1..0f34cac5a138 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java @@ -25,6 +25,7 @@ import org.apache.paimon.predicate.Predicate; import org.apache.paimon.schema.KeyValueFieldsExtractor; import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; import org.apache.paimon.stats.BinaryTableStats; import org.apache.paimon.stats.FieldStatsArraySerializer; import org.apache.paimon.stats.FieldStatsConverters; @@ -50,7 +51,7 @@ public KeyValueFileStoreScan( ScanBucketFilter bucketFilter, SnapshotManager snapshotManager, SchemaManager schemaManager, - long schemaId, + TableSchema schema, KeyValueFieldsExtractor keyValueFieldsExtractor, ManifestFile.Factory manifestFileFactory, ManifestList.Factory manifestListFactory, @@ -64,6 +65,7 @@ public KeyValueFileStoreScan( bucketFilter, snapshotManager, schemaManager, + schema, manifestFileFactory, manifestListFactory, numOfBuckets, @@ -72,10 +74,12 @@ public KeyValueFileStoreScan( branchName); this.fieldKeyStatsConverters = new FieldStatsConverters( - sid -> keyValueFieldsExtractor.keyFields(scanTableSchema(sid)), schemaId); + sid -> keyValueFieldsExtractor.keyFields(scanTableSchema(sid)), + schema.id()); this.fieldValueStatsConverters = new FieldStatsConverters( - sid -> keyValueFieldsExtractor.valueFields(scanTableSchema(sid)), schemaId); + sid -> keyValueFieldsExtractor.valueFields(scanTableSchema(sid)), + schema.id()); this.deletionVectorsEnabled = deletionVectorsEnabled; } 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 41b77d612885..e06ed9ea26b6 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 @@ -60,6 +60,7 @@ import org.apache.paimon.options.Options; import org.apache.paimon.schema.KeyValueFieldsExtractor; import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.CommitIncrement; import org.apache.paimon.utils.FieldsComparator; @@ -102,7 +103,7 @@ public class KeyValueFileStoreWrite extends MemoryFileStoreWrite { public KeyValueFileStoreWrite( FileIO fileIO, SchemaManager schemaManager, - long schemaId, + TableSchema schema, String commitUser, RowType keyType, RowType valueType, @@ -135,7 +136,7 @@ public KeyValueFileStoreWrite( KeyValueFileReaderFactory.builder( fileIO, schemaManager, - schemaId, + schema, keyType, valueType, FileFormatDiscover.of(options), @@ -145,7 +146,7 @@ public KeyValueFileStoreWrite( this.writerFactoryBuilder = KeyValueFileWriterFactory.builder( fileIO, - schemaId, + schema.id(), keyType, valueType, options.fileFormat(), diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java index 9c97d406cb7e..4d91328f24c7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java @@ -77,7 +77,7 @@ public AppendOnlyFileStore store() { new AppendOnlyFileStore( fileIO, schemaManager(), - tableSchema.id(), + tableSchema, new CoreOptions(tableSchema.options()), tableSchema.logicalPartitionType(), tableSchema.logicalBucketKeyType(), diff --git a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java index 7b30fb832ef0..f35afc64d5c9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java @@ -98,7 +98,7 @@ public KeyValueFileStore store() { new KeyValueFileStore( fileIO(), schemaManager(), - tableSchema.id(), + tableSchema, tableSchema.crossPartitionUpdate(), options, tableSchema.logicalPartitionType(), diff --git a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java index d9cc7f87866a..43e0297fd921 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java @@ -44,6 +44,7 @@ import org.apache.paimon.reader.RecordReaderIterator; import org.apache.paimon.schema.KeyValueFieldsExtractor; import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.CatalogEnvironment; import org.apache.paimon.table.ExpireSnapshots; import org.apache.paimon.table.ExpireSnapshotsImpl; @@ -103,11 +104,21 @@ private TestFileStore( RowType keyType, RowType valueType, KeyValueFieldsExtractor keyValueFieldsExtractor, - MergeFunctionFactory mfFactory) { + MergeFunctionFactory mfFactory, + TableSchema tableSchema) { super( FileIOFinder.find(new Path(root)), - new SchemaManager(FileIOFinder.find(new Path(root)), options.path()), - 0L, + schemaManager(root, options), + tableSchema != null + ? tableSchema + : new TableSchema( + 0L, + valueType.getFields(), + valueType.getFieldCount(), + partitionType.getFieldNames(), + keyType.getFieldNames(), + Collections.emptyMap(), + null), false, options, partitionType, @@ -127,6 +138,10 @@ private TestFileStore( this.commitIdentifier = 0L; } + private static SchemaManager schemaManager(String root, CoreOptions options) { + return new SchemaManager(FileIOFinder.find(new Path(root)), options.path()); + } + public AbstractFileStoreWrite newWrite() { return super.newWrite(commitUser); } @@ -563,6 +578,7 @@ public static class Builder { private final RowType valueType; private final KeyValueFieldsExtractor keyValueFieldsExtractor; private final MergeFunctionFactory mfFactory; + private final TableSchema tableSchema; private CoreOptions.ChangelogProducer changelogProducer; @@ -574,7 +590,8 @@ public Builder( RowType keyType, RowType valueType, KeyValueFieldsExtractor keyValueFieldsExtractor, - MergeFunctionFactory mfFactory) { + MergeFunctionFactory mfFactory, + TableSchema tableSchema) { this.format = format; this.root = root; this.numBuckets = numBuckets; @@ -583,6 +600,7 @@ public Builder( this.valueType = valueType; this.keyValueFieldsExtractor = keyValueFieldsExtractor; this.mfFactory = mfFactory; + this.tableSchema = tableSchema; this.changelogProducer = CoreOptions.ChangelogProducer.NONE; } @@ -620,7 +638,8 @@ public TestFileStore build() { keyType, valueType, keyValueFieldsExtractor, - mfFactory); + mfFactory, + tableSchema); } } } diff --git a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java index 05f260097a4d..ca7f75d6eb57 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java @@ -282,7 +282,7 @@ private KeyValueFileReaderFactory createReaderFactory( KeyValueFileReaderFactory.builder( fileIO, createTestSchemaManager(path), - 0, + createTestSchemaManager(path).schema(0), KEY_TYPE, DEFAULT_ROW_TYPE, ignore -> new FlushingFileFormat(format), diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java index 58d9dbe904d9..d53b94155229 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java @@ -235,7 +235,7 @@ private KeyValueFileReaderFactory createReaderFactory() { KeyValueFileReaderFactory.builder( FileIOFinder.find(path), createSchemaManager(path), - 0, + createSchemaManager(path).schema(0), keyType, rowType, ignore -> new FlushingFileFormat("avro"), diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java index 00d8eeb5a8e4..f4d27784a56d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java @@ -315,7 +315,7 @@ private KeyValueFileReaderFactory createReaderFactory() { KeyValueFileReaderFactory.builder( FileIOFinder.find(path), createSchemaManager(path), - 0, + createSchemaManager(path).schema(0), keyType, rowType, ignore -> new FlushingFileFormat("avro"), 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 f6ddf74ea699..a3bcc9bfa32c 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 @@ -147,7 +147,7 @@ private void recreateMergeTree(long targetFileSize) { KeyValueFileReaderFactory.builder( LocalFileIO.create(), createTestingSchemaManager(path), - 0, + createTestingSchemaManager(path).schema(0), keyType, valueType, ignore -> flushingAvro, diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java index 49e55ee4e1f6..9994c0809a88 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java @@ -36,6 +36,7 @@ import org.apache.paimon.mergetree.compact.DeduplicateMergeFunction; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.FileStorePathFactory; import org.apache.paimon.utils.RecordWriter; @@ -664,13 +665,14 @@ private TestFileStore createStore(TestKeyValueGenerator.GeneratorMode mode, int } SchemaManager schemaManager = new SchemaManager(fileIO, new Path(root)); - schemaManager.createTable( - new Schema( - rowType.getFields(), - partitionType.getFieldNames(), - TestKeyValueGenerator.getPrimaryKeys(mode), - Collections.emptyMap(), - null)); + TableSchema tableSchema = + schemaManager.createTable( + new Schema( + rowType.getFields(), + partitionType.getFieldNames(), + TestKeyValueGenerator.getPrimaryKeys(mode), + Collections.emptyMap(), + null)); return new TestFileStore.Builder( "avro", @@ -680,7 +682,8 @@ private TestFileStore createStore(TestKeyValueGenerator.GeneratorMode mode, int TestKeyValueGenerator.KEY_TYPE, rowType, TestKeyValueGenerator.TestKeyValueFieldsExtractor.EXTRACTOR, - DeduplicateMergeFunction.factory()) + DeduplicateMergeFunction.factory(), + tableSchema) .changelogProducer(changelogProducer) .build(); } diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java index 488f74f67e64..b0cec3f445da 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java @@ -35,6 +35,7 @@ import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.SchemaUtils; +import org.apache.paimon.schema.TableSchema; import org.apache.paimon.stats.ColStats; import org.apache.paimon.stats.Statistics; import org.apache.paimon.stats.StatsFileHandler; @@ -856,15 +857,16 @@ private TestFileStore createStore( ? FailingFileIO.getFailingPath(failingName, tempDir.toString()) : TraceableFileIO.SCHEME + "://" + tempDir.toString(); Path path = new Path(tempDir.toUri()); - SchemaUtils.forceCommit( - new SchemaManager(new LocalFileIO(), path), - new Schema( - TestKeyValueGenerator.DEFAULT_ROW_TYPE.getFields(), - TestKeyValueGenerator.DEFAULT_PART_TYPE.getFieldNames(), - TestKeyValueGenerator.getPrimaryKeys( - TestKeyValueGenerator.GeneratorMode.MULTI_PARTITIONED), - Collections.emptyMap(), - null)); + TableSchema tableSchema = + SchemaUtils.forceCommit( + new SchemaManager(new LocalFileIO(), path), + new Schema( + TestKeyValueGenerator.DEFAULT_ROW_TYPE.getFields(), + TestKeyValueGenerator.DEFAULT_PART_TYPE.getFieldNames(), + TestKeyValueGenerator.getPrimaryKeys( + TestKeyValueGenerator.GeneratorMode.MULTI_PARTITIONED), + Collections.emptyMap(), + null)); return new TestFileStore.Builder( "avro", root, @@ -873,7 +875,8 @@ private TestFileStore createStore( TestKeyValueGenerator.KEY_TYPE, TestKeyValueGenerator.DEFAULT_ROW_TYPE, TestKeyValueGenerator.TestKeyValueFieldsExtractor.EXTRACTOR, - DeduplicateMergeFunction.factory()) + DeduplicateMergeFunction.factory(), + tableSchema) .changelogProducer(changelogProducer) .build(); } diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreExpireTestBase.java b/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreExpireTestBase.java index d9b3469f9a16..5ebe76ed608d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreExpireTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreExpireTestBase.java @@ -86,7 +86,8 @@ private TestFileStore createStore() { TestKeyValueGenerator.KEY_TYPE, TestKeyValueGenerator.DEFAULT_ROW_TYPE, TestKeyValueGenerator.TestKeyValueFieldsExtractor.EXTRACTOR, - DeduplicateMergeFunction.factory()) + DeduplicateMergeFunction.factory(), + null) .changelogProducer(changelogProducer) .build(); } diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreReadTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreReadTest.java index a3c0142e70a0..07bf705e2bba 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreReadTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreReadTest.java @@ -268,7 +268,7 @@ private TestFileStore createStore( Path path = new Path(tempDir.toUri()); SchemaManager schemaManager = new SchemaManager(FileIOFinder.find(path), path); boolean valueCountMode = mfFactory.create() instanceof TestValueCountMergeFunction; - schemaManager.createTable( + Schema schema = new Schema( (valueCountMode ? keyType : valueType).getFields(), partitionType.getFieldNames(), @@ -280,7 +280,8 @@ private TestFileStore createStore( partitionType.getFieldNames().stream()) .collect(Collectors.toList()), Collections.emptyMap(), - null)); + null); + TableSchema tableSchema = schemaManager.createTable(schema); return new TestFileStore.Builder( "avro", tempDir.toString(), @@ -289,7 +290,8 @@ private TestFileStore createStore( keyType, valueType, extractor, - mfFactory) + mfFactory, + tableSchema) .build(); } diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java index 74dee018ef9c..7f1fc0d3c90e 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java @@ -76,7 +76,8 @@ public void beforeEach() throws Exception { TestKeyValueGenerator.KEY_TYPE, TestKeyValueGenerator.DEFAULT_ROW_TYPE, TestKeyValueGenerator.TestKeyValueFieldsExtractor.EXTRACTOR, - DeduplicateMergeFunction.factory()) + DeduplicateMergeFunction.factory(), + null) .build(); snapshotManager = store.snapshotManager(); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java index 7ffa94809454..d51aefb962c5 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java @@ -119,13 +119,11 @@ private TableRead createRead( RecordReader.RecordIterator> rowDataIteratorCreator) { SchemaManager schemaManager = new SchemaManager(LocalFileIO.create(), tablePath); - long schemaId = 0; CoreOptions options = new CoreOptions(new HashMap<>()); KeyValueFileStoreRead read = new KeyValueFileStoreRead( options, - schemaManager, - schemaId, + schemaManager.schema(0), KEY_TYPE, VALUE_TYPE, COMPARATOR, @@ -133,7 +131,7 @@ private TableRead createRead( KeyValueFileReaderFactory.builder( LocalFileIO.create(), schemaManager, - schemaId, + schemaManager.schema(0), KEY_TYPE, VALUE_TYPE, ignore -> avro, @@ -177,11 +175,12 @@ public RecordWriter createMergeTreeWriter(BinaryRow partition, int buc Map pathFactoryMap = new HashMap<>(); pathFactoryMap.put("avro", pathFactory); + SchemaManager schemaManager = new SchemaManager(LocalFileIO.create(), tablePath); RecordWriter writer = new KeyValueFileStoreWrite( LocalFileIO.create(), - new SchemaManager(LocalFileIO.create(), tablePath), - 0, + schemaManager, + schemaManager.schema(0), commitUser, KEY_TYPE, VALUE_TYPE, From 5c0f592229fa2ecd111150f9893d1bba31d3b348 Mon Sep 17 00:00:00 2001 From: sai <57552918+hzjhjjyy@users.noreply.github.com> Date: Mon, 18 Mar 2024 10:24:45 +0800 Subject: [PATCH 15/79] [core] Add more validation for sequence field during create table. (#3022) --- .../java/org/apache/paimon/CoreOptions.java | 36 ------- .../paimon/schema/SchemaValidation.java | 98 ++++++++++--------- .../paimon/schema/SchemaManagerTest.java | 2 +- .../org/apache/paimon/schema/SchemaUtils.java | 7 -- .../apache/paimon/schema/TableSchemaTest.java | 43 ++++++++ 5 files changed, 98 insertions(+), 88 deletions(-) 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 b5b7ea432ac8..19554296cd53 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -28,7 +28,6 @@ import org.apache.paimon.options.ConfigOption; import org.apache.paimon.options.MemorySize; import org.apache.paimon.options.Options; -import org.apache.paimon.options.OptionsUtils; import org.apache.paimon.options.description.DescribedEnum; import org.apache.paimon.options.description.Description; import org.apache.paimon.options.description.InlineElement; @@ -2103,41 +2102,6 @@ public InlineElement getDescription() { } } - /** Specifies the way of making up time precision for sequence field. */ - public enum SequenceAutoPadding implements DescribedEnum { - ROW_KIND_FLAG( - "row-kind-flag", - "Pads a bit flag to indicate whether it is retract (0) or add (1) message."), - SECOND_TO_MICRO( - "second-to-micro", - "Pads the sequence field that indicates time with precision of seconds to micro-second."), - MILLIS_TO_MICRO( - "millis-to-micro", - "Pads the sequence field that indicates time with precision of milli-second to micro-second."); - - private final String value; - private final String description; - - SequenceAutoPadding(String value, String description) { - this.value = value; - this.description = description; - } - - @Override - public String toString() { - return value; - } - - @Override - public InlineElement getDescription() { - return text(description); - } - - public static SequenceAutoPadding fromString(String s) { - return OptionsUtils.convertToEnum(s, SequenceAutoPadding.class); - } - } - /** The mode for tag creation. */ public enum TagCreationMode implements DescribedEnum { NONE("none", "No automatically created tags."), 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 5cd00d8c6208..cbdcb64e5a7a 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 @@ -42,7 +42,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -91,6 +90,8 @@ public static void validateTableSchema(TableSchema schema) { validateStartupMode(options); + validateSequenceField(schema, options); + validateSequenceGroup(schema, options); ChangelogProducer changelogProducer = options.changelogProducer(); @@ -167,44 +168,6 @@ public static void validateTableSchema(TableSchema schema) { } } - List sequenceField = options.sequenceField(); - if (sequenceField.size() > 0) { - checkArgument( - schema.fieldNames().containsAll(sequenceField), - "Nonexistent sequence fields: '%s'", - sequenceField); - } - - Optional rowkindField = options.rowkindField(); - rowkindField.ifPresent( - field -> - checkArgument( - schema.fieldNames().contains(field), - "Nonexistent rowkind field: '%s'", - field)); - - if (sequenceField.size() > 0) { - sequenceField.forEach( - field -> - checkArgument( - options.fieldAggFunc(field) == null, - "Should not define aggregation on sequence field: '%s'", - field)); - } - - CoreOptions.MergeEngine mergeEngine = options.mergeEngine(); - if (mergeEngine == CoreOptions.MergeEngine.FIRST_ROW) { - if (sequenceField.size() > 0) { - throw new IllegalArgumentException( - "Do not support use sequence field on FIRST_MERGE merge engine"); - } - - if (changelogProducer != ChangelogProducer.LOOKUP) { - throw new IllegalArgumentException( - "Only support 'lookup' changelog-producer on FIRST_MERGE merge engine"); - } - } - if (schema.crossPartitionUpdate()) { if (options.bucket() != -1) { throw new IllegalArgumentException( @@ -213,16 +176,23 @@ public static void validateTableSchema(TableSchema schema) { + "(Primary key constraint %s not include all partition fields %s).", schema.primaryKeys(), schema.partitionKeys())); } + } - if (sequenceField.size() > 0) { + if (options.mergeEngine() == CoreOptions.MergeEngine.FIRST_ROW) { + if (options.changelogProducer() != ChangelogProducer.LOOKUP) { throw new IllegalArgumentException( - String.format( - "You can not use sequence.field in cross partition update case " - + "(Primary key constraint %s not include all partition fields %s).", - schema.primaryKeys(), schema.partitionKeys())); + "Only support 'lookup' changelog-producer on FIRST_MERGE merge engine"); } } + options.rowkindField() + .ifPresent( + field -> + checkArgument( + schema.fieldNames().contains(field), + "Rowkind field: '%s' can not be found in table schema.", + field)); + if (options.deletionVectorsEnabled()) { validateForDeletionVectors(schema, options); } @@ -498,4 +468,44 @@ private static void validateForDeletionVectors(TableSchema schema, CoreOptions o !options.mergeEngine().equals(MergeEngine.FIRST_ROW), "Deletion vectors mode is not supported for first row merge engine now."); } + + private static void validateSequenceField(TableSchema schema, CoreOptions options) { + List sequenceField = options.sequenceField(); + if (sequenceField.size() > 0) { + Map fieldCount = + sequenceField.stream() + .collect(Collectors.toMap(field -> field, field -> 1, Integer::sum)); + + sequenceField.forEach( + field -> { + checkArgument( + schema.fieldNames().contains(field), + "Sequence field: '%s' can not be found in table schema.", + field); + + checkArgument( + options.fieldAggFunc(field) == null, + "Should not define aggregation on sequence field: '%s'.", + field); + + checkArgument( + fieldCount.get(field) == 1, + "Sequence field '%s' is defined repeatedly.", + field); + }); + + if (options.mergeEngine() == CoreOptions.MergeEngine.FIRST_ROW) { + throw new IllegalArgumentException( + "Do not support use sequence field on FIRST_MERGE merge engine."); + } + + if (schema.crossPartitionUpdate()) { + throw new IllegalArgumentException( + String.format( + "You can not use sequence.field in cross partition update case " + + "(Primary key constraint '%s' not include all partition fields '%s').", + schema.primaryKeys(), schema.partitionKeys())); + } + } + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java index db37284be961..b5ee8c639b95 100644 --- a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java @@ -131,7 +131,7 @@ public void testCreateTableIllegal() { "f4"), "")))) .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Nonexistent sequence fields: '[f4]'"); + .hasMessageContaining("Sequence field: 'f4' can not be found in table schema."); } @Test diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaUtils.java b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaUtils.java index 435c3263b113..67010e586709 100644 --- a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaUtils.java +++ b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaUtils.java @@ -18,7 +18,6 @@ package org.apache.paimon.schema; -import org.apache.paimon.CoreOptions; import org.apache.paimon.types.DataField; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.Preconditions; @@ -69,12 +68,6 @@ public static TableSchema forceCommit(SchemaManager manager, Schema updateSchema id = 0; } - String sequenceField = options.get(CoreOptions.SEQUENCE_FIELD.key()); - Preconditions.checkArgument( - sequenceField == null || rowType.getFieldNames().contains(sequenceField), - "Nonexistent sequence field: '%s'", - sequenceField); - TableSchema newSchema = new TableSchema( id, diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java b/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java index d735ea819afd..a39ce083bf22 100644 --- a/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java @@ -18,6 +18,7 @@ package org.apache.paimon.schema; +import org.apache.paimon.CoreOptions; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; @@ -30,7 +31,10 @@ import java.util.List; import java.util.Map; +import static org.apache.paimon.CoreOptions.AGG_FUNCTION; import static org.apache.paimon.CoreOptions.BUCKET; +import static org.apache.paimon.CoreOptions.FIELDS_PREFIX; +import static org.apache.paimon.CoreOptions.MERGE_ENGINE; import static org.apache.paimon.CoreOptions.SEQUENCE_FIELD; import static org.apache.paimon.schema.SchemaValidation.validateTableSchema; import static org.assertj.core.api.Assertions.assertThat; @@ -118,6 +122,45 @@ public void testHighestFieldId() { .hasMessage("Broken schema, field id 0 is duplicated."); } + @Test + public void testSequenceField() { + List fields = + Arrays.asList( + new DataField(0, "f0", DataTypes.INT()), + new DataField(1, "f1", DataTypes.INT()), + new DataField(2, "f2", DataTypes.INT()), + new DataField(3, "f3", DataTypes.INT())); + List partitionKeys = Collections.singletonList("f0"); + List primaryKeys = Collections.singletonList("f1"); + Map options = new HashMap<>(); + + TableSchema schema = + new TableSchema(1, fields, 10, partitionKeys, primaryKeys, options, ""); + + options.put(SEQUENCE_FIELD.key(), "f3"); + assertThatThrownBy(() -> validateTableSchema(schema)) + .hasMessageContaining( + "You can not use sequence.field in cross partition update case (Primary key constraint '[f1]' not include all partition fields '[f0]')."); + + options.put(SEQUENCE_FIELD.key(), "f4"); + assertThatThrownBy(() -> validateTableSchema(schema)) + .hasMessageContaining("Sequence field: 'f4' can not be found in table schema."); + + options.put(SEQUENCE_FIELD.key(), "f2,f3,f3"); + assertThatThrownBy(() -> validateTableSchema(schema)) + .hasMessageContaining("Sequence field 'f3' is defined repeatedly."); + + options.put(SEQUENCE_FIELD.key(), "f3"); + options.put(MERGE_ENGINE.key(), CoreOptions.MergeEngine.FIRST_ROW.toString()); + assertThatThrownBy(() -> validateTableSchema(schema)) + .hasMessageContaining( + "Do not support use sequence field on FIRST_MERGE merge engine."); + + options.put(FIELDS_PREFIX + ".f3." + AGG_FUNCTION, "max"); + assertThatThrownBy(() -> validateTableSchema(schema)) + .hasMessageContaining("Should not define aggregation on sequence field: 'f3'."); + } + static RowType newRowType(boolean isNullable, int fieldId) { return new RowType( isNullable, From 5c2ca595d359601c61fdd424681a0a408f270e6c Mon Sep 17 00:00:00 2001 From: cxzl25 <3898450+cxzl25@users.noreply.github.com> Date: Mon, 18 Mar 2024 10:48:57 +0800 Subject: [PATCH 16/79] [doc] Updated Spark quickstart documentation for using --packages (#2897) --- docs/content/engines/spark.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/content/engines/spark.md b/docs/content/engines/spark.md index ce0c45b39250..4a7fde0b6aab 100644 --- a/docs/content/engines/spark.md +++ b/docs/content/engines/spark.md @@ -86,6 +86,12 @@ Append path to paimon jar file to the `--jars` argument when starting `spark-sql spark-sql ... --jars /path/to/paimon-spark-3.3-{{< version >}}.jar ``` +OR use the `--packages` option. + +```bash +spark-sql ... --packages org.apache.paimon:paimon-spark-3.3:{{< version >}} +``` + Alternatively, you can copy `paimon-spark-3.3-{{< version >}}.jar` under `spark/jars` in your Spark installation directory. **Step 2: Specify Paimon Catalog** From 0d4cbf0a626834d4b1b005efb3c1801264b080c7 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Mon, 18 Mar 2024 11:22:50 +0800 Subject: [PATCH 17/79] [core] Use var length encoding for row position (#3031) --- .../data/serializer/RowCompactedSerializer.java | 3 ++- .../apache/paimon/utils/VarLengthIntUtils.java | 3 +++ .../org/apache/paimon/mergetree/LookupLevels.java | 15 +++++++++------ 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/data/serializer/RowCompactedSerializer.java b/paimon-common/src/main/java/org/apache/paimon/data/serializer/RowCompactedSerializer.java index 3faebdabeacb..72678c364883 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/serializer/RowCompactedSerializer.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/serializer/RowCompactedSerializer.java @@ -50,6 +50,7 @@ import static org.apache.paimon.types.DataTypeChecks.getPrecision; import static org.apache.paimon.types.DataTypeChecks.getScale; import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.apache.paimon.utils.VarLengthIntUtils.MAX_VAR_INT_SIZE; /** A {@link Serializer} for {@link InternalRow} using compacted binary. */ public class RowCompactedSerializer implements Serializer { @@ -415,7 +416,7 @@ private void writeTimestamp(Timestamp value, int precision) { private void writeUnsignedInt(int value) { checkArgument(value >= 0); - ensureCapacity(5); + ensureCapacity(MAX_VAR_INT_SIZE); int len = VarLengthIntUtils.encodeInt(buffer, position, value); position += len; } diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/VarLengthIntUtils.java b/paimon-common/src/main/java/org/apache/paimon/utils/VarLengthIntUtils.java index ea556c30752d..1b2baaa561d4 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/VarLengthIntUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/VarLengthIntUtils.java @@ -29,6 +29,9 @@ /** Utils for encoding int/long to var length bytes. */ public final class VarLengthIntUtils { + public static final int MAX_VAR_LONG_SIZE = 9; + public static final int MAX_VAR_INT_SIZE = 5; + /** @return bytes length. */ public static int encodeLong(DataOutput os, long value) throws IOException { diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/LookupLevels.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/LookupLevels.java index 3e7e127027cf..dd45e7fc19c8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/LookupLevels.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/LookupLevels.java @@ -48,6 +48,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.time.Duration; +import java.util.Arrays; import java.util.Comparator; import java.util.TreeSet; import java.util.function.Function; @@ -55,6 +56,9 @@ import static org.apache.paimon.mergetree.LookupUtils.fileKibiBytes; import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.apache.paimon.utils.VarLengthIntUtils.MAX_VAR_LONG_SIZE; +import static org.apache.paimon.utils.VarLengthIntUtils.decodeLong; +import static org.apache.paimon.utils.VarLengthIntUtils.encodeLong; /** Provide lookup by key. */ public class LookupLevels implements Levels.DropFileCallback, Closeable { @@ -342,10 +346,9 @@ public byte[] persistToDisk(KeyValue kv, long rowPosition) { segment.put(bytes.length - 1, kv.valueKind().toByteValue()); return bytes; } else { - byte[] bytes = new byte[8]; - MemorySegment segment = MemorySegment.wrap(bytes); - segment.putLong(0, rowPosition); - return bytes; + byte[] bytes = new byte[MAX_VAR_LONG_SIZE]; + int len = encodeLong(bytes, rowPosition); + return Arrays.copyOf(bytes, len); } } @@ -363,8 +366,8 @@ public PositionedKeyValue readFromDisk( fileName, rowPosition); } else { - MemorySegment segment = MemorySegment.wrap(bytes); - return new PositionedKeyValue(null, fileName, segment.getLong(0)); + long rowPosition = decodeLong(bytes, 0); + return new PositionedKeyValue(null, fileName, rowPosition); } } } From 8ff370b747cca5bcf22953f3fceef8748574b241 Mon Sep 17 00:00:00 2001 From: zhoulii Date: Mon, 18 Mar 2024 13:42:19 +0800 Subject: [PATCH 18/79] [oss] use multi-release mechanism to ship jaxb (#2658) --- .../apache/paimon/plugin/PluginLoader.java | 1 + paimon-filesystems/paimon-oss-impl/pom.xml | 73 ++++++++++ .../src/main/resources/META-INF/NOTICE | 5 + .../resources/META-INF/licenses/LICENSE.jaxb | 135 ++++++++++++++++++ paimon-filesystems/paimon-oss/pom.xml | 20 +++ 5 files changed, 234 insertions(+) create mode 100644 paimon-filesystems/paimon-oss-impl/src/main/resources/META-INF/licenses/LICENSE.jaxb diff --git a/paimon-common/src/main/java/org/apache/paimon/plugin/PluginLoader.java b/paimon-common/src/main/java/org/apache/paimon/plugin/PluginLoader.java index 52100e156940..a8dcfa8dccc1 100644 --- a/paimon-common/src/main/java/org/apache/paimon/plugin/PluginLoader.java +++ b/paimon-common/src/main/java/org/apache/paimon/plugin/PluginLoader.java @@ -42,6 +42,7 @@ public class PluginLoader { Stream.concat( Arrays.stream(PARENT_FIRST_LOGGING_PATTERNS), Stream.of( + "javax.xml.bind", "org.codehaus.janino", "org.codehaus.commons", "org.apache.commons.lang3")) diff --git a/paimon-filesystems/paimon-oss-impl/pom.xml b/paimon-filesystems/paimon-oss-impl/pom.xml index 3be40262bb3f..b956e85d7d20 100644 --- a/paimon-filesystems/paimon-oss-impl/pom.xml +++ b/paimon-filesystems/paimon-oss-impl/pom.xml @@ -99,6 +99,76 @@ + + org.apache.maven.plugins + maven-jar-plugin + + + + jar + + + + + + + + true + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-javax-jars + process-resources + + copy + + + + + + + javax.xml.bind + jaxb-api + ${jaxb.api.version} + jar + true + + + ${project.build.directory}/temporary + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + unpack-javax-libraries + process-resources + + run + + + + + + + + + + + + + + + org.apache.maven.plugins @@ -123,6 +193,9 @@ .gitkeep mime.types mozilla/** + META-INF/maven/** + META-INF/versions/11/META-INF/maven/** + META-INF/LICENSE.txt diff --git a/paimon-filesystems/paimon-oss-impl/src/main/resources/META-INF/NOTICE b/paimon-filesystems/paimon-oss-impl/src/main/resources/META-INF/NOTICE index c3f0182b81b4..8fd7912fcef9 100644 --- a/paimon-filesystems/paimon-oss-impl/src/main/resources/META-INF/NOTICE +++ b/paimon-filesystems/paimon-oss-impl/src/main/resources/META-INF/NOTICE @@ -85,3 +85,8 @@ This project bundles the following dependencies under BSD License (https://opens You find it under licenses/LICENSE.stax2api. - org.codehaus.woodstox:stax2-api:4.2.1 (https://github.com/FasterXML/stax2-api/tree/stax2-api-4.2.1) + +This project bundles the following dependencies under the CDDL 1.1 license. +You find it under licenses/LICENSE.jaxb. + +- javax.xml.bind:jaxb-api:2.3.1 diff --git a/paimon-filesystems/paimon-oss-impl/src/main/resources/META-INF/licenses/LICENSE.jaxb b/paimon-filesystems/paimon-oss-impl/src/main/resources/META-INF/licenses/LICENSE.jaxb new file mode 100644 index 000000000000..fd16ea9546a4 --- /dev/null +++ b/paimon-filesystems/paimon-oss-impl/src/main/resources/META-INF/licenses/LICENSE.jaxb @@ -0,0 +1,135 @@ +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1 + +1. Definitions. + + 1.1. "Contributor" means each individual or entity that creates or contributes to the creation of Modifications. + + 1.2. "Contributor Version" means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. + + 1.3. "Covered Software" means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. + + 1.4. "Executable" means the Covered Software in any form other than Source Code. + + 1.5. "Initial Developer" means the individual or entity that first makes Original Software available under this License. + + 1.6. "Larger Work" means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. + + 1.7. "License" means this document. + + 1.8. "Licensable" means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. + + 1.9. "Modifications" means the Source Code and Executable form of any of the following: + + A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; + + B. Any new file that contains any part of the Original Software or previous Modification; or + + C. Any new file that is contributed or otherwise made available under the terms of this License. + + 1.10. "Original Software" means the Source Code and Executable form of computer software code that is originally released under this License. + + 1.11. "Patent Claims" means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. + + 1.12. "Source Code" means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. + + 1.13. "You" (or "Your") means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, "You" includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, "control" means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. + +2. License Grants. + + 2.1. The Initial Developer Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof). + + (c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License. + + (d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. + + 2.2. Contributor Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). + + (c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party. + + (d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. + +3. Distribution Obligations. + + 3.1. Availability of Source Code. + + Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. + + 3.2. Modifications. + + The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. + + 3.3. Required Notices. + + You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. + + 3.4. Application of Additional Terms. + + You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients' rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. + + 3.5. Distribution of Executable Versions. + + You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient's rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. + + 3.6. Larger Works. + + You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. + +4. Versions of the License. + + 4.1. New Versions. + + Oracle is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. + + 4.2. Effect of New Versions. + + You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. + + 4.3. Modified Versions. + + When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. + +5. DISCLAIMER OF WARRANTY. + + COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN "AS IS" BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. + +6. TERMINATION. + + 6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. + + 6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as "Participant") alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. + + 6.3. If You assert a patent infringement claim against Participant alleging that the Participant Software directly or indirectly infringes any patent where such claim is resolved (such as by license or settlement) prior to the initiation of patent infringement litigation, then the reasonable value of the licenses granted by such Participant under Sections 2.1 or 2.2 shall be taken into account in determining the amount or value of any payment or license. + + 6.4. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + + UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. + +8. U.S. GOVERNMENT END USERS. + + The Covered Software is a "commercial item," as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of "commercial computer software" (as that term is defined at 48 C.F.R. ? 252.227-7014(a)(1)) and "commercial computer software documentation" as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. + +9. MISCELLANEOUS. + + This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction's conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys' fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. + +10. RESPONSIBILITY FOR CLAIMS. + + As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. + +---------- +NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) +The code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California. diff --git a/paimon-filesystems/paimon-oss/pom.xml b/paimon-filesystems/paimon-oss/pom.xml index c5eb1037e290..0dfd6aeef125 100644 --- a/paimon-filesystems/paimon-oss/pom.xml +++ b/paimon-filesystems/paimon-oss/pom.xml @@ -56,6 +56,26 @@ + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + + true + + + + + org.apache.maven.plugins maven-dependency-plugin From 554c8588818a0e4e7e62b869870644e62885a46a Mon Sep 17 00:00:00 2001 From: tsreaper Date: Mon, 18 Mar 2024 14:45:33 +0800 Subject: [PATCH 19/79] [core] Data files with delete records should not be upgraded directly to max level (#2962) --- .../org/apache/paimon/io/DataFileMeta.java | 57 ++++++++++++++---- .../paimon/io/DataFileMetaSerializer.java | 6 +- .../paimon/io/KeyValueDataFileWriter.java | 8 ++- .../org/apache/paimon/mergetree/Levels.java | 10 +++- .../compact/MergeTreeCompactManager.java | 1 + .../compact/MergeTreeCompactTask.java | 20 ++++++- ...endOnlyTableCompactionCoordinatorTest.java | 3 +- .../crosspartition/IndexBootstrapTest.java | 3 +- .../paimon/io/DataFileTestDataGenerator.java | 3 +- .../apache/paimon/io/DataFileTestUtils.java | 11 ++-- .../ManifestCommittableSerializerTest.java | 3 +- .../manifest/ManifestFileMetaTestBase.java | 10 ++-- .../apache/paimon/mergetree/LevelsTest.java | 2 +- .../compact/IntervalPartitionTest.java | 3 +- .../compact/UniversalCompactionTest.java | 2 +- .../operation/CleanedFileStoreExpireTest.java | 3 +- .../paimon/table/sink/TableWriteTest.java | 58 ++++++++++++++++++- .../table/source/SplitGeneratorTest.java | 3 +- .../CompactionTaskSimpleSerializerTest.java | 3 +- .../FileStoreSourceSplitGeneratorTest.java | 3 +- .../FileStoreSourceSplitSerializerTest.java | 3 +- 21 files changed, 175 insertions(+), 40 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java index 0b1c5c50d4cd..3712411947f6 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java @@ -32,6 +32,8 @@ import org.apache.paimon.types.IntType; import org.apache.paimon.types.RowType; +import javax.annotation.Nullable; + import java.time.LocalDateTime; import java.time.ZoneId; import java.util.ArrayList; @@ -58,6 +60,8 @@ public class DataFileMeta { private final String fileName; private final long fileSize; + + // total number of rows (including add & delete) in this file private final long rowCount; private final BinaryRow minKey; @@ -73,6 +77,12 @@ public class DataFileMeta { private final List extraFiles; private final Timestamp creationTime; + // rowCount = addRowCount + deleteRowCount + // Why don't we keep addRowCount and deleteRowCount? + // Because in previous versions of DataFileMeta, we only keep rowCount. + // We have to keep the compatibility. + private final @Nullable Long deleteRowCount; + public static DataFileMeta forAppend( String fileName, long fileSize, @@ -92,7 +102,8 @@ public static DataFileMeta forAppend( minSequenceNumber, maxSequenceNumber, schemaId, - DUMMY_LEVEL); + DUMMY_LEVEL, + 0L); } public DataFileMeta( @@ -106,7 +117,8 @@ public DataFileMeta( long minSequenceNumber, long maxSequenceNumber, long schemaId, - int level) { + int level, + @Nullable Long deleteRowCount) { this( fileName, fileSize, @@ -120,7 +132,8 @@ public DataFileMeta( schemaId, level, Collections.emptyList(), - Timestamp.fromLocalDateTime(LocalDateTime.now()).toMillisTimestamp()); + Timestamp.fromLocalDateTime(LocalDateTime.now()).toMillisTimestamp(), + deleteRowCount); } public DataFileMeta( @@ -136,9 +149,11 @@ public DataFileMeta( long schemaId, int level, List extraFiles, - Timestamp creationTime) { + Timestamp creationTime, + @Nullable Long deleteRowCount) { this.fileName = fileName; this.fileSize = fileSize; + this.rowCount = rowCount; this.minKey = minKey; @@ -152,6 +167,8 @@ public DataFileMeta( this.schemaId = schemaId; this.extraFiles = Collections.unmodifiableList(extraFiles); this.creationTime = creationTime; + + this.deleteRowCount = deleteRowCount; } public String fileName() { @@ -166,6 +183,14 @@ public long rowCount() { return rowCount; } + public Optional addRowCount() { + return Optional.ofNullable(deleteRowCount).map(c -> rowCount - c); + } + + public Optional deleteRowCount() { + return Optional.ofNullable(deleteRowCount); + } + public BinaryRow minKey() { return minKey; } @@ -250,7 +275,8 @@ public DataFileMeta upgrade(int newLevel) { schemaId, newLevel, extraFiles, - creationTime); + creationTime, + deleteRowCount); } public List collectFiles(DataFilePathFactory pathFactory) { @@ -274,11 +300,15 @@ public DataFileMeta copy(List newExtraFiles) { schemaId, level, newExtraFiles, - creationTime); + creationTime, + deleteRowCount); } @Override public boolean equals(Object o) { + if (o == this) { + return true; + } if (!(o instanceof DataFileMeta)) { return false; } @@ -295,7 +325,8 @@ public boolean equals(Object o) { && schemaId == that.schemaId && level == that.level && Objects.equals(extraFiles, that.extraFiles) - && Objects.equals(creationTime, that.creationTime); + && Objects.equals(creationTime, that.creationTime) + && Objects.equals(deleteRowCount, that.deleteRowCount); } @Override @@ -313,13 +344,17 @@ public int hashCode() { schemaId, level, extraFiles, - creationTime); + creationTime, + deleteRowCount); } @Override public String toString() { return String.format( - "{%s, %d, %d, %s, %s, %s, %s, %d, %d, %d, %d, %s, %s}", + "{fileName: %s, fileSize: %d, rowCount: %d, " + + "minKey: %s, maxKey: %s, keyStats: %s, valueStats: %s, " + + "minSequenceNumber: %d, maxSequenceNumber: %d, " + + "schemaId: %d, level: %d, extraFiles: %s, creationTime: %s, deleteRowCount: %d}", fileName, fileSize, rowCount, @@ -332,7 +367,8 @@ public String toString() { schemaId, level, extraFiles, - creationTime); + creationTime, + deleteRowCount); } public static RowType schema() { @@ -350,6 +386,7 @@ public static RowType schema() { fields.add(new DataField(10, "_LEVEL", new IntType(false))); fields.add(new DataField(11, "_EXTRA_FILES", new ArrayType(false, newStringType(false)))); fields.add(new DataField(12, "_CREATION_TIME", DataTypes.TIMESTAMP_MILLIS())); + fields.add(new DataField(13, "_DELETE_ROW_COUNT", new BigIntType(true))); return new RowType(fields); } diff --git a/paimon-core/src/main/java/org/apache/paimon/io/DataFileMetaSerializer.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFileMetaSerializer.java index a823b5791d63..f91e3d293361 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/DataFileMetaSerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFileMetaSerializer.java @@ -53,7 +53,8 @@ public InternalRow toRow(DataFileMeta meta) { meta.schemaId(), meta.level(), toStringArrayData(meta.extraFiles()), - meta.creationTime()); + meta.creationTime(), + meta.deleteRowCount().orElse(null)); } @Override @@ -71,6 +72,7 @@ public DataFileMeta fromRow(InternalRow row) { row.getLong(9), row.getInt(10), fromStringArrayData(row.getArray(11)), - row.getTimestamp(12, 3)); + row.getTimestamp(12, 3), + row.isNullAt(13) ? null : row.getLong(13)); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java index c5b2de741ea9..e2e5441f680e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java @@ -67,6 +67,7 @@ public class KeyValueDataFileWriter private InternalRow maxKey = null; private long minSeqNumber = Long.MAX_VALUE; private long maxSeqNumber = Long.MIN_VALUE; + private long deleteRecordCount = 0; public KeyValueDataFileWriter( FileIO fileIO, @@ -111,6 +112,10 @@ public void write(KeyValue kv) throws IOException { updateMinSeqNumber(kv); updateMaxSeqNumber(kv); + if (kv.valueKind().isRetract()) { + deleteRecordCount++; + } + if (LOG.isDebugEnabled()) { LOG.debug("Write to Path " + path + " key value " + kv.toString(keyType, valueType)); } @@ -162,6 +167,7 @@ public DataFileMeta result() throws IOException { minSeqNumber, maxSeqNumber, schemaId, - level); + level, + deleteRecordCount); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/Levels.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/Levels.java index 94800e8eba5d..350b693dbf8b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/Levels.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/Levels.java @@ -50,11 +50,11 @@ public Levels( this.keyComparator = keyComparator; // in case the num of levels is not specified explicitly - int restoredMaxLevel = + int restoredNumLevels = Math.max( numLevels, inputFiles.stream().mapToInt(DataFileMeta::level).max().orElse(-1) + 1); - checkArgument(restoredMaxLevel > 1, "levels must be at least 2."); + checkArgument(restoredNumLevels > 1, "Number of levels must be at least 2."); this.level0 = new TreeSet<>( (a, b) -> { @@ -70,7 +70,7 @@ public Levels( } }); this.levels = new ArrayList<>(); - for (int i = 1; i < restoredMaxLevel; i++) { + for (int i = 1; i < restoredNumLevels; i++) { levels.add(SortedRun.empty()); } @@ -108,6 +108,10 @@ public int numberOfLevels() { return levels.size() + 1; } + public int maxLevel() { + return levels.size(); + } + public int numberOfSortedRuns() { int numberOfSortedRuns = level0.size(); for (SortedRun run : levels) { 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 7bdb44118f2b..80316eacbdee 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 @@ -182,6 +182,7 @@ private void submitCompaction(CompactUnit unit, boolean dropDelete) { rewriter, unit, dropDelete, + levels.maxLevel(), metricsReporter); if (LOG.isDebugEnabled()) { LOG.debug( 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 7299fbb5c95d..5f96743fee04 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 @@ -44,6 +44,7 @@ public class MergeTreeCompactTask extends CompactTask { private final List> partitioned; private final boolean dropDelete; + private final int maxLevel; // metric private int upgradeFilesNum; @@ -54,6 +55,7 @@ public MergeTreeCompactTask( CompactRewriter rewriter, CompactUnit unit, boolean dropDelete, + int maxLevel, @Nullable CompactionMetrics.Reporter metricsReporter) { super(metricsReporter); this.minFileSize = minFileSize; @@ -61,6 +63,7 @@ public MergeTreeCompactTask( this.outputLevel = unit.outputLevel(); this.partitioned = new IntervalPartition(unit.files(), keyComparator).partition(); this.dropDelete = dropDelete; + this.maxLevel = maxLevel; this.upgradeFilesNum = 0; } @@ -107,10 +110,20 @@ protected String logMetric( } private void upgrade(DataFileMeta file, CompactResult toUpdate) throws Exception { - if (file.level() != outputLevel) { + if (file.level() == outputLevel) { + return; + } + + if (outputLevel != maxLevel || file.deleteRowCount().map(d -> d == 0).orElse(false)) { CompactResult upgradeResult = rewriter.upgrade(outputLevel, file); toUpdate.merge(upgradeResult); upgradeFilesNum++; + } else { + // files with delete records should not be upgraded directly to max level + List> candidate = new ArrayList<>(); + candidate.add(new ArrayList<>()); + candidate.get(0).add(SortedRun.fromSingle(file)); + rewriteImpl(candidate, toUpdate); } } @@ -130,6 +143,11 @@ private void rewrite(List> candidate, CompactResult toUpdate) th return; } } + rewriteImpl(candidate, toUpdate); + } + + private void rewriteImpl(List> candidate, CompactResult toUpdate) + throws Exception { CompactResult rewriteResult = rewriter.rewrite(outputLevel, dropDelete, candidate); toUpdate.merge(rewriteResult); candidate.clear(); diff --git a/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyTableCompactionCoordinatorTest.java b/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyTableCompactionCoordinatorTest.java index 0b00cfefa65e..e6209e2234f1 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyTableCompactionCoordinatorTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyTableCompactionCoordinatorTest.java @@ -185,6 +185,7 @@ private DataFileMeta newFile(long fileSize) { 0, 0, 0, - 0); + 0, + 0L); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/crosspartition/IndexBootstrapTest.java b/paimon-core/src/test/java/org/apache/paimon/crosspartition/IndexBootstrapTest.java index cd9568d465eb..50df6e4971ec 100644 --- a/paimon-core/src/test/java/org/apache/paimon/crosspartition/IndexBootstrapTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/crosspartition/IndexBootstrapTest.java @@ -132,7 +132,8 @@ private static DataFileMeta newFile(long timeMillis) { Timestamp.fromLocalDateTime( Instant.ofEpochMilli(timeMillis) .atZone(ZoneId.systemDefault()) - .toLocalDateTime())); + .toLocalDateTime()), + 0L); } private Pair row(int pt, int col, int pk, int bucket) { diff --git a/paimon-core/src/test/java/org/apache/paimon/io/DataFileTestDataGenerator.java b/paimon-core/src/test/java/org/apache/paimon/io/DataFileTestDataGenerator.java index 6301fbc3d715..aed01ca21857 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/DataFileTestDataGenerator.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/DataFileTestDataGenerator.java @@ -160,7 +160,8 @@ private Data createDataFile(List kvs, int level, BinaryRow partition, minSequenceNumber, maxSequenceNumber, 0, - level), + level, + kvs.stream().filter(kv -> kv.valueKind().isRetract()).count()), kvs); } diff --git a/paimon-core/src/test/java/org/apache/paimon/io/DataFileTestUtils.java b/paimon-core/src/test/java/org/apache/paimon/io/DataFileTestUtils.java index 062c3384a4b0..624237b38680 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/DataFileTestUtils.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/DataFileTestUtils.java @@ -40,7 +40,7 @@ public static DataFileMeta newFile(long minSeq, long maxSeq) { return new DataFileMeta( "", maxSeq - minSeq + 1, - maxSeq - minSeq + 1, + 0L, DataFileMeta.EMPTY_MIN_KEY, DataFileMeta.EMPTY_MAX_KEY, DataFileMeta.EMPTY_KEY_STATS, @@ -50,7 +50,8 @@ public static DataFileMeta newFile(long minSeq, long maxSeq) { 0L, DataFileMeta.DUMMY_LEVEL, Collections.emptyList(), - Timestamp.fromEpochMillis(100)); + Timestamp.fromEpochMillis(100), + maxSeq - minSeq + 1); } public static DataFileMeta newFile() { @@ -65,7 +66,8 @@ public static DataFileMeta newFile() { 0, 0, 0, - 0); + 0, + 0L); } public static DataFileMeta newFile( @@ -81,7 +83,8 @@ public static DataFileMeta newFile( 0, maxSequence, 0, - level); + level, + 0L); } public static BinaryRow row(int i) { diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java index 88b309bfd759..ee279c097a17 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java @@ -113,6 +113,7 @@ public static DataFileMeta newFile(int name, int level) { 0, 1, 0, - level); + level, + 0L); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java index 3a9754decb84..1a36346c1147 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java @@ -72,7 +72,7 @@ protected ManifestEntry makeEntry(boolean isAdd, String fileName, Integer partit fileName, 0, // not used 0, // not used - binaryRow, // not useds + binaryRow, // not used binaryRow, // not used StatsTestUtils.newEmptyTableStats(), // not used StatsTestUtils.newEmptyTableStats(), // not used @@ -81,7 +81,9 @@ protected ManifestEntry makeEntry(boolean isAdd, String fileName, Integer partit 0, // not used 0, // not used Collections.emptyList(), - Timestamp.fromEpochMillis(200000))); + Timestamp.fromEpochMillis(200000), + 0L // not used + )); } protected ManifestFileMeta makeManifest(ManifestEntry... entries) { @@ -242,7 +244,7 @@ public static ManifestEntry makeEntry( 0, // not used 0, // not used 0, // not used - 0 // not used - )); + 0, // not used + 0L)); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/LevelsTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/LevelsTest.java index 2630be65cfd3..c424b6094277 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/LevelsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/LevelsTest.java @@ -69,6 +69,6 @@ public void testLevel0WithSameSequenceNumbers() { public static DataFileMeta newFile(int level) { return new DataFileMeta( - UUID.randomUUID().toString(), 0, 1, row(0), row(0), null, null, 0, 1, 0, level); + UUID.randomUUID().toString(), 0, 1, row(0), row(0), null, null, 0, 1, 0, level, 0L); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/IntervalPartitionTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/IntervalPartitionTest.java index 02d7967a15a8..c4edd76e264e 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/IntervalPartitionTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/IntervalPartitionTest.java @@ -180,7 +180,8 @@ private DataFileMeta makeInterval(int left, int right) { 0, 0, Collections.emptyList(), - Timestamp.fromEpochMillis(100000)); + Timestamp.fromEpochMillis(100000), + 0L); } private List> toMultiset(List> sections) { diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/UniversalCompactionTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/UniversalCompactionTest.java index 747023b2ee3e..313f9799a5a8 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/UniversalCompactionTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/UniversalCompactionTest.java @@ -357,6 +357,6 @@ private LevelSortedRun level(int level, long size) { } static DataFileMeta file(long size) { - return new DataFileMeta("", size, 1, null, null, null, null, 0, 0, 0, 0); + return new DataFileMeta("", size, 1, null, null, null, null, 0, 0, 0, 0, 0L); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/CleanedFileStoreExpireTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/CleanedFileStoreExpireTest.java index 463e8ba088eb..02e3ab9e333c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/CleanedFileStoreExpireTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/CleanedFileStoreExpireTest.java @@ -85,7 +85,8 @@ public void testExpireExtraFiles() throws IOException { 0, 0, extraFiles, - Timestamp.now()); + Timestamp.now(), + 0L); ManifestEntry add = new ManifestEntry(FileKind.ADD, partition, 0, 1, dataFile); ManifestEntry delete = new ManifestEntry(FileKind.DELETE, partition, 0, 1, dataFile); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/sink/TableWriteTest.java b/paimon-core/src/test/java/org/apache/paimon/table/sink/TableWriteTest.java index ccbade9ade41..a97ed9c54fd8 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/sink/TableWriteTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/sink/TableWriteTest.java @@ -19,6 +19,8 @@ package org.apache.paimon.table.sink; import org.apache.paimon.CoreOptions; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.data.BinaryRowWriter; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManagerImpl; @@ -40,6 +42,7 @@ import org.apache.paimon.table.source.TableScan; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowKind; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.TraceableFileIO; @@ -220,14 +223,63 @@ public void testChangelogWhenNotWaitForCompaction() throws Exception { StreamTableScan scan = table.newStreamScan(); TableRead read = table.newRead(); + assertThat(streamingRead(scan, read, latestSnapshotId)) + .hasSize(numPartitions * numRecordsPerPartition); + } + + @Test + public void testUpgradeToMaxLevel() throws Exception { + Options conf = new Options(); + conf.set(CoreOptions.BUCKET, 1); + conf.set(CoreOptions.CHANGELOG_PRODUCER, CoreOptions.ChangelogProducer.FULL_COMPACTION); + + FileStoreTable table = createFileStoreTable(conf); + TableWriteImpl write = + table.newWrite(commitUser).withIOManager(new IOManagerImpl(tempDir.toString())); + StreamTableCommit commit = table.newCommit(commitUser); + + write.write(GenericRow.of(1, 1, 10L)); + write.write(GenericRow.of(1, 2, 20L)); + write.write(GenericRow.ofKind(RowKind.DELETE, 1, 2, 20L)); + commit.commit(0, write.prepareCommit(false, 0)); + + write.compact(partition(1), 0, true); + commit.commit(1, write.prepareCommit(true, 1)); + + write.write(GenericRow.of(1, 2, 21L)); + write.compact(partition(1), 0, true); + commit.commit(2, write.prepareCommit(true, 2)); + + write.close(); + commit.close(); + + Map readOptions = new HashMap<>(); + readOptions.put(CoreOptions.SCAN_SNAPSHOT_ID.key(), "1"); + table = table.copy(readOptions); + long latestSnapshotId = table.snapshotManager().latestSnapshotId(); + + StreamTableScan scan = table.newStreamScan(); + TableRead read = table.newRead(); + assertThat(streamingRead(scan, read, latestSnapshotId)).hasSize(2); + } + + private BinaryRow partition(int x) { + BinaryRow partition = new BinaryRow(1); + BinaryRowWriter writer = new BinaryRowWriter(partition); + writer.writeInt(0, x); + writer.complete(); + return partition; + } + + List streamingRead( + StreamTableScan scan, TableRead read, long numStreamingSnapshots) throws Exception { List actual = new ArrayList<>(); - for (long i = 0; i <= latestSnapshotId; i++) { + for (long i = 0; i <= numStreamingSnapshots; i++) { RecordReader reader = read.createReader(scan.plan().splits()); reader.forEachRemaining(actual::add); reader.close(); } - - assertThat(actual).hasSize(numPartitions * numRecordsPerPartition); + return actual; } private FileStoreTable createFileStoreTable(Options conf) throws Exception { diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java index 663d0dad8922..6d97eda5f69a 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java @@ -50,7 +50,8 @@ public static DataFileMeta newFileFromSequence( minSequence, maxSequence, 0, - 0); + 0, + 0L); } @Test diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactionTaskSimpleSerializerTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactionTaskSimpleSerializerTest.java index 70f127291009..c607e7a8f4a4 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactionTaskSimpleSerializerTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactionTaskSimpleSerializerTest.java @@ -74,6 +74,7 @@ private DataFileMeta newFile() { 0, 1, 0, - 0); + 0, + 0L); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitGeneratorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitGeneratorTest.java index 5015e92194ca..4f53932a2877 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitGeneratorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitGeneratorTest.java @@ -111,7 +111,8 @@ private DataSplit dataSplit(int partition, int bucket, String... fileNames) { 0, // not used 0, // not used 0, // not used - 0 // not used + 0, // not used + 0L // not used )); } return DataSplit.builder() diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitSerializerTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitSerializerTest.java index d0cafbb9581c..cbe6cb86d8f3 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitSerializerTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitSerializerTest.java @@ -84,7 +84,8 @@ public static DataFileMeta newFile(int level) { 0, 1, 0, - level); + level, + 0L); } public static FileStoreSourceSplit newSourceSplit( From f4d05213395ced891dc5081401bde794cb7353b7 Mon Sep 17 00:00:00 2001 From: wgcn <1026688210@qq.com> Date: Mon, 18 Mar 2024 14:52:05 +0800 Subject: [PATCH 20/79] [Flink] Introduce range strategy for sort compaction. (#2749) --- .../generated/core_configuration.html | 7 + .../java/org/apache/paimon/CoreOptions.java | 18 ++ .../types/InternalRowToSizeVisitor.java | 299 ++++++++++++++++++ .../types/InternalRowToSizeVisitorTest.java | 195 ++++++++++++ .../paimon/flink/shuffle/RangeShuffle.java | 167 +++++++--- .../apache/paimon/flink/sorter/SortUtils.java | 4 +- ...rtCompactActionForUnawareBucketITCase.java | 34 +- .../flink/shuffle/RangeShuffleTest.java | 75 +++++ 8 files changed, 750 insertions(+), 49 deletions(-) create mode 100644 paimon-common/src/main/java/org/apache/paimon/types/InternalRowToSizeVisitor.java create mode 100644 paimon-common/src/test/java/org/apache/paimon/types/InternalRowToSizeVisitorTest.java create mode 100644 paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/shuffle/RangeShuffleTest.java diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index 52645cf8cc78..ecf92c2f1067 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -593,6 +593,13 @@ Duration In watermarking, if a source remains idle beyond the specified timeout duration, it triggers snapshot advancement and facilitates tag creation. + +
sort-compaction.range-strategy
+ QUANTITY +

Enum

+ The range strategy of sort compaction, the default value is quantity. +If the data size allocated for the sorting task is uneven,which may lead to performance bottlenecks, the config can be set to size.

Possible values:
  • "SIZE"
  • "QUANTITY"
+
sort-engine
loser-tree 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 19554296cd53..9deeb6324f3a 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -1083,6 +1083,14 @@ public class CoreOptions implements Serializable { "Whether to enable deletion vectors mode. In this mode, index files containing deletion" + " vectors are generated when data is written, which marks the data for deletion." + " During read operations, by applying these index files, merging can be avoided."); + public static final ConfigOption SORT_RANG_STRATEGY = + key("sort-compaction.range-strategy") + .enumType(RangeStrategy.class) + .defaultValue(RangeStrategy.QUANTITY) + .withDescription( + "The range strategy of sort compaction, the default value is quantity.\n" + + "If the data size allocated for the sorting task is uneven,which may lead to performance bottlenecks, " + + "the config can be set to size."); private final Options options; @@ -1150,6 +1158,10 @@ public String partitionDefaultName() { return options.get(PARTITION_DEFAULT_NAME); } + public boolean sortBySize() { + return options.get(SORT_RANG_STRATEGY) == RangeStrategy.SIZE; + } + public static FileFormat createFileFormat( Options options, ConfigOption formatOption) { String formatIdentifier = options.get(formatOption).toString(); @@ -2210,6 +2222,12 @@ public InlineElement getDescription() { } } + /** Specifies range strategy. */ + public enum RangeStrategy { + SIZE, + QUANTITY + } + /** Specifies the log consistency mode for table. */ public enum ConsumerMode implements DescribedEnum { EXACTLY_ONCE( diff --git a/paimon-common/src/main/java/org/apache/paimon/types/InternalRowToSizeVisitor.java b/paimon-common/src/main/java/org/apache/paimon/types/InternalRowToSizeVisitor.java new file mode 100644 index 000000000000..4e1ad782cf21 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/types/InternalRowToSizeVisitor.java @@ -0,0 +1,299 @@ +/* + * 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.types; + +import org.apache.paimon.data.DataGetters; +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalMap; +import org.apache.paimon.data.InternalRow; + +import java.util.List; +import java.util.function.BiFunction; + +/** The class is to calculate the occupied space size based on Datatype. */ +public class InternalRowToSizeVisitor + implements DataTypeVisitor> { + + public static final int NULL_SIZE = 0; + + @Override + public BiFunction visit(CharType charType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return row.getString(index).toBytes().length; + } + }; + } + + @Override + public BiFunction visit(VarCharType varCharType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return row.getString(index).toBytes().length; + } + }; + } + + @Override + public BiFunction visit(BooleanType booleanType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return 1; + } + }; + } + + @Override + public BiFunction visit(BinaryType binaryType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return row.getBinary(index).length; + } + }; + } + + @Override + public BiFunction visit(VarBinaryType varBinaryType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return row.getBinary(index).length; + } + }; + } + + @Override + public BiFunction visit(DecimalType decimalType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return row.getDecimal(index, decimalType.getPrecision(), decimalType.getScale()) + .toUnscaledBytes() + .length; + } + }; + } + + @Override + public BiFunction visit(TinyIntType tinyIntType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return 1; + } + }; + } + + @Override + public BiFunction visit(SmallIntType smallIntType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return 2; + } + }; + } + + @Override + public BiFunction visit(IntType intType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return 4; + } + }; + } + + @Override + public BiFunction visit(BigIntType bigIntType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return 8; + } + }; + } + + @Override + public BiFunction visit(FloatType floatType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return 4; + } + }; + } + + @Override + public BiFunction visit(DoubleType doubleType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return 8; + } + }; + } + + @Override + public BiFunction visit(DateType dateType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return 4; + } + }; + } + + @Override + public BiFunction visit(TimeType timeType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return 4; + } + }; + } + + @Override + public BiFunction visit(TimestampType timestampType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return 8; + } + }; + } + + @Override + public BiFunction visit( + LocalZonedTimestampType localZonedTimestampType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + return 8; + } + }; + } + + @Override + public BiFunction visit(ArrayType arrayType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + BiFunction function = + arrayType.getElementType().accept(this); + InternalArray internalArray = row.getArray(index); + + int size = 0; + for (int i = 0; i < internalArray.size(); i++) { + size += function.apply(internalArray, i); + } + + return size; + } + }; + } + + @Override + public BiFunction visit(MultisetType multisetType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + BiFunction function = + multisetType.getElementType().accept(this); + InternalMap map = row.getMap(index); + + int size = 0; + for (int i = 0; i < map.size(); i++) { + size += function.apply(map.keyArray(), i); + } + + return size; + } + }; + } + + @Override + public BiFunction visit(MapType mapType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + + BiFunction keyFunction = + mapType.getKeyType().accept(this); + BiFunction valueFunction = + mapType.getValueType().accept(this); + + InternalMap map = row.getMap(index); + + int size = 0; + for (int i = 0; i < map.size(); i++) { + size += keyFunction.apply(map.keyArray(), i); + } + + for (int i = 0; i < map.size(); i++) { + size += valueFunction.apply(map.valueArray(), i); + } + + return size; + } + }; + } + + @Override + public BiFunction visit(RowType rowType) { + return (row, index) -> { + if (row.isNullAt(index)) { + return NULL_SIZE; + } else { + int size = 0; + List fieldTypes = rowType.getFieldTypes(); + InternalRow nestRow = row.getRow(index, rowType.getFieldCount()); + for (int i = 0; i < fieldTypes.size(); i++) { + DataType dataType = fieldTypes.get(i); + size += dataType.accept(this).apply(nestRow, i); + } + return size; + } + }; + } +} diff --git a/paimon-common/src/test/java/org/apache/paimon/types/InternalRowToSizeVisitorTest.java b/paimon-common/src/test/java/org/apache/paimon/types/InternalRowToSizeVisitorTest.java new file mode 100644 index 000000000000..cfdae649c190 --- /dev/null +++ b/paimon-common/src/test/java/org/apache/paimon/types/InternalRowToSizeVisitorTest.java @@ -0,0 +1,195 @@ +/* + * 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.types; + +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.DataGetters; +import org.apache.paimon.data.Decimal; +import org.apache.paimon.data.GenericArray; +import org.apache.paimon.data.GenericMap; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.Timestamp; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.math.BigDecimal; +import java.util.Collections; +import java.util.List; +import java.util.function.BiFunction; +import java.util.stream.Collectors; + +/** Test for InternalRowToSizeVisitor. */ +public class InternalRowToSizeVisitorTest { + + private List> feildSizeCalculator; + + @BeforeEach + void setUp() { + RowType rowType = + RowType.builder() + .field("a0", DataTypes.INT()) + .field("a1", DataTypes.TINYINT()) + .field("a2", DataTypes.SMALLINT()) + .field("a3", DataTypes.BIGINT()) + .field("a4", DataTypes.STRING()) + .field("a5", DataTypes.DOUBLE()) + .field("a6", DataTypes.ARRAY(DataTypes.STRING())) + .field("a7", DataTypes.CHAR(100)) + .field("a8", DataTypes.VARCHAR(100)) + .field("a9", DataTypes.BOOLEAN()) + .field("a10", DataTypes.DATE()) + .field("a11", DataTypes.TIME()) + .field("a12", DataTypes.TIMESTAMP()) + .field("a13", DataTypes.TIMESTAMP_MILLIS()) + .field("a14", DataTypes.DECIMAL(3, 3)) + .field("a15", DataTypes.BYTES()) + .field("a16", DataTypes.FLOAT()) + .field("a17", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())) + .field("a18", DataTypes.ROW(DataTypes.FIELD(100, "b1", DataTypes.STRING()))) + .field("a19", DataTypes.BINARY(100)) + .field("a20", DataTypes.VARBINARY(100)) + .field("a21", DataTypes.MULTISET(DataTypes.STRING())) + .field( + "a22", + DataTypes.ROW( + DataTypes.FIELD( + 101, + "b2", + DataTypes.ROW( + DataTypes.FIELD( + 102, + "b3", + DataTypes.MAP( + DataTypes.STRING(), + DataTypes.STRING())), + DataTypes.FIELD( + 103, + "b4", + DataTypes.ARRAY( + DataTypes.STRING())), + DataTypes.FIELD( + 104, + "b5", + DataTypes.MULTISET( + DataTypes.STRING())))))) + .field("a23", DataTypes.MULTISET(DataTypes.STRING())) + .build(); + + InternalRowToSizeVisitor internalRowToSizeVisitor = new InternalRowToSizeVisitor(); + feildSizeCalculator = + rowType.getFieldTypes().stream() + .map(dataType -> dataType.accept(internalRowToSizeVisitor)) + .collect(Collectors.toList()); + } + + @Test + void testCalculatorSize() { + GenericRow row = new GenericRow(24); + + row.setField(0, 1); + Assertions.assertThat(feildSizeCalculator.get(0).apply(row, 0)).isEqualTo(4); + + row.setField(1, (byte) 1); + Assertions.assertThat(feildSizeCalculator.get(1).apply(row, 1)).isEqualTo(1); + + row.setField(2, (short) 1); + Assertions.assertThat(feildSizeCalculator.get(2).apply(row, 2)).isEqualTo(2); + + row.setField(3, 1L); + Assertions.assertThat(feildSizeCalculator.get(3).apply(row, 3)).isEqualTo(8); + + row.setField(4, BinaryString.fromString("a")); + Assertions.assertThat(feildSizeCalculator.get(4).apply(row, 4)).isEqualTo(1); + + row.setField(5, 0.5D); + Assertions.assertThat(feildSizeCalculator.get(5).apply(row, 5)).isEqualTo(8); + + row.setField(6, new GenericArray(new Object[] {BinaryString.fromString("1")})); + Assertions.assertThat(feildSizeCalculator.get(6).apply(row, 6)).isEqualTo(1); + + row.setField(7, BinaryString.fromString("3")); + Assertions.assertThat(feildSizeCalculator.get(7).apply(row, 7)).isEqualTo(1); + + row.setField(8, BinaryString.fromString("3")); + Assertions.assertThat(feildSizeCalculator.get(8).apply(row, 8)).isEqualTo(1); + + row.setField(9, true); + Assertions.assertThat(feildSizeCalculator.get(9).apply(row, 9)).isEqualTo(1); + + row.setField(10, 375); + Assertions.assertThat(feildSizeCalculator.get(10).apply(row, 10)).isEqualTo(4); + + row.setField(11, 100); + Assertions.assertThat(feildSizeCalculator.get(11).apply(row, 11)).isEqualTo(4); + + row.setField(12, Timestamp.fromEpochMillis(1685548953000L)); + Assertions.assertThat(feildSizeCalculator.get(12).apply(row, 12)).isEqualTo(8); + + row.setField(13, Timestamp.fromEpochMillis(1685548953000L)); + Assertions.assertThat(feildSizeCalculator.get(13).apply(row, 13)).isEqualTo(8); + + row.setField(14, Decimal.fromBigDecimal(new BigDecimal("0.22"), 3, 3)); + Assertions.assertThat(feildSizeCalculator.get(14).apply(row, 14)).isEqualTo(2); + + row.setField(15, new byte[] {1, 5, 2}); + Assertions.assertThat(feildSizeCalculator.get(15).apply(row, 15)).isEqualTo(3); + + row.setField(16, 0.26F); + Assertions.assertThat(feildSizeCalculator.get(16).apply(row, 16)).isEqualTo(4); + + row.setField( + 17, + new GenericMap( + Collections.singletonMap( + BinaryString.fromString("k"), BinaryString.fromString("v")))); + Assertions.assertThat(feildSizeCalculator.get(17).apply(row, 17)).isEqualTo(2); + + row.setField(18, GenericRow.of(BinaryString.fromString("cc"))); + Assertions.assertThat(feildSizeCalculator.get(18).apply(row, 18)).isEqualTo(2); + + row.setField(19, "bb".getBytes()); + Assertions.assertThat(feildSizeCalculator.get(19).apply(row, 19)).isEqualTo(2); + + row.setField(20, "aa".getBytes()); + Assertions.assertThat(feildSizeCalculator.get(20).apply(row, 20)).isEqualTo(2); + + row.setField( + 21, new GenericMap(Collections.singletonMap(BinaryString.fromString("set"), 1))); + + Assertions.assertThat(feildSizeCalculator.get(21).apply(row, 21)).isEqualTo(3); + + row.setField( + 22, + GenericRow.of( + GenericRow.of( + new GenericMap( + Collections.singletonMap( + BinaryString.fromString("k"), + BinaryString.fromString("v"))), + new GenericArray(new Object[] {BinaryString.fromString("1")}), + new GenericMap( + Collections.singletonMap( + BinaryString.fromString("set"), 1))))); + Assertions.assertThat(feildSizeCalculator.get(22).apply(row, 22)).isEqualTo(6); + + Assertions.assertThat(feildSizeCalculator.get(23).apply(row, 23)).isEqualTo(0); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java index a81cc77109f6..9c67e88559aa 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java @@ -18,18 +18,26 @@ package org.apache.paimon.flink.shuffle; +import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.data.DataGetters; +import org.apache.paimon.flink.FlinkRowWrapper; +import org.apache.paimon.types.InternalRowToSizeVisitor; +import org.apache.paimon.types.RowType; import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.SerializableSupplier; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.ListTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.api.operators.InputSelectable; @@ -54,12 +62,13 @@ import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.PriorityQueue; import java.util.Random; +import java.util.function.BiFunction; +import java.util.stream.Collectors; /** * RangeShuffle Util to shuffle the input stream by the sampling range. See `rangeShuffleBykey` @@ -89,28 +98,33 @@ public static DataStream> rangeShuffleByKey( TypeInformation keyTypeInformation, int sampleSize, int rangeNum, - int outParallelism) { + int outParallelism, + RowType valueRowType, + boolean isSortBySize) { Transformation> input = inputDataStream.getTransformation(); - OneInputTransformation, T> keyInput = + OneInputTransformation, Tuple2> keyInput = new OneInputTransformation<>( input, - "ABSTRACT KEY", - new StreamMap<>(a -> a.f0), - keyTypeInformation, + "ABSTRACT KEY AND SIZE", + new StreamMap<>(new KeyAndSizeExtractor<>(valueRowType, isSortBySize)), + new TupleTypeInfo<>(keyTypeInformation, BasicTypeInfo.INT_TYPE_INFO), input.getParallelism()); // 1. Fixed size sample in each partitions. - OneInputTransformation> localSample = + OneInputTransformation, Tuple3> localSample = new OneInputTransformation<>( keyInput, "LOCAL SAMPLE", new LocalSampleOperator<>(sampleSize), - new TupleTypeInfo<>(BasicTypeInfo.DOUBLE_TYPE_INFO, keyTypeInformation), + new TupleTypeInfo<>( + BasicTypeInfo.DOUBLE_TYPE_INFO, + keyTypeInformation, + BasicTypeInfo.INT_TYPE_INFO), keyInput.getParallelism()); // 2. Collect all the samples and gather them into a sorted key range. - OneInputTransformation, List> sampleAndHistogram = + OneInputTransformation, List> sampleAndHistogram = new OneInputTransformation<>( localSample, "GLOBAL SAMPLE", @@ -155,6 +169,47 @@ public static DataStream> rangeShuffleByKey( outParallelism)); } + /** KeyAndSizeExtractor is responsible for extracting the sort key and row size. */ + public static class KeyAndSizeExtractor + extends RichMapFunction, Tuple2> { + private final RowType rowType; + private final boolean isSortBySize; + private transient List> fieldSizeCalculator; + + public KeyAndSizeExtractor(RowType rowType, boolean isSortBySize) { + this.rowType = rowType; + this.isSortBySize = isSortBySize; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + InternalRowToSizeVisitor internalRowToSizeVisitor = new InternalRowToSizeVisitor(); + fieldSizeCalculator = + rowType.getFieldTypes().stream() + .map(dataType -> dataType.accept(internalRowToSizeVisitor)) + .collect(Collectors.toList()); + } + + @Override + public Tuple2 map(Tuple2 keyAndRowData) throws Exception { + if (isSortBySize) { + int size = 0; + for (int i = 0; i < fieldSizeCalculator.size(); i++) { + size += + fieldSizeCalculator + .get(i) + .apply(new FlinkRowWrapper(keyAndRowData.f1), i); + } + return new Tuple2<>(keyAndRowData.f0, size); + } else { + // when basing on quantity, we don't need the size of the data, so setting it to a + // constant of 1 would be sufficient. + return new Tuple2<>(keyAndRowData.f0, 1); + } + } + } + /** * LocalSampleOperator wraps the sample logic on the partition side (the first phase of * distributed sample algorithm). Outputs sampled weight with record. @@ -162,15 +217,17 @@ public static DataStream> rangeShuffleByKey( *

See {@link Sampler}. */ @Internal - public static class LocalSampleOperator extends TableStreamOperator> - implements OneInputStreamOperator>, BoundedOneInput { + public static class LocalSampleOperator + extends TableStreamOperator> + implements OneInputStreamOperator, Tuple3>, + BoundedOneInput { private static final long serialVersionUID = 1L; private final int numSample; - private transient Collector> collector; - private transient Sampler sampler; + private transient Collector> collector; + private transient Sampler> sampler; public LocalSampleOperator(int numSample) { this.numSample = numSample; @@ -184,15 +241,17 @@ public void open() throws Exception { } @Override - public void processElement(StreamRecord streamRecord) throws Exception { + public void processElement(StreamRecord> streamRecord) throws Exception { sampler.collect(streamRecord.getValue()); } @Override - public void endInput() throws Exception { - Iterator> sampled = sampler.sample(); + public void endInput() { + Iterator>> sampled = sampler.sample(); while (sampled.hasNext()) { - collector.collect(sampled.next()); + Tuple2> next = sampled.next(); + + collector.collect(new Tuple3<>(next.f0, next.f1.f0, next.f1.f1)); } } } @@ -203,7 +262,8 @@ public void endInput() throws Exception { *

See {@link Sampler}. */ private static class GlobalSampleOperator extends TableStreamOperator> - implements OneInputStreamOperator, List>, BoundedOneInput { + implements OneInputStreamOperator, List>, + BoundedOneInput { private static final long serialVersionUID = 1L; @@ -213,7 +273,7 @@ private static class GlobalSampleOperator extends TableStreamOperator private transient Comparator keyComparator; private transient Collector> collector; - private transient Sampler sampler; + private transient Sampler> sampler; public GlobalSampleOperator( int numSample, @@ -233,35 +293,32 @@ public void open() throws Exception { } @Override - public void processElement(StreamRecord> record) throws Exception { - Tuple2 tuple = record.getValue(); - sampler.collect(tuple.f0, tuple.f1); + public void processElement(StreamRecord> record) + throws Exception { + Tuple3 tuple = record.getValue(); + sampler.collect(tuple.f0, new Tuple2<>(tuple.f1, tuple.f2)); } @Override - public void endInput() throws Exception { - Iterator> sampled = sampler.sample(); + public void endInput() { + Iterator>> sampled = sampler.sample(); + + List> sampledData = new ArrayList<>(); - List sampledData = new ArrayList<>(); while (sampled.hasNext()) { sampledData.add(sampled.next().f1); } - sampledData.sort(keyComparator); + sampledData.sort((o1, o2) -> keyComparator.compare(o1.f0, o2.f0)); - int boundarySize = rangesNum - 1; - @SuppressWarnings("unchecked") - T[] boundaries = (T[]) new Object[boundarySize]; - if (sampledData.size() > 0) { - double avgRange = sampledData.size() / (double) rangesNum; - for (int i = 1; i < rangesNum; i++) { - T record = sampledData.get((int) (i * avgRange)); - boundaries[i - 1] = record; - } - collector.collect(Arrays.asList(boundaries)); + List range; + if (sampledData.isEmpty()) { + range = new ArrayList<>(); } else { - collector.collect(Collections.emptyList()); + range = Arrays.asList(allocateRangeBaseSize(sampledData, rangesNum)); } + + collector.collect(range); } } @@ -488,4 +545,40 @@ public int get() { return list.get(RANDOM.nextInt(list.size())); } } + + @VisibleForTesting + static T[] allocateRangeBaseSize(List> sampledData, int rangesNum) { + int sampeNum = sampledData.size(); + int boundarySize = rangesNum - 1; + @SuppressWarnings("unchecked") + T[] boundaries = (T[]) new Object[boundarySize]; + + if (!sampledData.isEmpty()) { + long restSize = sampledData.stream().mapToLong(t -> (long) t.f1).sum(); + double stepRange = restSize / (double) rangesNum; + + int currentWeight = 0; + int index = 0; + + for (int i = 0; i < boundarySize; i++) { + while (currentWeight < stepRange && index < sampeNum) { + boundaries[i] = sampledData.get(Math.min(index, sampeNum - 1)).f0; + int sampleWeight = sampledData.get(index++).f1; + currentWeight += sampleWeight; + restSize -= sampleWeight; + } + + currentWeight = 0; + stepRange = restSize / (double) (rangesNum - i - 1); + } + } + + for (int i = 0; i < boundarySize; i++) { + if (boundaries[i] == null) { + boundaries[i] = sampledData.get(sampeNum - 1).f0; + } + } + + return boundaries; + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java index 4d049ff7241e..c31aaa1a4c78 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java @@ -146,7 +146,9 @@ public Tuple2 map(RowData value) { keyTypeInformation, sampleSize, rangeNum, - sinkParallelism) + sinkParallelism, + valueRowType, + options.sortBySize()) .map( a -> new JoinedRow(convertor.apply(a.f0), new FlinkRowWrapper(a.f1)), internalRowType) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java index ee43ca58e8a3..272b3516a3b8 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java @@ -331,30 +331,35 @@ public void testSortCompactionOnEmptyData() throws Exception { } private void zorder(List columns) throws Exception { + String rangeStrategy = RANDOM.nextBoolean() ? "size" : "quantity"; if (RANDOM.nextBoolean()) { - createAction("zorder", columns).run(); + createAction("zorder", rangeStrategy, columns).run(); } else { - callProcedure("zorder", columns); + callProcedure("zorder", rangeStrategy, columns); } } private void hilbert(List columns) throws Exception { + String rangeStrategy = RANDOM.nextBoolean() ? "size" : "quantity"; if (RANDOM.nextBoolean()) { - createAction("hilbert", columns).run(); + createAction("hilbert", rangeStrategy, columns).run(); } else { - callProcedure("hilbert", columns); + callProcedure("hilbert", rangeStrategy, columns); } } private void order(List columns) throws Exception { + String rangeStrategy = RANDOM.nextBoolean() ? "size" : "quantity"; if (RANDOM.nextBoolean()) { - createAction("order", columns).run(); + createAction("order", rangeStrategy, columns).run(); } else { - callProcedure("order", columns); + callProcedure("order", rangeStrategy, columns); } } - private SortCompactAction createAction(String orderStrategy, List columns) { + private SortCompactAction createAction( + String orderStrategy, String rangeStrategy, List columns) { + return createAction( SortCompactAction.class, "compact", @@ -367,14 +372,21 @@ private SortCompactAction createAction(String orderStrategy, List column "--order_strategy", orderStrategy, "--order_by", - String.join(",", columns)); + String.join(",", columns), + "--table_conf sort-compaction.range-strategy=" + rangeStrategy, + rangeStrategy); } - private void callProcedure(String orderStrategy, List orderByColumns) { + private void callProcedure( + String orderStrategy, String rangeStrategy, List orderByColumns) { callProcedure( String.format( - "CALL sys.compact('%s.%s', 'ALL', '%s', '%s')", - database, tableName, orderStrategy, String.join(",", orderByColumns)), + "CALL sys.compact('%s.%s', 'ALL', '%s', '%s','sort-compaction.range-strategy=%s')", + database, + tableName, + orderStrategy, + String.join(",", orderByColumns), + rangeStrategy), false, true); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/shuffle/RangeShuffleTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/shuffle/RangeShuffleTest.java new file mode 100644 index 000000000000..cb79a8628180 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/shuffle/RangeShuffleTest.java @@ -0,0 +1,75 @@ +/* + * 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.shuffle; + +import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; + +/** Test for {@link RangeShuffle}. */ +class RangeShuffleTest { + + @Test + void testAllocateRange() { + + // the size of test data is even + List> test0 = + Lists.newArrayList( + // key and size + new Tuple2<>(1, 1), + new Tuple2<>(2, 1), + new Tuple2<>(3, 1), + new Tuple2<>(4, 1), + new Tuple2<>(5, 1), + new Tuple2<>(6, 1)); + Assertions.assertEquals( + "[2, 4]", Arrays.deepToString(RangeShuffle.allocateRangeBaseSize(test0, 3))); + + // the size of test data is uneven,but can be evenly split based size + List> test2 = + Lists.newArrayList( + new Tuple2<>(1, 1), + new Tuple2<>(2, 1), + new Tuple2<>(3, 1), + new Tuple2<>(4, 1), + new Tuple2<>(5, 4), + new Tuple2<>(6, 4), + new Tuple2<>(7, 4)); + Assertions.assertEquals( + "[4, 5, 6]", Arrays.deepToString(RangeShuffle.allocateRangeBaseSize(test2, 4))); + + // the size of test data is uneven,and can not be evenly split + List> test1 = + Lists.newArrayList( + new Tuple2<>(1, 1), + new Tuple2<>(2, 2), + new Tuple2<>(3, 3), + new Tuple2<>(4, 1), + new Tuple2<>(5, 2), + new Tuple2<>(6, 3)); + + Assertions.assertEquals( + "[3, 5]", Arrays.deepToString(RangeShuffle.allocateRangeBaseSize(test1, 3))); + } +} From cc1cbc87a0bfb2350ca7bfddbaab57e45a8e693d Mon Sep 17 00:00:00 2001 From: chao chen <59957056+waywtdcc@users.noreply.github.com> Date: Mon, 18 Mar 2024 15:32:50 +0800 Subject: [PATCH 21/79] [spark] spark sql support get/list function (#2880) --- .../org/apache/paimon/spark/SparkGenericCatalog.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java index 931b1f1923d1..3bdff73762cb 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java @@ -39,6 +39,7 @@ import org.apache.spark.sql.connector.catalog.CatalogExtension; import org.apache.spark.sql.connector.catalog.CatalogPlugin; import org.apache.spark.sql.connector.catalog.CatalogUtils; +import org.apache.spark.sql.connector.catalog.FunctionCatalog; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.NamespaceChange; import org.apache.spark.sql.connector.catalog.SupportsNamespaces; @@ -316,18 +317,22 @@ private SupportsNamespaces asNamespaceCatalog() { return (SupportsNamespaces) sessionCatalog; } + private FunctionCatalog asFunctionCatalog() { + return (FunctionCatalog) sessionCatalog; + } + @Override public Identifier[] listFunctions(String[] namespace) throws NoSuchNamespaceException { if (namespace.length == 0 || isSystemNamespace(namespace) || namespaceExists(namespace)) { return new Identifier[0]; } - throw new NoSuchNamespaceException(namespace); + return asFunctionCatalog().listFunctions(namespace); } @Override public UnboundFunction loadFunction(Identifier ident) throws NoSuchFunctionException { - throw new NoSuchFunctionException(ident); + return asFunctionCatalog().loadFunction(ident); } private static boolean isSystemNamespace(String[] namespace) { From fba132e3f8bb67f7a55392c5f3926a8cdf263105 Mon Sep 17 00:00:00 2001 From: YeJunHao <41894543+leaves12138@users.noreply.github.com> Date: Mon, 18 Mar 2024 16:10:50 +0800 Subject: [PATCH 22/79] [docs] Trino docs update for sharing file system. (#3035) --- docs/content/engines/overview.md | 7 +-- docs/content/engines/trino.md | 61 ++++++++++---------------- docs/content/filesystems/oss.md | 14 +++--- docs/content/filesystems/s3.md | 9 +--- docs/content/how-to/altering-tables.md | 2 +- docs/content/how-to/querying-tables.md | 12 +---- docs/content/project/download.md | 7 +-- 7 files changed, 39 insertions(+), 73 deletions(-) diff --git a/docs/content/engines/overview.md b/docs/content/engines/overview.md index 05aa69e86e54..31c5eb7a784a 100644 --- a/docs/content/engines/overview.md +++ b/docs/content/engines/overview.md @@ -38,10 +38,11 @@ Apache Spark and Apache Hive. | Spark | 3.1 - 3.5 | ✅ | ✅ | ✅ | ✅ | ✅ | ✅(3.3+) | ✅ | | Hive | 2.1 - 3.1 | ✅ | ✅ | ✅ | ❌ | ❌ | ❌ | ❌ | | Spark | 2.4 | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | -| Trino | 358 - 422 | ✅ | ❌ | ✅ | ✅ | ❌ | ❌ | ❌ | +| Trino | 422 - 426 | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | +| Trino | 427 - 439 | ✅ | ❌ | ✅ | ✅ | ❌ | ❌ | ❌ | | Presto | 0.236 - 0.280 | ✅ | ❌ | ✅ | ✅ | ❌ | ❌ | ❌ | -| [StarRocks](https://docs.starrocks.io/docs/data_source/catalog/paimon_catalog/) | 3.1+ | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | -| [Doris](https://doris.apache.org/docs/lakehouse/multi-catalog/paimon/) | 2.0+ | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | +| [StarRocks](https://docs.starrocks.io/docs/data_source/catalog/paimon_catalog/) | 3.1+ | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | +| [Doris](https://doris.apache.org/docs/lakehouse/multi-catalog/paimon/) | 2.0+ | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | Recommended versions are Flink 1.17.2, Spark 3.5.0, Hive 2.3.9 diff --git a/docs/content/engines/trino.md b/docs/content/engines/trino.md index 1a6e06d2e48c..9c2bdfd3f91e 100644 --- a/docs/content/engines/trino.md +++ b/docs/content/engines/trino.md @@ -30,7 +30,13 @@ This documentation is a guide for using Paimon in Trino. ## Version -Paimon currently supports Trino 358 and above. +Paimon currently supports Trino 422 and above. + +## Filesystem + +From version 0.8, paimon share trino filesystem for all actions, which means, iyou should +config trino filesystem before using trino-paimon. You can find information about how to config +filesystems for trino on trino official website. ## Preparing Paimon Jar File @@ -43,38 +49,17 @@ https://paimon.apache.org/docs/master/project/download/ {{< unstable >}} -| Version | Package | -|------------|-----------------------------------------------------------------------------------------------------------------------------------------------| -| [358, 368) | [paimon-trino-358-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-358/{{< version >}}/) | -| [368, 369) | [paimon-trino-368-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-368/{{< version >}}/) | -| [369, 370) | [paimon-trino-369-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-369/{{< version >}}/) | -| [370, 388) | [paimon-trino-370-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-370/{{< version >}}/) | -| [388, 393) | [paimon-trino-388-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-388/{{< version >}}/) | -| [393, 422] | [paimon-trino-393-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-393/{{< version >}}/) | -| [422, latest] | [paimon-trino-422-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-422/{{< version >}}/) | +| Version | Package | +|---------------|-----------------------------------------------------------------------------------------------------------------------------------------------| +| [422, 427] | [paimon-trino-422-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-422/{{< version >}}/) | +| [427, latest] | [paimon-trino-427-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-427/{{< version >}}/) | {{< /unstable >}} You can also manually build a bundled jar from the source code. However, there are a few preliminary steps that need to be taken before compiling: - To build from the source code, [clone the git repository]({{< trino_github_repo >}}). -- Install JDK11 and JDK17 locally, and configure JDK11 as a global environment variable; -- Configure the toolchains.xml file in ${{ MAVEN_HOME }}, the content is as follows. - -``` - - - jdk - - 17 - adopt - - - ${{ JAVA_HOME }} - - - -``` +- Install JDK17 locally, and configure JDK17 as a global environment variable; Then,you can build bundled jar with the following command: @@ -111,7 +96,7 @@ Let Paimon use a secure temporary directory. ```bash tar -zxf paimon-trino--{{< version >}}-plugin.tar.gz -C ${TRINO_HOME}/plugin ``` -the variable `trino-version` is module name, must be one of 358, 368, 369, 370, 388, 393, 422. +the variable `trino-version` is module name, must be one of 422, 427. > NOTE: For JDK 17, when Deploying Trino, should add jvm options: `--add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED` ### Configure @@ -128,6 +113,15 @@ If you are using HDFS, choose one of the following ways to configure your HDFS: - set environment variable HADOOP_CONF_DIR. - configure `hadoop-conf-dir` in the properties. +If you are using a hadoop filesystem, you can still use trino-hdfs and trino-hive to config it. +For example, if you use oss as a storage, you can write in `paimon.properties` according to [Trino Reference](https://trino.io/docs/current/connector/hive.html#hdfs-configuration): + +``` +hive.config.resources=/path/to/core-site.xml +``` + +Then, config core-site.xml according to [Jindo Reference](https://github.com/aliyun/alibabacloud-jindodata/blob/master/docs/user/4.x/4.6.x/4.6.12/oss/presto/jindosdk_on_presto.md) + ## Kerberos You can configure kerberos keytab file when using KERBEROS authentication in the properties. @@ -194,7 +188,7 @@ SELECT * FROM paimon.test_db.orders ## Query with Time Traveling {{< tabs "time-travel-example" >}} -{{< tab "version >=368" >}} +{{< tab "version >=422" >}} ```sql -- read the snapshot from specified timestamp @@ -206,15 +200,6 @@ SELECT * FROM t FOR VERSION AS OF 1; {{< /tab >}} -{{< tab "version < 368" >}} - -```sql --- read the snapshot from specified timestamp with a long value in unix milliseconds -SET SESSION paimon.scan_timestamp_millis=1679486589444; -SELECT * FROM t; -``` - -{{< /tab >}} {{< /tabs >}} diff --git a/docs/content/filesystems/oss.md b/docs/content/filesystems/oss.md index 472d81f1f6cb..c2cd98848320 100644 --- a/docs/content/filesystems/oss.md +++ b/docs/content/filesystems/oss.md @@ -106,14 +106,14 @@ SELECT COUNT(1) FROM test_table; {{< tab "Trino" >}} -Place `paimon-oss-{{< version >}}.jar` together with `paimon-trino-{{< version >}}.jar` under `plugin/paimon` directory. +From version 0.8, paimon-trino use trino filesystem as basic file read and write system. We strongly recommend you to use jindo-sdk in trino. + +You can find [How to config jindo sdk on trino](https://github.com/aliyun/alibabacloud-jindodata/blob/master/docs/user/4.x/4.6.x/4.6.12/oss/presto/jindosdk_on_presto.md) here. +Please note that: + * Use paimon to replace hive-hadoop2 when you decompress the plugin jar and find location to put in. + * You can specify the `core-site.xml` in `paimon.properties` on configuration [hive.config.resources](https://trino.io/docs/current/connector/hive.html#hdfs-configuration). + * Presto and Jindo are the in the same configaration method. -Add options in `etc/catalog/paimon.properties`. -```shell -fs.oss.endpoint=oss-cn-hangzhou.aliyuncs.com -fs.oss.accessKeyId=xxx -fs.oss.accessKeySecret=yyy -``` {{< /tab >}} diff --git a/docs/content/filesystems/s3.md b/docs/content/filesystems/s3.md index d1334fda8b40..23d96be571ef 100644 --- a/docs/content/filesystems/s3.md +++ b/docs/content/filesystems/s3.md @@ -106,14 +106,9 @@ SELECT COUNT(1) FROM test_table; {{< tab "Trino" >}} -Place `paimon-s3-{{< version >}}.jar` together with `paimon-trino-{{< version >}}.jar` under `plugin/paimon` directory. +Paimon use shared trino filesystem as basic read and write system. -Add options in `etc/catalog/paimon.properties`. -```shell -s3.endpoint=your-endpoint-hostname -s3.access-key=xxx -s3.secret-key=yyy -``` +Please refer to [Trino S3](https://trino.io/docs/current/object-storage/file-system-s3.html) to config s3 filesystem in trino. {{< /tab >}} diff --git a/docs/content/how-to/altering-tables.md b/docs/content/how-to/altering-tables.md index ce9ca73d7991..5a6045cbaa79 100644 --- a/docs/content/how-to/altering-tables.md +++ b/docs/content/how-to/altering-tables.md @@ -58,7 +58,7 @@ ALTER TABLE my_table SET TBLPROPERTIES ( ALTER TABLE my_table SET PROPERTIES write_buffer_size = '256 MB'; ``` -> NOTE: Versions below Trino 368 do not support changing/adding table properties. +> NOTE: Versions below Trino 427 do not support changing/adding table properties. {{< /tab >}} diff --git a/docs/content/how-to/querying-tables.md b/docs/content/how-to/querying-tables.md index de74c7c965cd..ce6c74521c0b 100644 --- a/docs/content/how-to/querying-tables.md +++ b/docs/content/how-to/querying-tables.md @@ -123,17 +123,7 @@ spark.read {{< /tab >}} -{{< tab "Trino" >}} - -```sql --- read the snapshot from specified timestamp with a long value in unix milliseconds -SET SESSION paimon.scan_timestamp_millis=1679486589444; -SELECT * FROM t; -``` - -{{< /tab >}} - -{{< tab "Trino 368+" >}} +{{< tab "Trino 422+" >}} ```sql -- read the snapshot from specified timestamp diff --git a/docs/content/project/download.md b/docs/content/project/download.md index 4d8bc1fc99ed..9b39cc8eeaee 100644 --- a/docs/content/project/download.md +++ b/docs/content/project/download.md @@ -53,13 +53,8 @@ This documentation is a guide for downloading Paimon Jars. | Presto 0.268 | [paimon-presto-0.268-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-presto-0.268/{{< version >}}/) | | Presto 0.273 | [paimon-presto-0.273-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-presto-0.273/{{< version >}}/) | | Presto SQL 332 | [paimon-prestosql-332-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-prestosql-332/{{< version >}}/) | -| Trino 358 | [paimon-trino-358-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-358/{{< version >}}/) | -| Trino 368 | [paimon-trino-368-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-368/{{< version >}}/) | -| Trino 369 | [paimon-trino-369-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-369/{{< version >}}/) | -| Trino 370 | [paimon-trino-370-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-370/{{< version >}}/) | -| Trino 388 | [paimon-trino-388-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-388/{{< version >}}/) | -| Trino 393 | [paimon-trino-393-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-393/{{< version >}}/) | | Trino 422 | [paimon-trino-422-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-422/{{< version >}}/) | +| Trino 427 | [paimon-trino-427-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-427/{{< version >}}/) | {{< /unstable >}} From b6dd27a91fd3b40dcf0621ec86a485be326cc321 Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Mon, 18 Mar 2024 17:02:15 +0800 Subject: [PATCH 23/79] [core] Fix dv table with partial-update and aggregate (#3036) --- .../operation/KeyValueFileStoreWrite.java | 10 ++- .../paimon/schema/SchemaValidation.java | 3 +- .../paimon/flink/DeletionVectorITCase.java | 89 ++++++++++++++++++- 3 files changed, 94 insertions(+), 8 deletions(-) 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 e06ed9ea26b6..6f74e67208de 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 @@ -80,6 +80,7 @@ import java.util.function.Supplier; import static org.apache.paimon.CoreOptions.ChangelogProducer.FULL_COMPACTION; +import static org.apache.paimon.CoreOptions.MergeEngine.DEDUPLICATE; import static org.apache.paimon.CoreOptions.MergeEngine.FIRST_ROW; import static org.apache.paimon.io.DataFileMeta.getMaxSequenceNumber; import static org.apache.paimon.lookup.LookupStoreFactory.bfGenerator; @@ -255,8 +256,6 @@ private MergeTreeCompactRewriter createRewriter( @Nullable FieldsComparator userDefinedSeqComparator, Levels levels, @Nullable DeletionVectorsMaintainer dvMaintainer) { - KeyValueFileReaderFactory.Builder readerFactoryBuilder = - this.readerFactoryBuilder.copyWithoutProjection(); DeletionVector.Factory dvFactory = DeletionVector.factory(dvMaintainer); KeyValueFileReaderFactory readerFactory = readerFactoryBuilder.build(partition, bucket, dvFactory); @@ -286,10 +285,11 @@ private MergeTreeCompactRewriter createRewriter( if (mergeEngine == FIRST_ROW) { if (options.deletionVectorsEnabled()) { throw new UnsupportedOperationException( - "Deletion vectors mode is not supported for first row merge engine now."); + "First row merge engine does not need deletion vectors because there is no deletion of old data in this merge engine."); } lookupReaderFactory = readerFactoryBuilder + .copyWithoutProjection() .withValueProjection(new int[0][]) .build(partition, bucket, dvFactory); processor = new ContainsValueProcessor(); @@ -298,7 +298,9 @@ private MergeTreeCompactRewriter createRewriter( processor = lookupStrategy.deletionVector ? new PositionedKeyValueProcessor( - valueType, lookupStrategy.produceChangelog) + valueType, + lookupStrategy.produceChangelog + || mergeEngine != DEDUPLICATE) : new KeyValueProcessor(valueType); wrapperFactory = new LookupMergeFunctionWrapperFactory<>( 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 cbdcb64e5a7a..cb52570cd2da 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 @@ -463,10 +463,9 @@ private static void validateForDeletionVectors(TableSchema schema, CoreOptions o || options.changelogProducer() == ChangelogProducer.LOOKUP, "Deletion vectors mode is only supported for none or lookup changelog producer now."); - // todo: implement it checkArgument( !options.mergeEngine().equals(MergeEngine.FIRST_ROW), - "Deletion vectors mode is not supported for first row merge engine now."); + "First row merge engine does not need deletion vectors because there is no deletion of old data in this merge engine."); } private static void validateSequenceField(TableSchema schema, CoreOptions options) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/DeletionVectorITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/DeletionVectorITCase.java index 162e29eece60..1fe424b331e5 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/DeletionVectorITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/DeletionVectorITCase.java @@ -48,7 +48,7 @@ public void testStreamingReadDVTable(String changelogProducer) throws Exception // test read from APPEND snapshot try (BlockingIterator iter = streamSqlBlockIter( - "SELECT * FROM T /*+ OPTIONS('scan.mode'='from-snapshot-full','scan.snapshot-id' = '3') */"); ) { + "SELECT * FROM T /*+ OPTIONS('scan.mode'='from-snapshot-full','scan.snapshot-id' = '3') */")) { assertThat(iter.collect(12)) .containsExactlyInAnyOrder( Row.ofKind(RowKind.INSERT, 1, "111111111"), @@ -68,7 +68,7 @@ public void testStreamingReadDVTable(String changelogProducer) throws Exception // test read from COMPACT snapshot try (BlockingIterator iter = streamSqlBlockIter( - "SELECT * FROM T /*+ OPTIONS('scan.mode'='from-snapshot-full','scan.snapshot-id' = '4') */"); ) { + "SELECT * FROM T /*+ OPTIONS('scan.mode'='from-snapshot-full','scan.snapshot-id' = '4') */")) { assertThat(iter.collect(8)) .containsExactlyInAnyOrder( Row.ofKind(RowKind.INSERT, 1, "111111111"), @@ -113,4 +113,89 @@ public void testBatchReadDVTable(String changelogProducer) { .containsExactlyInAnyOrder( Row.of(1, "111111111"), Row.of(2, "2_1"), Row.of(3, "3_1"), Row.of(4, "4")); } + + @ParameterizedTest + @ValueSource(strings = {"none", "lookup"}) + public void testDVTableWithAggregationMergeEngine(String changelogProducer) throws Exception { + sql( + String.format( + "CREATE TABLE T (id INT PRIMARY KEY NOT ENFORCED, v INT) " + + "WITH ('deletion-vectors.enabled' = 'true', 'changelog-producer' = '%s', " + + "'merge-engine'='aggregation', 'fields.v.aggregate-function'='sum')", + changelogProducer)); + + sql("INSERT INTO T VALUES (1, 111111111), (2, 2), (3, 3), (4, 4)"); + + sql("INSERT INTO T VALUES (2, 1), (3, 1)"); + + sql("INSERT INTO T VALUES (2, 1), (4, 1)"); + + // test batch read + assertThat(batchSql("SELECT * FROM T")) + .containsExactlyInAnyOrder( + Row.of(1, 111111111), Row.of(2, 4), Row.of(3, 4), Row.of(4, 5)); + + // test streaming read + if (changelogProducer.equals("lookup")) { + try (BlockingIterator iter = + streamSqlBlockIter( + "SELECT * FROM T /*+ OPTIONS('scan.mode'='from-snapshot-full','scan.snapshot-id' = '4') */")) { + assertThat(iter.collect(8)) + .containsExactlyInAnyOrder( + Row.ofKind(RowKind.INSERT, 1, 111111111), + Row.ofKind(RowKind.INSERT, 2, 3), + Row.ofKind(RowKind.INSERT, 3, 4), + Row.ofKind(RowKind.INSERT, 4, 4), + Row.ofKind(RowKind.UPDATE_BEFORE, 2, 3), + Row.ofKind(RowKind.UPDATE_AFTER, 2, 4), + Row.ofKind(RowKind.UPDATE_BEFORE, 4, 4), + Row.ofKind(RowKind.UPDATE_AFTER, 4, 5)); + } + } + } + + @ParameterizedTest + @ValueSource(strings = {"none", "lookup"}) + public void testDVTableWithPartialUpdateMergeEngine(String changelogProducer) throws Exception { + sql( + String.format( + "CREATE TABLE T (id INT PRIMARY KEY NOT ENFORCED, v1 STRING, v2 STRING) " + + "WITH ('deletion-vectors.enabled' = 'true', 'changelog-producer' = '%s', " + + "'merge-engine'='partial-update')", + changelogProducer)); + + sql( + "INSERT INTO T VALUES (1, '111111111', '1'), (2, '2', CAST(NULL AS STRING)), (3, '3', '3'), (4, CAST(NULL AS STRING), '4')"); + + sql("INSERT INTO T VALUES (2, CAST(NULL AS STRING), '2'), (3, '3_1', '3_1')"); + + sql( + "INSERT INTO T VALUES (2, '2_1', CAST(NULL AS STRING)), (4, '4', CAST(NULL AS STRING))"); + + // test batch read + assertThat(batchSql("SELECT * FROM T")) + .containsExactlyInAnyOrder( + Row.of(1, "111111111", "1"), + Row.of(2, "2_1", "2"), + Row.of(3, "3_1", "3_1"), + Row.of(4, "4", "4")); + + // test streaming read + if (changelogProducer.equals("lookup")) { + try (BlockingIterator iter = + streamSqlBlockIter( + "SELECT * FROM T /*+ OPTIONS('scan.mode'='from-snapshot-full','scan.snapshot-id' = '4') */")) { + assertThat(iter.collect(8)) + .containsExactlyInAnyOrder( + Row.ofKind(RowKind.INSERT, 1, "111111111", "1"), + Row.ofKind(RowKind.INSERT, 2, "2", "2"), + Row.ofKind(RowKind.INSERT, 3, "3_1", "3_1"), + Row.ofKind(RowKind.INSERT, 4, null, "4"), + Row.ofKind(RowKind.UPDATE_BEFORE, 2, "2", "2"), + Row.ofKind(RowKind.UPDATE_AFTER, 2, "2_1", "2"), + Row.ofKind(RowKind.UPDATE_BEFORE, 4, null, "4"), + Row.ofKind(RowKind.UPDATE_AFTER, 4, "4", "4")); + } + } + } } From 2df0c1e4b33ec3e7c1d2ccc8a01fa816912423c0 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Mon, 18 Mar 2024 19:05:48 +0800 Subject: [PATCH 24/79] [core] Introduce BatchTableCommit.truncateTable (#3037) --- .../benchmark/TableWriterBenchmark.java | 56 +++++++++---------- .../paimon/operation/FileStoreCommit.java | 2 +- .../paimon/operation/FileStoreCommitImpl.java | 2 +- .../paimon/table/sink/BatchTableCommit.java | 6 ++ .../paimon/table/sink/TableCommitImpl.java | 13 ++++- .../paimon/flink/sink/FlinkTableSink.java | 10 +--- ...pportsRowLevelOperationFlinkTableSink.java | 2 +- .../DeleteFromPaimonTableCommand.scala | 2 +- .../commands/PaimonTruncateTableCommand.scala | 2 +- 9 files changed, 52 insertions(+), 43 deletions(-) diff --git a/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableWriterBenchmark.java b/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableWriterBenchmark.java index 5ced4248bffa..04d1d7342751 100644 --- a/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableWriterBenchmark.java +++ b/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableWriterBenchmark.java @@ -20,14 +20,13 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.options.Options; -import org.apache.paimon.table.sink.CommitMessage; -import org.apache.paimon.table.sink.StreamTableCommit; -import org.apache.paimon.table.sink.StreamTableWrite; -import org.apache.paimon.table.sink.StreamWriteBuilder; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.sink.BatchTableCommit; +import org.apache.paimon.table.sink.BatchTableWrite; +import org.apache.paimon.table.sink.BatchWriteBuilder; import org.junit.jupiter.api.Test; -import java.util.List; import java.util.concurrent.atomic.AtomicInteger; /** Benchmark for table writer. */ @@ -41,9 +40,9 @@ public void testAvro() throws Exception { /* * Java HotSpot(TM) 64-Bit Server VM 1.8.0_301-b09 on Mac OS X 10.16 * Intel(R) Core(TM) i7-9750H CPU @ 2.60GHz - * avro: Best/Avg Time(ms) Row Rate(M/s) Per Row(ns) Relative + * avro: Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative * --------------------------------------------------------------------------------- - * avro_write 5847 / 7296 0.1 19489.5 1.0X + * avro_write 40309 / 41161 74.4 13436.3 1.0X */ } @@ -56,9 +55,9 @@ public void testAvroWithoutStats() throws Exception { /* * Java HotSpot(TM) 64-Bit Server VM 1.8.0_301-b09 on Mac OS X 10.16 * Intel(R) Core(TM) i7-9750H CPU @ 2.60GHz - * avro: Best/Avg Time(ms) Row Rate(M/s) Per Row(ns) Relative + * avro: Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative * --------------------------------------------------------------------------------- - * avro_write 4701 / 5780 0.1 15669.6 1.0X + * avro_write 31817 / 32359 94.3 10605.6 1.0X */ } @@ -71,9 +70,9 @@ public void testOrcNoCompression() throws Exception { /* * Java HotSpot(TM) 64-Bit Server VM 1.8.0_301-b09 on Mac OS X 10.16 * Intel(R) Core(TM) i7-9750H CPU @ 2.60GHz - * orc: Best/Avg Time(ms) Row Rate(M/s) Per Row(ns) Relative + * orc: Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative * --------------------------------------------------------------------------------- - * orc_write 8448 / 9584 0.0 28160.1 1.0X + * orc_write 32751 / 33032 91.6 10917.0 1.0X */ } @@ -85,9 +84,9 @@ public void testParquet() throws Exception { /* * Java HotSpot(TM) 64-Bit Server VM 1.8.0_301-b09 on Mac OS X 10.16 * Intel(R) Core(TM) i7-9750H CPU @ 2.60GHz - * parquet: Best/Avg Time(ms) Row Rate(M/s) Per Row(ns) Relative + * parquet: Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative * --------------------------------------------------------------------------------- - * parquet_write 10872 / 12566 0.0 36239.7 1.0X + * parquet_write 46279 / 46715 64.8 15426.3 1.0X */ } @@ -99,44 +98,45 @@ public void testOrc() throws Exception { /* * Java HotSpot(TM) 64-Bit Server VM 1.8.0_301-b09 on Mac OS X 10.16 * Intel(R) Core(TM) i7-9750H CPU @ 2.60GHz - * orc: Best/Avg Time(ms) Row Rate(M/s) Per Row(ns) Relative + * orc: Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative * --------------------------------------------------------------------------------- - * orc_write 8690 / 9771 0.0 28968.0 1.0X + * orc_write 36489 / 36697 82.2 12163.1 1.0X */ } public void innerTest(String name, Options options) throws Exception { options.set(CoreOptions.BUCKET, 1); - StreamWriteBuilder writeBuilder = createTable(options, "T").newStreamWriteBuilder(); - StreamTableWrite write = writeBuilder.newWrite(); - StreamTableCommit commit = writeBuilder.newCommit(); - long valuesPerIteration = 300_000; + Table table = createTable(options, "T"); + long valuesPerIteration = 3_000_000; Benchmark benchmark = new Benchmark(name, valuesPerIteration) .setNumWarmupIters(1) .setOutputPerIteration(true); AtomicInteger writeCount = new AtomicInteger(0); - AtomicInteger commitIdentifier = new AtomicInteger(0); benchmark.addCase( "write", - 5, + 3, () -> { + BatchWriteBuilder writeBuilder = table.newBatchWriteBuilder(); + BatchTableWrite write = writeBuilder.newWrite(); + BatchTableCommit commit = writeBuilder.newCommit(); for (int i = 0; i < valuesPerIteration; i++) { try { write.write(newRandomRow()); writeCount.incrementAndGet(); - if (writeCount.get() % 10_000 == 0) { - List commitMessages = - write.prepareCommit(false, commitIdentifier.get()); - commit.commit(commitIdentifier.get(), commitMessages); - commitIdentifier.incrementAndGet(); - } } catch (Exception e) { throw new RuntimeException(e); } } + try { + commit.commit(write.prepareCommit()); + writeBuilder.newCommit().truncateTable(); + write.close(); + commit.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } }); benchmark.run(); - write.close(); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommit.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommit.java index fad4c2b70cfe..7151f25124b1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommit.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommit.java @@ -78,7 +78,7 @@ void overwrite( */ void dropPartitions(List> partitions, long commitIdentifier); - void purgeTable(long commitIdentifier); + void truncateTable(long commitIdentifier); /** Abort an unsuccessful commit. The data files will be deleted. */ void abort(List commitMessages); diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java index 46cfceb145c7..d3ce76b0a65d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java @@ -493,7 +493,7 @@ public void dropPartitions(List> partitions, long commitIden } @Override - public void purgeTable(long commitIdentifier) { + public void truncateTable(long commitIdentifier) { tryOverwrite( null, Collections.emptyList(), diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/BatchTableCommit.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/BatchTableCommit.java index 894aec3e5eb6..f0c9b59e3178 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/BatchTableCommit.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/BatchTableCommit.java @@ -50,4 +50,10 @@ public interface BatchTableCommit extends TableCommit { * @param commitMessages commit messages from table write */ void commit(List commitMessages); + + /** + * Truncate table, like normal {@link #commit}, files are not immediately deleted, they are only + * logically deleted and will be deleted after the snapshot expires. + */ + void truncateTable(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/TableCommitImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/TableCommitImpl.java index ab01943bbb60..c76b750a11b2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/TableCommitImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/TableCommitImpl.java @@ -65,6 +65,7 @@ import java.util.stream.Collectors; import static org.apache.paimon.CoreOptions.ExpireExecutionMode; +import static org.apache.paimon.table.sink.BatchWriteBuilder.COMMIT_IDENTIFIER; import static org.apache.paimon.utils.Preconditions.checkState; /** An abstraction layer above {@link FileStoreCommit} to provide snapshot commit and expiration. */ @@ -159,9 +160,19 @@ public Set filterCommitted(Set commitIdentifiers) { @Override public void commit(List commitMessages) { + checkCommitted(); + commit(COMMIT_IDENTIFIER, commitMessages); + } + + @Override + public void truncateTable() { + checkCommitted(); + commit.truncateTable(COMMIT_IDENTIFIER); + } + + private void checkCommitted() { checkState(!batchCommitted, "BatchTableCommit only support one-time committing."); batchCommitted = true; - commit(BatchWriteBuilder.COMMIT_IDENTIFIER, commitMessages); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java index 50bc45b752f8..b1211b0e738c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java @@ -19,10 +19,7 @@ package org.apache.paimon.flink.sink; import org.apache.paimon.flink.log.LogStoreTableFactory; -import org.apache.paimon.operation.FileStoreCommit; -import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; -import org.apache.paimon.table.sink.BatchWriteBuilder; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.connector.sink.abilities.SupportsTruncate; @@ -30,8 +27,6 @@ import javax.annotation.Nullable; -import java.util.UUID; - /** Table sink to create sink. */ public class FlinkTableSink extends SupportsRowLevelOperationFlinkTableSink implements SupportsTruncate { @@ -46,9 +41,6 @@ public FlinkTableSink( @Override public void executeTruncation() { - FileStoreCommit commit = - ((FileStoreTable) table).store().newCommit(UUID.randomUUID().toString()); - long identifier = BatchWriteBuilder.COMMIT_IDENTIFIER; - commit.purgeTable(identifier); + table.newBatchWriteBuilder().newCommit().truncateTable(); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/SupportsRowLevelOperationFlinkTableSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/SupportsRowLevelOperationFlinkTableSink.java index c45fd168f946..0d2bd3962dee 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/SupportsRowLevelOperationFlinkTableSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/SupportsRowLevelOperationFlinkTableSink.java @@ -165,7 +165,7 @@ public Optional executeDeletion() { ((FileStoreTable) table).store().newCommit(UUID.randomUUID().toString()); long identifier = BatchWriteBuilder.COMMIT_IDENTIFIER; if (deletePredicate == null) { - commit.purgeTable(identifier); + commit.truncateTable(identifier); return Optional.empty(); } else if (deleteIsDropPartition()) { commit.dropPartitions(Collections.singletonList(deletePartitions()), identifier); diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/DeleteFromPaimonTableCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/DeleteFromPaimonTableCommand.scala index 3b8e801a2e3a..ff3aa253ca49 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/DeleteFromPaimonTableCommand.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/DeleteFromPaimonTableCommand.scala @@ -64,7 +64,7 @@ trait DeleteFromPaimonTableCommandBase extends PaimonLeafRunnableCommand with Pa if (forceDeleteByRows) { deleteRowsByCondition(sparkSession) } else if (deletePredicate.isEmpty) { - commit.purgeTable(BatchWriteBuilder.COMMIT_IDENTIFIER) + commit.truncateTable(BatchWriteBuilder.COMMIT_IDENTIFIER) } else { val visitor = new OnlyPartitionKeyEqualVisitor(table.partitionKeys) if (deletePredicate.get.visit(visitor)) { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonTruncateTableCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonTruncateTableCommand.scala index 9ca27b631487..e9125e3e649d 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonTruncateTableCommand.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonTruncateTableCommand.scala @@ -40,7 +40,7 @@ case class PaimonTruncateTableCommand(v2Table: SparkTable, partitionSpec: TableP val commit = table.store.newCommit(UUID.randomUUID.toString) if (partitionSpec.isEmpty) { - commit.purgeTable(BatchWriteBuilder.COMMIT_IDENTIFIER) + commit.truncateTable(BatchWriteBuilder.COMMIT_IDENTIFIER) } else { commit.dropPartitions( Collections.singletonList(partitionSpec.asJava), From 7175bd8e8ac97d0561fdb2cf92352febdc3fad75 Mon Sep 17 00:00:00 2001 From: wgcn <1026688210@qq.com> Date: Tue, 19 Mar 2024 14:29:03 +0800 Subject: [PATCH 25/79] [core] orc/parquet reader obtain the fileSize from metadata (#2918) --- .../paimon/format/FormatReaderContext.java | 54 +++++++++++++++++++ .../paimon/format/FormatReaderFactory.java | 7 +-- .../io/KeyValueDataFileRecordReader.java | 9 ++-- .../paimon/io/KeyValueFileReaderFactory.java | 10 ++-- .../paimon/io/RowDataFileRecordReader.java | 6 ++- .../operation/AppendOnlyFileStoreRead.java | 1 + .../paimon/io/KeyValueFileReadWriteTest.java | 27 ++++++++++ .../paimon/format/avro/AvroBulkFormat.java | 10 ++-- .../paimon/format/orc/OrcReaderFactory.java | 16 +++--- .../format/parquet/ParquetReaderFactory.java | 14 +++-- .../format/orc/OrcReaderFactoryTest.java | 9 +++- 11 files changed, 126 insertions(+), 37 deletions(-) create mode 100644 paimon-common/src/main/java/org/apache/paimon/format/FormatReaderContext.java diff --git a/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderContext.java b/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderContext.java new file mode 100644 index 000000000000..b1ad3fa47e1a --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderContext.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.format; + +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.reader.RecordReader; + +/** the context for creating RecordReader {@link RecordReader}. */ +public class FormatReaderContext { + private final FileIO fileIO; + private final Path file; + private final Integer poolSize; + private final Long fileSize; + + public FormatReaderContext(FileIO fileIO, Path file, Integer poolSize, Long fileSize) { + this.fileIO = fileIO; + this.file = file; + this.poolSize = poolSize; + this.fileSize = fileSize; + } + + public FileIO getFileIO() { + return fileIO; + } + + public Path getFile() { + return file; + } + + public Integer getPoolSize() { + return poolSize; + } + + public Long getFileSize() { + return fileSize; + } +} 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 b2b179159b8e..f524ff4a1465 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 @@ -29,8 +29,9 @@ /** A factory to create {@link RecordReader} for file. */ public interface FormatReaderFactory extends Serializable { - RecordReader createReader(FileIO fileIO, Path file) throws IOException; + default RecordReader createReader(FileIO fileIO, Path file) throws IOException { + return createReader(new FormatReaderContext(fileIO, file, null, null)); + } - RecordReader createReader(FileIO fileIO, Path file, int poolSize) - throws IOException; + RecordReader createReader(FormatReaderContext context) throws IOException; } diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java index fe38ae146463..4e7dfec9e55f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java @@ -26,6 +26,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.PartitionInfo; import org.apache.paimon.data.columnar.ColumnarRowIterator; +import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; @@ -58,13 +59,13 @@ public KeyValueDataFileRecordReader( @Nullable Integer poolSize, @Nullable int[] indexMapping, @Nullable CastFieldGetter[] castMapping, - @Nullable PartitionInfo partitionInfo) + @Nullable PartitionInfo partitionInfo, + long fileSize) throws IOException { FileUtils.checkExists(fileIO, path); this.reader = - poolSize == null - ? readerFactory.createReader(fileIO, path) - : readerFactory.createReader(fileIO, path, poolSize); + readerFactory.createReader( + new FormatReaderContext(fileIO, path, poolSize, fileSize)); this.serializer = new KeyValueSerializer(keyType, valueType); this.level = level; this.indexMapping = indexMapping; diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java index cc7534e9aed6..3123518c29bf 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java @@ -93,9 +93,9 @@ public RecordReader createRecordReader( long schemaId, String fileName, long fileSize, int level) throws IOException { if (fileSize >= asyncThreshold && fileName.endsWith("orc")) { return new AsyncRecordReader<>( - () -> createRecordReader(schemaId, fileName, level, false, 2)); + () -> createRecordReader(schemaId, fileName, level, false, 2, fileSize)); } - return createRecordReader(schemaId, fileName, level, true, null); + return createRecordReader(schemaId, fileName, level, true, null, fileSize); } private RecordReader createRecordReader( @@ -103,7 +103,8 @@ private RecordReader createRecordReader( String fileName, int level, boolean reuseFormat, - @Nullable Integer poolSize) + @Nullable Integer poolSize, + long fileSize) throws IOException { String formatIdentifier = DataFilePathFactory.formatIdentifier(fileName); @@ -131,7 +132,8 @@ private RecordReader createRecordReader( poolSize, bulkFormatMapping.getIndexMapping(), bulkFormatMapping.getCastMapping(), - PartitionUtils.create(bulkFormatMapping.getPartitionPair(), partition)); + PartitionUtils.create(bulkFormatMapping.getPartitionPair(), partition), + fileSize); Optional deletionVector = dvFactory.create(fileName); if (deletionVector.isPresent() && !deletionVector.get().isEmpty()) { recordReader = new ApplyDeletionVectorReader<>(recordReader, deletionVector.get()); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileRecordReader.java b/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileRecordReader.java index ee8f9c26f8b0..b461ebf0b6b9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileRecordReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileRecordReader.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.PartitionInfo; import org.apache.paimon.data.columnar.ColumnarRowIterator; +import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; @@ -46,12 +47,15 @@ public class RowDataFileRecordReader implements RecordReader { public RowDataFileRecordReader( FileIO fileIO, Path path, + long fileSize, FormatReaderFactory readerFactory, @Nullable int[] indexMapping, @Nullable CastFieldGetter[] castMapping, @Nullable PartitionInfo partitionInfo) throws IOException { - this.reader = FileUtils.createFormatReader(fileIO, readerFactory, path); + FileUtils.checkExists(fileIO, path); + FormatReaderContext context = new FormatReaderContext(fileIO, path, null, fileSize); + this.reader = readerFactory.createReader(context); this.indexMapping = indexMapping; this.partitionInfo = partitionInfo; this.castMapping = castMapping; diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java index 8363c297ad7b..b67edaaf3e6b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java @@ -177,6 +177,7 @@ public RecordReader createReader(DataSplit split) throws IOExceptio new RowDataFileRecordReader( fileIO, dataFilePathFactory.toPath(file.fileName()), + file.fileSize(), bulkFormatMapping.getReaderFactory(), bulkFormatMapping.getIndexMapping(), bulkFormatMapping.getCastMapping(), diff --git a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java index ca7f75d6eb57..552eac0a13f2 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java @@ -48,6 +48,8 @@ import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; import java.util.HashMap; @@ -385,4 +387,29 @@ private void checkRollingFiles( assertThat(meta.level()).isEqualTo(expected.level()); } } + + @ParameterizedTest + @ValueSource(strings = {"parquet", "orc", "avro"}) + public void testReaderUseFileSizeFromMetadata(String format) throws Exception { + DataFileTestDataGenerator.Data data = gen.next(); + KeyValueFileWriterFactory writerFactory = createWriterFactory(tempDir.toString(), format); + DataFileMetaSerializer serializer = new DataFileMetaSerializer(); + + RollingFileWriter writer = + writerFactory.createRollingMergeTreeFileWriter(0); + writer.write(CloseableIterator.fromList(data.content, kv -> {})); + writer.close(); + List actualMetas = writer.result(); + + KeyValueFileReaderFactory readerFactory = + createReaderFactory(tempDir.toString(), format, null, null); + assertData( + data, + actualMetas, + TestKeyValueGenerator.KEY_SERIALIZER, + TestKeyValueGenerator.DEFAULT_ROW_SERIALIZER, + serializer, + readerFactory, + kv -> kv); + } } 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 da60eedda113..717f998959a8 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 @@ -19,6 +19,7 @@ package org.apache.paimon.format.avro; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; @@ -49,14 +50,9 @@ public AvroBulkFormat(RowType projectedRowType) { } @Override - public RecordReader createReader(FileIO fileIO, Path file) throws IOException { - return new AvroReader(fileIO, file); - } - - @Override - public RecordReader createReader(FileIO fileIO, Path file, int poolSize) + public RecordReader createReader(FormatReaderContext formatReaderContext) throws IOException { - throw new UnsupportedOperationException(); + return new AvroReader(formatReaderContext.getFileIO(), formatReaderContext.getFile()); } private class AvroReader implements RecordReader { 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 696665777243..cdc46139f790 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 @@ -23,6 +23,7 @@ import org.apache.paimon.data.columnar.ColumnarRow; import org.apache.paimon.data.columnar.ColumnarRowIterator; import org.apache.paimon.data.columnar.VectorizedColumnBatch; +import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.format.fs.HadoopReadOnlyFileSystem; import org.apache.paimon.format.orc.filter.OrcFilters; @@ -88,14 +89,13 @@ public OrcReaderFactory( // ------------------------------------------------------------------------ @Override - public OrcVectorizedReader createReader(FileIO fileIO, Path file) throws IOException { - return createReader(fileIO, file, 1); - } - - @Override - public OrcVectorizedReader createReader(FileIO fileIO, Path file, int poolSize) - throws IOException { + public OrcVectorizedReader createReader(FormatReaderContext context) throws IOException { + int poolSize = context.getPoolSize() == null ? 1 : context.getPoolSize(); Pool poolOfBatches = createPoolOfBatches(poolSize); + + FileIO fileIO = context.getFileIO(); + Long fileSize = context.getFileSize(); + Path file = context.getFile(); RecordReader orcReader = createRecordReader( hadoopConfigWrapper.getHadoopConfig(), @@ -104,7 +104,7 @@ public OrcVectorizedReader createReader(FileIO fileIO, Path file, int poolSize) fileIO, file, 0, - fileIO.getFileSize(file)); + fileSize == null ? fileIO.getFileSize(file) : fileSize); return new OrcVectorizedReader(orcReader, poolOfBatches); } 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 2c2985d32ee1..29cf45a65260 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 @@ -24,6 +24,7 @@ import org.apache.paimon.data.columnar.ColumnarRowIterator; import org.apache.paimon.data.columnar.VectorizedColumnBatch; import org.apache.paimon.data.columnar.writable.WritableColumnVector; +import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.format.parquet.reader.ColumnReader; import org.apache.paimon.format.parquet.reader.ParquetDecimalVector; @@ -87,9 +88,12 @@ public ParquetReaderFactory(Options conf, RowType projectedType, int batchSize) } @Override - public ParquetReader createReader(FileIO fileIO, Path filePath) throws IOException { + public ParquetReader createReader(FormatReaderContext context) throws IOException { + Path filePath = context.getFile(); + FileIO fileIO = context.getFileIO(); + Long fileSize = context.getFileSize(); final long splitOffset = 0; - final long splitLength = fileIO.getFileSize(filePath); + final long splitLength = fileSize == null ? fileIO.getFileSize(filePath) : fileSize; ParquetReadOptions.Builder builder = ParquetReadOptions.builder().withRange(splitOffset, splitOffset + splitLength); @@ -108,12 +112,6 @@ public ParquetReader createReader(FileIO fileIO, Path filePath) throws IOExcepti return new ParquetReader(reader, requestedSchema, reader.getRecordCount(), poolOfBatches); } - @Override - public RecordReader createReader(FileIO fileIO, Path file, int poolSize) - throws IOException { - throw new UnsupportedOperationException(); - } - private void setReadOptions(ParquetReadOptions.Builder builder) { builder.useSignedStringMinMax( conf.getBoolean("parquet.strings.signed-min-max.enabled", false)); diff --git a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcReaderFactoryTest.java b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcReaderFactoryTest.java index a5160f5c7990..5a0f4925ddbc 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcReaderFactoryTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcReaderFactoryTest.java @@ -19,6 +19,7 @@ package org.apache.paimon.format.orc; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.format.orc.filter.OrcFilters; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; @@ -179,8 +180,10 @@ void testReadRowPositionWithRandomFilterAndPool() throws IOException { AtomicBoolean isFirst = new AtomicBoolean(true); + LocalFileIO localFileIO = new LocalFileIO(); try (RecordReader reader = - format.createReader(new LocalFileIO(), flatFile, randomPooSize)) { + format.createReader( + new FormatReaderContext(localFileIO, flatFile, randomPooSize, null))) { reader.forEachRemainingWithPosition( (rowPosition, row) -> { // check filter: _col0 > randomStart @@ -202,8 +205,10 @@ void testReadRowPositionWithTransformAndFilter() throws IOException { int randomPooSize = new Random().nextInt(3) + 1; OrcReaderFactory format = createFormat(FLAT_FILE_TYPE, new int[] {2, 0, 1}); + LocalFileIO localFileIO = new LocalFileIO(); try (RecordReader reader = - format.createReader(new LocalFileIO(), flatFile, randomPooSize)) { + format.createReader( + new FormatReaderContext(localFileIO, flatFile, randomPooSize, null))) { reader.transform(row -> row) .filter(row -> row.getInt(1) % 123 == 0) .forEachRemainingWithPosition( From 36ae4ae01f951c7698f7595504b8b17f12f35c94 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Tue, 19 Mar 2024 14:46:33 +0800 Subject: [PATCH 26/79] [core] Optimize write performance (#3039) --- .../benchmark/TableWriterBenchmark.java | 2 +- .../paimon/data/AbstractBinaryWriter.java | 6 - .../serializer/InternalRowSerializer.java | 28 +- .../serializer/InternalRowSerializerTest.java | 12 +- .../paimon/format/orc/writer/FieldWriter.java | 28 ++ .../format/orc/writer/FieldWriterFactory.java | 310 ++++++++++++++++++ .../format/orc/writer/RowDataVectorizer.java | 266 +-------------- 7 files changed, 373 insertions(+), 279 deletions(-) create mode 100644 paimon-format/src/main/java/org/apache/paimon/format/orc/writer/FieldWriter.java create mode 100644 paimon-format/src/main/java/org/apache/paimon/format/orc/writer/FieldWriterFactory.java diff --git a/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableWriterBenchmark.java b/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableWriterBenchmark.java index 04d1d7342751..9812afebba1c 100644 --- a/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableWriterBenchmark.java +++ b/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableWriterBenchmark.java @@ -100,7 +100,7 @@ public void testOrc() throws Exception { * Intel(R) Core(TM) i7-9750H CPU @ 2.60GHz * orc: Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative * --------------------------------------------------------------------------------- - * orc_write 36489 / 36697 82.2 12163.1 1.0X + * orc_write 31812 / 32223 94.3 10604.1 1.0X */ } diff --git a/paimon-common/src/main/java/org/apache/paimon/data/AbstractBinaryWriter.java b/paimon-common/src/main/java/org/apache/paimon/data/AbstractBinaryWriter.java index f8de8538ecba..b8bc3bd938ed 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/AbstractBinaryWriter.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/AbstractBinaryWriter.java @@ -177,12 +177,6 @@ public void writeTimestamp(int pos, Timestamp value, int precision) { } } - private void zeroBytes(int offset, int size) { - for (int i = offset; i < offset + size; i++) { - segment.put(i, (byte) 0); - } - } - protected void zeroOutPaddingBytes(int numBytes) { if ((numBytes & 0x07) > 0) { segment.putLong(cursor + ((numBytes >> 3) << 3), 0L); diff --git a/paimon-common/src/main/java/org/apache/paimon/data/serializer/InternalRowSerializer.java b/paimon-common/src/main/java/org/apache/paimon/data/serializer/InternalRowSerializer.java index ab2ff7c8b851..8b4810f57b71 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/serializer/InternalRowSerializer.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/serializer/InternalRowSerializer.java @@ -23,8 +23,10 @@ import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.BinaryRowWriter; import org.apache.paimon.data.BinaryWriter; +import org.apache.paimon.data.BinaryWriter.ValueSetter; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.InternalRow.FieldGetter; import org.apache.paimon.data.NestedRow; import org.apache.paimon.io.DataInputView; import org.apache.paimon.io.DataOutputView; @@ -33,16 +35,17 @@ import java.io.IOException; import java.util.Arrays; -import java.util.stream.IntStream; /** Serializer for {@link InternalRow}. */ public class InternalRowSerializer extends AbstractRowDataSerializer { + private static final long serialVersionUID = 1L; private final BinaryRowSerializer binarySerializer; private final DataType[] types; private final Serializer[] fieldSerializers; - private final InternalRow.FieldGetter[] fieldGetters; + private final FieldGetter[] fieldGetters; + private final ValueSetter[] valueSetters; private transient BinaryRow reuseRow; private transient BinaryRowWriter reuseWriter; @@ -65,10 +68,13 @@ public InternalRowSerializer(DataType[] types, Serializer[] fieldSerializers) this.types = types; this.fieldSerializers = fieldSerializers; this.binarySerializer = new BinaryRowSerializer(types.length); - this.fieldGetters = - IntStream.range(0, types.length) - .mapToObj(i -> InternalRow.createFieldGetter(types[i], i)) - .toArray(InternalRow.FieldGetter[]::new); + this.fieldGetters = new FieldGetter[types.length]; + this.valueSetters = new ValueSetter[types.length]; + for (int i = 0; i < types.length; i++) { + DataType type = types[i]; + fieldGetters[i] = InternalRow.createFieldGetter(type, i); + valueSetters[i] = BinaryWriter.createValueSetter(type); + } } @Override @@ -149,15 +155,11 @@ public BinaryRow toBinaryRow(InternalRow row) { reuseWriter.reset(); reuseWriter.writeRowKind(row.getRowKind()); for (int i = 0; i < types.length; i++) { - if (row.isNullAt(i)) { + Object field = fieldGetters[i].getFieldOrNull(row); + if (field == null) { reuseWriter.setNullAt(i); } else { - BinaryWriter.write( - reuseWriter, - i, - fieldGetters[i].getFieldOrNull(row), - types[i], - fieldSerializers[i]); + valueSetters[i].setValue(reuseWriter, i, field); } } reuseWriter.complete(); diff --git a/paimon-common/src/test/java/org/apache/paimon/data/serializer/InternalRowSerializerTest.java b/paimon-common/src/test/java/org/apache/paimon/data/serializer/InternalRowSerializerTest.java index 192c19514259..c6feb9d5812f 100644 --- a/paimon-common/src/test/java/org/apache/paimon/data/serializer/InternalRowSerializerTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/data/serializer/InternalRowSerializerTest.java @@ -50,11 +50,7 @@ abstract class InternalRowSerializerTest extends SerializerTestInstance { + + public static final FieldWriterFactory WRITER_FACTORY = new FieldWriterFactory(); + + private static final FieldWriter STRING_WRITER = + (rowId, column, getters, columnId) -> { + BytesColumnVector vector = (BytesColumnVector) column; + byte[] bytes = getters.getString(columnId).toBytes(); + vector.setVal(rowId, bytes, 0, bytes.length); + }; + + private static final FieldWriter BYTES_WRITER = + (rowId, column, getters, columnId) -> { + BytesColumnVector vector = (BytesColumnVector) column; + byte[] bytes = getters.getBinary(columnId); + vector.setVal(rowId, bytes, 0, bytes.length); + }; + + private static final FieldWriter BOOLEAN_WRITER = + (rowId, column, getters, columnId) -> + ((LongColumnVector) column).vector[rowId] = + getters.getBoolean(columnId) ? 1 : 0; + + private static final FieldWriter INT_WRITER = + (rowId, column, getters, columnId) -> + ((LongColumnVector) column).vector[rowId] = getters.getInt(columnId); + + private static final FieldWriter TINYINT_WRITER = + (rowId, column, getters, columnId) -> + ((LongColumnVector) column).vector[rowId] = getters.getByte(columnId); + + private static final FieldWriter SMALLINT_WRITER = + (rowId, column, getters, columnId) -> + ((LongColumnVector) column).vector[rowId] = getters.getShort(columnId); + + private static final FieldWriter BIGINT_WRITER = + (rowId, column, getters, columnId) -> + ((LongColumnVector) column).vector[rowId] = getters.getLong(columnId); + + private static final FieldWriter FLOAT_WRITER = + (rowId, column, getters, columnId) -> + ((DoubleColumnVector) column).vector[rowId] = getters.getFloat(columnId); + + private static final FieldWriter DOUBLE_WRITER = + (rowId, column, getters, columnId) -> + ((DoubleColumnVector) column).vector[rowId] = getters.getDouble(columnId); + + @Override + public FieldWriter visit(CharType charType) { + return STRING_WRITER; + } + + @Override + public FieldWriter visit(VarCharType varCharType) { + return STRING_WRITER; + } + + @Override + public FieldWriter visit(BooleanType booleanType) { + return BOOLEAN_WRITER; + } + + @Override + public FieldWriter visit(BinaryType binaryType) { + return BYTES_WRITER; + } + + @Override + public FieldWriter visit(VarBinaryType varBinaryType) { + return BYTES_WRITER; + } + + @Override + public FieldWriter visit(TinyIntType tinyIntType) { + return TINYINT_WRITER; + } + + @Override + public FieldWriter visit(SmallIntType smallIntType) { + return SMALLINT_WRITER; + } + + @Override + public FieldWriter visit(IntType intType) { + return INT_WRITER; + } + + @Override + public FieldWriter visit(BigIntType bigIntType) { + return BIGINT_WRITER; + } + + @Override + public FieldWriter visit(FloatType floatType) { + return FLOAT_WRITER; + } + + @Override + public FieldWriter visit(DoubleType doubleType) { + return DOUBLE_WRITER; + } + + @Override + public FieldWriter visit(DateType dateType) { + return INT_WRITER; + } + + @Override + public FieldWriter visit(TimeType timeType) { + return INT_WRITER; + } + + @Override + public FieldWriter visit(TimestampType timestampType) { + return (rowId, column, getters, columnId) -> { + Timestamp timestamp = + getters.getTimestamp(columnId, timestampType.getPrecision()).toSQLTimestamp(); + TimestampColumnVector vector = (TimestampColumnVector) column; + vector.set(rowId, timestamp); + }; + } + + @Override + public FieldWriter visit(LocalZonedTimestampType localZonedTimestampType) { + return (rowId, column, getters, columnId) -> { + Timestamp timestamp = + getters.getTimestamp(columnId, localZonedTimestampType.getPrecision()) + .toSQLTimestamp(); + TimestampColumnVector vector = (TimestampColumnVector) column; + vector.set(rowId, timestamp); + }; + } + + @Override + public FieldWriter visit(DecimalType decimalType) { + return (rowId, column, getters, columnId) -> { + DecimalColumnVector vector = (DecimalColumnVector) column; + Decimal decimal = + getters.getDecimal( + columnId, decimalType.getPrecision(), decimalType.getScale()); + HiveDecimal hiveDecimal = HiveDecimal.create(decimal.toBigDecimal()); + vector.set(rowId, hiveDecimal); + }; + } + + @Override + public FieldWriter visit(ArrayType arrayType) { + FieldWriter elementWriter = arrayType.getElementType().accept(this); + return (rowId, column, getters, columnId) -> { + ListColumnVector listColumnVector = (ListColumnVector) column; + InternalArray arrayData = getters.getArray(columnId); + listColumnVector.lengths[rowId] = arrayData.size(); + listColumnVector.offsets[rowId] = listColumnVector.childCount; + listColumnVector.childCount += listColumnVector.lengths[rowId]; + ensureSize( + listColumnVector.child, + listColumnVector.childCount, + listColumnVector.offsets[rowId] != 0); + + for (int i = 0; i < arrayData.size(); i++) { + ColumnVector fieldColumn = listColumnVector.child; + int fieldIndex = (int) listColumnVector.offsets[rowId] + i; + if (arrayData.isNullAt(i)) { + fieldColumn.noNulls = false; + fieldColumn.isNull[fieldIndex] = true; + } else { + elementWriter.write(fieldIndex, fieldColumn, arrayData, i); + } + } + }; + } + + @Override + public FieldWriter visit(MapType mapType) { + FieldWriter keyWriter = mapType.getKeyType().accept(this); + FieldWriter valueWriter = mapType.getValueType().accept(this); + return (rowId, column, getters, columnId) -> { + MapColumnVector mapColumnVector = (MapColumnVector) column; + InternalMap mapData = getters.getMap(columnId); + InternalArray keyArray = mapData.keyArray(); + InternalArray valueArray = mapData.valueArray(); + mapColumnVector.lengths[rowId] = mapData.size(); + mapColumnVector.offsets[rowId] = mapColumnVector.childCount; + mapColumnVector.childCount += mapColumnVector.lengths[rowId]; + ensureSize( + mapColumnVector.keys, + mapColumnVector.childCount, + mapColumnVector.offsets[rowId] != 0); + ensureSize( + mapColumnVector.values, + mapColumnVector.childCount, + mapColumnVector.offsets[rowId] != 0); + + for (int i = 0; i < keyArray.size(); i++) { + int fieldIndex = (int) mapColumnVector.offsets[rowId] + i; + + ColumnVector keyColumn = mapColumnVector.keys; + if (keyArray.isNullAt(i)) { + keyColumn.noNulls = false; + keyColumn.isNull[fieldIndex] = true; + } else { + keyWriter.write(fieldIndex, keyColumn, keyArray, i); + } + + ColumnVector valueColumn = mapColumnVector.values; + if (valueArray.isNullAt(i)) { + valueColumn.noNulls = false; + valueColumn.isNull[fieldIndex] = true; + } else { + valueWriter.write(fieldIndex, valueColumn, valueArray, i); + } + } + }; + } + + @Override + public FieldWriter visit(RowType rowType) { + List fieldWriters = + rowType.getFieldTypes().stream() + .map(t -> t.accept(this)) + .collect(Collectors.toList()); + return (rowId, column, getters, columnId) -> { + StructColumnVector structColumnVector = (StructColumnVector) column; + InternalRow structRow = getters.getRow(columnId, structColumnVector.fields.length); + for (int i = 0; i < structRow.getFieldCount(); i++) { + ColumnVector fieldColumn = structColumnVector.fields[i]; + if (structRow.isNullAt(i)) { + fieldColumn.noNulls = false; + fieldColumn.isNull[rowId] = true; + } else { + fieldWriters.get(i).write(rowId, fieldColumn, structRow, i); + } + } + }; + } + + @Override + public FieldWriter visit(MultisetType multisetType) { + throw new UnsupportedOperationException("Unsupported multisetType: " + multisetType); + } + + private static void ensureSize(ColumnVector cv, int size, boolean preserveData) { + int currentLength = cv.isNull.length; + if (currentLength < size) { + cv.ensureSize(Math.max(currentLength * 2, size), preserveData); + } + } +} diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/RowDataVectorizer.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/RowDataVectorizer.java index b05b5e208995..21443cdf9463 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/RowDataVectorizer.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/RowDataVectorizer.java @@ -18,274 +18,42 @@ package org.apache.paimon.format.orc.writer; -import org.apache.paimon.data.GenericRow; -import org.apache.paimon.data.InternalArray; -import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DecimalType; -import org.apache.paimon.types.LocalZonedTimestampType; -import org.apache.paimon.types.MapType; -import org.apache.paimon.types.RowType; -import org.apache.paimon.types.TimestampType; -import org.apache.hadoop.hive.common.type.HiveDecimal; -import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector; import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; -import java.sql.Timestamp; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.paimon.format.orc.writer.FieldWriterFactory.WRITER_FACTORY; /** A {@link Vectorizer} of {@link InternalRow} type element. */ public class RowDataVectorizer extends Vectorizer { - private final DataType[] fieldTypes; + private final List fieldWriters; public RowDataVectorizer(String schema, DataType[] fieldTypes) { super(schema); - this.fieldTypes = fieldTypes; + this.fieldWriters = + Arrays.stream(fieldTypes) + .map(t -> t.accept(WRITER_FACTORY)) + .collect(Collectors.toList()); } @Override public void vectorize(InternalRow row, VectorizedRowBatch batch) { int rowId = batch.size++; for (int i = 0; i < row.getFieldCount(); ++i) { - setColumn(rowId, batch.cols[i], fieldTypes[i], row, i); - } - } - - private static void setColumn( - int rowId, ColumnVector column, DataType type, InternalRow row, int columnId) { - if (row.isNullAt(columnId)) { - column.noNulls = false; - column.isNull[rowId] = true; - return; - } - - switch (type.getTypeRoot()) { - case CHAR: - case VARCHAR: - { - BytesColumnVector vector = (BytesColumnVector) column; - byte[] bytes = row.getString(columnId).toBytes(); - vector.setVal(rowId, bytes, 0, bytes.length); - break; - } - case BOOLEAN: - { - LongColumnVector vector = (LongColumnVector) column; - vector.vector[rowId] = row.getBoolean(columnId) ? 1 : 0; - break; - } - case BINARY: - case VARBINARY: - { - BytesColumnVector vector = (BytesColumnVector) column; - byte[] bytes = row.getBinary(columnId); - vector.setVal(rowId, bytes, 0, bytes.length); - break; - } - case DECIMAL: - { - DecimalType dt = (DecimalType) type; - DecimalColumnVector vector = (DecimalColumnVector) column; - vector.set( - rowId, - HiveDecimal.create( - row.getDecimal(columnId, dt.getPrecision(), dt.getScale()) - .toBigDecimal())); - break; - } - case TINYINT: - { - LongColumnVector vector = (LongColumnVector) column; - vector.vector[rowId] = row.getByte(columnId); - break; - } - case SMALLINT: - { - LongColumnVector vector = (LongColumnVector) column; - vector.vector[rowId] = row.getShort(columnId); - break; - } - case DATE: - case TIME_WITHOUT_TIME_ZONE: - case INTEGER: - { - LongColumnVector vector = (LongColumnVector) column; - vector.vector[rowId] = row.getInt(columnId); - break; - } - case BIGINT: - { - LongColumnVector vector = (LongColumnVector) column; - vector.vector[rowId] = row.getLong(columnId); - break; - } - case FLOAT: - { - DoubleColumnVector vector = (DoubleColumnVector) column; - vector.vector[rowId] = row.getFloat(columnId); - break; - } - case DOUBLE: - { - DoubleColumnVector vector = (DoubleColumnVector) column; - vector.vector[rowId] = row.getDouble(columnId); - break; - } - case TIMESTAMP_WITHOUT_TIME_ZONE: - { - TimestampType tt = (TimestampType) type; - Timestamp timestamp = - row.getTimestamp(columnId, tt.getPrecision()).toSQLTimestamp(); - TimestampColumnVector vector = (TimestampColumnVector) column; - vector.set(rowId, timestamp); - break; - } - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - { - LocalZonedTimestampType lt = (LocalZonedTimestampType) type; - Timestamp timestamp = - row.getTimestamp(columnId, lt.getPrecision()).toSQLTimestamp(); - TimestampColumnVector vector = (TimestampColumnVector) column; - vector.set(rowId, timestamp); - break; - } - case ARRAY: - { - ListColumnVector listColumnVector = (ListColumnVector) column; - setColumn(rowId, listColumnVector, type, row, columnId); - break; - } - case MAP: - { - MapColumnVector mapColumnVector = (MapColumnVector) column; - setColumn(rowId, mapColumnVector, type, row, columnId); - break; - } - case ROW: - { - StructColumnVector structColumnVector = (StructColumnVector) column; - setColumn(rowId, structColumnVector, type, row, columnId); - break; - } - default: - throw new UnsupportedOperationException("Unsupported type: " + type); - } - } - - private static void setColumn( - int rowId, - ListColumnVector listColumnVector, - DataType type, - InternalRow row, - int columnId) { - InternalArray arrayData = row.getArray(columnId); - ArrayType arrayType = (ArrayType) type; - listColumnVector.lengths[rowId] = arrayData.size(); - listColumnVector.offsets[rowId] = listColumnVector.childCount; - listColumnVector.childCount += listColumnVector.lengths[rowId]; - ensureSize( - listColumnVector.child, - listColumnVector.childCount, - listColumnVector.offsets[rowId] != 0); - - InternalRow convertedRowData = convert(arrayData, arrayType.getElementType()); - for (int i = 0; i < arrayData.size(); i++) { - setColumn( - (int) listColumnVector.offsets[rowId] + i, - listColumnVector.child, - arrayType.getElementType(), - convertedRowData, - i); - } - } - - private static void setColumn( - int rowId, - MapColumnVector mapColumnVector, - DataType type, - InternalRow row, - int columnId) { - InternalMap mapData = row.getMap(columnId); - MapType mapType = (MapType) type; - InternalArray keyArray = mapData.keyArray(); - InternalArray valueArray = mapData.valueArray(); - mapColumnVector.lengths[rowId] = mapData.size(); - mapColumnVector.offsets[rowId] = mapColumnVector.childCount; - mapColumnVector.childCount += mapColumnVector.lengths[rowId]; - ensureSize( - mapColumnVector.keys, - mapColumnVector.childCount, - mapColumnVector.offsets[rowId] != 0); - ensureSize( - mapColumnVector.values, - mapColumnVector.childCount, - mapColumnVector.offsets[rowId] != 0); - - InternalRow convertedKeyRowData = convert(keyArray, mapType.getKeyType()); - InternalRow convertedValueRowData = convert(valueArray, mapType.getValueType()); - for (int i = 0; i < keyArray.size(); i++) { - setColumn( - (int) mapColumnVector.offsets[rowId] + i, - mapColumnVector.keys, - mapType.getKeyType(), - convertedKeyRowData, - i); - setColumn( - (int) mapColumnVector.offsets[rowId] + i, - mapColumnVector.values, - mapType.getValueType(), - convertedValueRowData, - i); - } - } - - private static void setColumn( - int rowId, - StructColumnVector structColumnVector, - DataType type, - InternalRow row, - int columnId) { - InternalRow structRow = row.getRow(columnId, structColumnVector.fields.length); - RowType rowType = (RowType) type; - for (int i = 0; i < structRow.getFieldCount(); i++) { - ColumnVector cv = structColumnVector.fields[i]; - setColumn(rowId, cv, rowType.getTypeAt(i), structRow, i); - } - } - - private static void ensureSize(ColumnVector cv, int size, boolean preserveData) { - int currentLength = cv.isNull.length; - if (currentLength < size) { - cv.ensureSize(Math.max(currentLength * 2, size), preserveData); - } - } - - /** - * Converting ArrayData to RowData for calling {@link RowDataVectorizer#setColumn(int, - * ColumnVector, DataType, InternalRow, int)} recursively with array. - * - * @param arrayData input ArrayData. - * @param arrayFieldType DataType of input ArrayData. - * @return RowData. - */ - private static InternalRow convert(InternalArray arrayData, DataType arrayFieldType) { - GenericRow rowData = new GenericRow(arrayData.size()); - InternalArray.ElementGetter elementGetter = - InternalArray.createElementGetter(arrayFieldType); - for (int i = 0; i < arrayData.size(); i++) { - rowData.setField(i, elementGetter.getElementOrNull(arrayData, i)); + ColumnVector fieldColumn = batch.cols[i]; + if (row.isNullAt(i)) { + fieldColumn.noNulls = false; + fieldColumn.isNull[rowId] = true; + } else { + fieldWriters.get(i).write(rowId, fieldColumn, row, i); + } } - return rowData; } } From cb8cabc315e8bfc34ae0862c59f3930fd934b2a2 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Tue, 19 Mar 2024 16:44:20 +0800 Subject: [PATCH 27/79] [core] Reduce getFileSize for avro reader (#3040) --- .../paimon/format/FormatReaderContext.java | 22 +++++------ .../paimon/format/FormatReaderFactory.java | 14 +++++-- .../paimon/format/OrcFormatReaderContext.java | 38 +++++++++++++++++++ .../paimon/format/FormatReadWriteTest.java | 8 +++- .../io/KeyValueDataFileRecordReader.java | 16 ++------ .../paimon/io/KeyValueFileReaderFactory.java | 16 +++++--- .../paimon/io/RowDataFileRecordReader.java | 2 +- .../org/apache/paimon/manifest/FileEntry.java | 2 +- .../paimon/manifest/ManifestFileMeta.java | 2 +- .../apache/paimon/manifest/ManifestList.java | 3 -- .../operation/AbstractFileStoreScan.java | 2 + .../paimon/operation/SnapshotDeletion.java | 2 +- .../org/apache/paimon/utils/FileUtils.java | 25 +++++------- .../org/apache/paimon/utils/ObjectsCache.java | 20 ++++++---- .../org/apache/paimon/utils/ObjectsFile.java | 36 +++++++++++++----- .../org/apache/paimon/FileFormatTest.java | 8 +++- .../paimon/manifest/ManifestFileMetaTest.java | 2 + .../manifest/ManifestFileMetaTestBase.java | 24 ++++++------ .../paimon/manifest/ManifestFileTest.java | 2 +- .../apache/paimon/stats/StatsTableTest.java | 3 +- .../paimon/stats/TestTableStatsExtractor.java | 19 ++++++++-- .../apache/paimon/utils/ObjectsCacheTest.java | 26 +++++++++---- .../paimon/format/avro/AvroBulkFormat.java | 9 ++--- .../paimon/format/orc/OrcReaderFactory.java | 20 +++++----- .../format/parquet/ParquetReaderFactory.java | 17 +++------ .../paimon/format/BulkFileFormatTest.java | 8 +++- .../format/orc/OrcReaderFactoryTest.java | 27 ++++++++++--- .../format/parquet/ParquetReadWriteTest.java | 27 +++++++++++-- 28 files changed, 261 insertions(+), 139 deletions(-) create mode 100644 paimon-common/src/main/java/org/apache/paimon/format/OrcFormatReaderContext.java diff --git a/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderContext.java b/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderContext.java index b1ad3fa47e1a..92a569e031de 100644 --- a/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderContext.java +++ b/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderContext.java @@ -23,32 +23,30 @@ import org.apache.paimon.reader.RecordReader; /** the context for creating RecordReader {@link RecordReader}. */ -public class FormatReaderContext { +public class FormatReaderContext implements FormatReaderFactory.Context { + private final FileIO fileIO; private final Path file; - private final Integer poolSize; - private final Long fileSize; + private final long fileSize; - public FormatReaderContext(FileIO fileIO, Path file, Integer poolSize, Long fileSize) { + public FormatReaderContext(FileIO fileIO, Path file, long fileSize) { this.fileIO = fileIO; this.file = file; - this.poolSize = poolSize; this.fileSize = fileSize; } - public FileIO getFileIO() { + @Override + public FileIO fileIO() { return fileIO; } - public Path getFile() { + @Override + public Path filePath() { return file; } - public Integer getPoolSize() { - return poolSize; - } - - public Long getFileSize() { + @Override + public long fileSize() { return fileSize; } } 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 f524ff4a1465..ce92bb751252 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 @@ -29,9 +29,15 @@ /** A factory to create {@link RecordReader} for file. */ public interface FormatReaderFactory extends Serializable { - default RecordReader createReader(FileIO fileIO, Path file) throws IOException { - return createReader(new FormatReaderContext(fileIO, file, null, null)); - } + RecordReader createReader(Context context) throws IOException; + + /** Context for creating reader. */ + interface Context { + + FileIO fileIO(); - RecordReader createReader(FormatReaderContext context) throws IOException; + Path filePath(); + + long fileSize(); + } } diff --git a/paimon-common/src/main/java/org/apache/paimon/format/OrcFormatReaderContext.java b/paimon-common/src/main/java/org/apache/paimon/format/OrcFormatReaderContext.java new file mode 100644 index 000000000000..8b761867fa8e --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/format/OrcFormatReaderContext.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.format; + +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.reader.RecordReader; + +/** The context for creating orc {@link RecordReader}. */ +public class OrcFormatReaderContext extends FormatReaderContext { + + private final int poolSize; + + public OrcFormatReaderContext(FileIO fileIO, Path filePath, long fileSize, int poolSize) { + super(fileIO, filePath, fileSize); + this.poolSize = poolSize; + } + + public int poolSize() { + return poolSize; + } +} diff --git a/paimon-common/src/test/java/org/apache/paimon/format/FormatReadWriteTest.java b/paimon-common/src/test/java/org/apache/paimon/format/FormatReadWriteTest.java index 400ef1109674..556f2f603c47 100644 --- a/paimon-common/src/test/java/org/apache/paimon/format/FormatReadWriteTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/format/FormatReadWriteTest.java @@ -96,7 +96,9 @@ public void testSimpleTypes() throws IOException { out.close(); RecordReader reader = - format.createReaderFactory(rowType).createReader(fileIO, file); + format.createReaderFactory(rowType) + .createReader( + new FormatReaderContext(fileIO, file, fileIO.getFileSize(file))); List result = new ArrayList<>(); reader.forEachRemaining(row -> result.add(serializer.copy(row))); @@ -123,7 +125,9 @@ public void testFullTypes() throws IOException { out.close(); RecordReader reader = - format.createReaderFactory(rowType).createReader(fileIO, file); + format.createReaderFactory(rowType) + .createReader( + new FormatReaderContext(fileIO, file, fileIO.getFileSize(file))); List result = new ArrayList<>(); reader.forEachRemaining(result::add); assertThat(result.size()).isEqualTo(1); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java index 4e7dfec9e55f..92be0ff684a5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java @@ -26,10 +26,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.PartitionInfo; import org.apache.paimon.data.columnar.ColumnarRowIterator; -import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.format.FormatReaderFactory; -import org.apache.paimon.fs.FileIO; -import org.apache.paimon.fs.Path; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.FileUtils; @@ -50,22 +47,17 @@ public class KeyValueDataFileRecordReader implements RecordReader { @Nullable private final CastFieldGetter[] castMapping; public KeyValueDataFileRecordReader( - FileIO fileIO, FormatReaderFactory readerFactory, - Path path, + FormatReaderFactory.Context context, RowType keyType, RowType valueType, int level, - @Nullable Integer poolSize, @Nullable int[] indexMapping, @Nullable CastFieldGetter[] castMapping, - @Nullable PartitionInfo partitionInfo, - long fileSize) + @Nullable PartitionInfo partitionInfo) throws IOException { - FileUtils.checkExists(fileIO, path); - this.reader = - readerFactory.createReader( - new FormatReaderContext(fileIO, path, poolSize, fileSize)); + FileUtils.checkExists(context.fileIO(), context.filePath()); + this.reader = readerFactory.createReader(context); this.serializer = new KeyValueSerializer(keyType, valueType); this.level = level; this.indexMapping = indexMapping; diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java index 3123518c29bf..184857b45691 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java @@ -25,7 +25,10 @@ import org.apache.paimon.deletionvectors.DeletionVector; import org.apache.paimon.format.FileFormatDiscover; import org.apache.paimon.format.FormatKey; +import org.apache.paimon.format.FormatReaderContext; +import org.apache.paimon.format.OrcFormatReaderContext; import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; import org.apache.paimon.partition.PartitionUtils; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.reader.RecordReader; @@ -103,7 +106,7 @@ private RecordReader createRecordReader( String fileName, int level, boolean reuseFormat, - @Nullable Integer poolSize, + @Nullable Integer orcPoolSize, long fileSize) throws IOException { String formatIdentifier = DataFilePathFactory.formatIdentifier(fileName); @@ -121,19 +124,20 @@ private RecordReader createRecordReader( new FormatKey(schemaId, formatIdentifier), key -> formatSupplier.get()) : formatSupplier.get(); + Path filePath = pathFactory.toPath(fileName); RecordReader recordReader = new KeyValueDataFileRecordReader( - fileIO, bulkFormatMapping.getReaderFactory(), - pathFactory.toPath(fileName), + orcPoolSize == null + ? new FormatReaderContext(fileIO, filePath, fileSize) + : new OrcFormatReaderContext( + fileIO, filePath, fileSize, orcPoolSize), keyType, valueType, level, - poolSize, bulkFormatMapping.getIndexMapping(), bulkFormatMapping.getCastMapping(), - PartitionUtils.create(bulkFormatMapping.getPartitionPair(), partition), - fileSize); + PartitionUtils.create(bulkFormatMapping.getPartitionPair(), partition)); Optional deletionVector = dvFactory.create(fileName); if (deletionVector.isPresent() && !deletionVector.get().isEmpty()) { recordReader = new ApplyDeletionVectorReader<>(recordReader, deletionVector.get()); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileRecordReader.java b/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileRecordReader.java index b461ebf0b6b9..ed891a32b2e1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileRecordReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileRecordReader.java @@ -54,7 +54,7 @@ public RowDataFileRecordReader( @Nullable PartitionInfo partitionInfo) throws IOException { FileUtils.checkExists(fileIO, path); - FormatReaderContext context = new FormatReaderContext(fileIO, path, null, fileSize); + FormatReaderContext context = new FormatReaderContext(fileIO, path, fileSize); this.reader = readerFactory.createReader(context); this.indexMapping = indexMapping; this.partitionInfo = partitionInfo; diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java b/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java index 46f36be7f1e0..e0a6d25b71a3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java @@ -162,7 +162,7 @@ static List>> readManifestEntries( for (ManifestFileMeta file : manifestFiles) { Future> future = CompletableFuture.supplyAsync( - () -> manifestFile.read(file.fileName()), + () -> manifestFile.read(file.fileName(), file.fileSize()), FileUtils.COMMON_IO_FORK_JOIN_POOL); result.add( () -> { diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFileMeta.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFileMeta.java index c0bcdd061a91..105e150a94b3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFileMeta.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFileMeta.java @@ -321,7 +321,7 @@ public static Optional> tryFullCompaction( for (; j < base.size(); j++) { ManifestFileMeta file = base.get(j); boolean contains = false; - for (ManifestEntry entry : manifestFile.read(file.fileName)) { + for (ManifestEntry entry : manifestFile.read(file.fileName, file.fileSize)) { checkArgument(entry.kind() == FileKind.ADD); if (deleteEntries.contains(entry.identifier())) { contains = true; diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestList.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestList.java index fc2986c9c0b7..84781cdea0ea 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestList.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestList.java @@ -39,8 +39,6 @@ */ public class ManifestList extends ObjectsFile { - private final FormatWriterFactory writerFactory; - private ManifestList( FileIO fileIO, ManifestFileMetaSerializer serializer, @@ -49,7 +47,6 @@ private ManifestList( PathFactory pathFactory, @Nullable SegmentsCache cache) { super(fileIO, serializer, readerFactory, writerFactory, pathFactory, cache); - this.writerFactory = writerFactory; } /** diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java index 52983f4b6e2b..94df882ff3f9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java @@ -458,6 +458,7 @@ private List readManifestFileMeta(ManifestFileMeta manifest) { .create() .read( manifest.fileName(), + manifest.fileSize(), ManifestEntry.createCacheRowFilter(manifestCacheFilter, numOfBuckets), ManifestEntry.createEntryRowFilter( partitionFilter, bucketFilter, numOfBuckets)); @@ -469,6 +470,7 @@ private List readSimpleEntries(ManifestFileMeta manifest) { .createSimpleFileEntryReader() .read( manifest.fileName(), + manifest.fileSize(), // use filter for ManifestEntry // currently, projection is not pushed down to file format // see SimpleFileEntrySerializer diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/SnapshotDeletion.java b/paimon-core/src/main/java/org/apache/paimon/operation/SnapshotDeletion.java index 23224f1c1378..f3c353aadd40 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/SnapshotDeletion.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/SnapshotDeletion.java @@ -69,7 +69,7 @@ public void cleanUnusedDataFiles(Snapshot snapshot, Predicate ski // try read manifests List manifestFileNames = readManifestFileNames(tryReadManifestList(snapshot.deltaManifestList())); - List manifestEntries = new ArrayList<>(); + List manifestEntries; // data file path -> (original manifest entry, extra file paths) Map>> dataFileToDelete = new HashMap<>(); for (String manifest : manifestFileNames) { diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/FileUtils.java b/paimon-core/src/main/java/org/apache/paimon/utils/FileUtils.java index 69cd004202c5..f1278ab717c8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/FileUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/FileUtils.java @@ -19,17 +19,18 @@ package org.apache.paimon.utils; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; import org.apache.paimon.reader.RecordReader; +import javax.annotation.Nullable; + import java.io.FileNotFoundException; import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; -import java.util.List; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.ForkJoinWorkerThread; import java.util.stream.Stream; @@ -70,18 +71,6 @@ public static synchronized ForkJoinPool getScanIoForkJoinPool(int parallelism) { return scanIoForkJoinPool; } - public static List readListFromFile( - FileIO fileIO, - Path path, - ObjectSerializer serializer, - FormatReaderFactory readerFactory) - throws IOException { - List result = new ArrayList<>(); - createFormatReader(fileIO, readerFactory, path) - .forEachRemaining(row -> result.add(serializer.fromRow(row))); - return result; - } - /** * List versioned files for the directory. * @@ -143,8 +132,12 @@ public static void checkExists(FileIO fileIO, Path file) throws IOException { } public static RecordReader createFormatReader( - FileIO fileIO, FormatReaderFactory format, Path file) throws IOException { + FileIO fileIO, FormatReaderFactory format, Path file, @Nullable Long fileSize) + throws IOException { checkExists(fileIO, file); - return format.createReader(fileIO, file); + if (fileSize == null) { + fileSize = fileIO.getFileSize(file); + } + return format.createReader(new FormatReaderContext(fileIO, file, fileSize)); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java b/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java index cfbe094577fa..40482c2f5569 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java @@ -27,11 +27,13 @@ import org.apache.paimon.memory.MemorySegment; import org.apache.paimon.memory.MemorySegmentSource; +import javax.annotation.Nullable; + import java.io.EOFException; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.function.Function; +import java.util.function.BiFunction; /** Cache records to {@link SegmentsCache} by compacted serializer. */ public class ObjectsCache { @@ -39,21 +41,25 @@ public class ObjectsCache { private final SegmentsCache cache; private final ObjectSerializer serializer; private final InternalRowSerializer rowSerializer; - private final Function> reader; + private final BiFunction> reader; public ObjectsCache( SegmentsCache cache, ObjectSerializer serializer, - Function> reader) { + BiFunction> reader) { this.cache = cache; this.serializer = serializer; this.rowSerializer = new InternalRowSerializer(serializer.fieldTypes()); this.reader = reader; } - public List read(K key, Filter loadFilter, Filter readFilter) + public List read( + K key, + @Nullable Long fileSize, + Filter loadFilter, + Filter readFilter) throws IOException { - Segments segments = cache.getSegments(key, k -> readSegments(k, loadFilter)); + Segments segments = cache.getSegments(key, k -> readSegments(k, fileSize, loadFilter)); List entries = new ArrayList<>(); RandomAccessInputView view = new RandomAccessInputView( @@ -71,8 +77,8 @@ public List read(K key, Filter loadFilter, Filter r } } - private Segments readSegments(K key, Filter loadFilter) { - try (CloseableIterator iterator = reader.apply(key)) { + private Segments readSegments(K key, @Nullable Long fileSize, Filter loadFilter) { + try (CloseableIterator iterator = reader.apply(key, fileSize)) { ArrayList segments = new ArrayList<>(); MemorySegmentSource segmentSource = () -> MemorySegment.allocateHeapMemory(cache.pageSize()); 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 61a465e4b6a2..474b757b6f5d 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 @@ -75,11 +75,20 @@ public long fileSize(String fileName) { } public List read(String fileName) { - return read(fileName, Filter.alwaysTrue(), Filter.alwaysTrue()); + return read(fileName, null); + } + + public List read(String fileName, @Nullable Long fileSize) { + return read(fileName, fileSize, Filter.alwaysTrue(), Filter.alwaysTrue()); } public List readWithIOException(String fileName) throws IOException { - return readWithIOException(fileName, Filter.alwaysTrue(), Filter.alwaysTrue()); + return readWithIOException(fileName, null); + } + + public List readWithIOException(String fileName, @Nullable Long fileSize) + throws IOException { + return readWithIOException(fileName, fileSize, Filter.alwaysTrue(), Filter.alwaysTrue()); } public boolean exists(String fileName) { @@ -91,23 +100,29 @@ public boolean exists(String fileName) { } public List read( - String fileName, Filter loadFilter, Filter readFilter) { + String fileName, + @Nullable Long fileSize, + Filter loadFilter, + Filter readFilter) { try { - return readWithIOException(fileName, loadFilter, readFilter); + return readWithIOException(fileName, fileSize, loadFilter, readFilter); } catch (IOException e) { throw new RuntimeException("Failed to read manifest list " + fileName, e); } } - public List readWithIOException( - String fileName, Filter loadFilter, Filter readFilter) + private List readWithIOException( + String fileName, + @Nullable Long fileSize, + Filter loadFilter, + Filter readFilter) throws IOException { if (cache != null) { - return cache.read(fileName, loadFilter, readFilter); + return cache.read(fileName, fileSize, loadFilter, readFilter); } RecordReader reader = - createFormatReader(fileIO, readerFactory, pathFactory.toPath(fileName)); + createFormatReader(fileIO, readerFactory, pathFactory.toPath(fileName), fileSize); if (readFilter != Filter.ALWAYS_TRUE) { reader = reader.filter(readFilter); } @@ -143,9 +158,10 @@ public String writeWithoutRolling(Iterator records) { } } - private CloseableIterator createIterator(String fileName) { + private CloseableIterator createIterator( + String fileName, @Nullable Long fileSize) { try { - return createFormatReader(fileIO, readerFactory, pathFactory.toPath(fileName)) + return createFormatReader(fileIO, readerFactory, pathFactory.toPath(fileName), fileSize) .toCloseableIterator(); } catch (IOException e) { throw new RuntimeException(e); diff --git a/paimon-core/src/test/java/org/apache/paimon/FileFormatTest.java b/paimon-core/src/test/java/org/apache/paimon/FileFormatTest.java index 74503ad47bb6..fc097bee3885 100644 --- a/paimon-core/src/test/java/org/apache/paimon/FileFormatTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/FileFormatTest.java @@ -22,6 +22,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.format.FileFormat; import org.apache.paimon.format.FileFormatDiscover; +import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.format.FormatWriter; import org.apache.paimon.format.FormatWriterFactory; import org.apache.paimon.format.orc.OrcFileFormat; @@ -71,7 +72,12 @@ public void testWriteRead(@TempDir java.nio.file.Path tempDir) throws IOExceptio // read RecordReader reader = - avro.createReaderFactory(rowType).createReader(LocalFileIO.create(), path); + avro.createReaderFactory(rowType) + .createReader( + new FormatReaderContext( + LocalFileIO.create(), + path, + LocalFileIO.create().getFileSize(path))); List result = new ArrayList<>(); reader.forEachRemaining( rowData -> result.add(GenericRow.of(rowData.getInt(0), rowData.getInt(1)))); diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTest.java index c08543784951..9e44493142ce 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTest.java @@ -27,6 +27,7 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -60,6 +61,7 @@ public void beforeEach() { manifestFile = createManifestFile(tempDir.toString()); } + @Disabled // TODO wrong test to rely on self-defined file size @ParameterizedTest @ValueSource(ints = {2, 3, 4}) public void testMergeWithoutFullCompaction(int numLastBits) { diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java index 1a36346c1147..e066eeaf9ace 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java @@ -87,14 +87,7 @@ protected ManifestEntry makeEntry(boolean isAdd, String fileName, Integer partit } protected ManifestFileMeta makeManifest(ManifestEntry... entries) { - ManifestFileMeta writtenMeta = getManifestFile().write(Arrays.asList(entries)).get(0); - return new ManifestFileMeta( - writtenMeta.fileName(), - entries.length * 100, // for testing purpose - writtenMeta.numAddedFiles(), - writtenMeta.numDeletedFiles(), - writtenMeta.partitionStats(), - 0); + return getManifestFile().write(Arrays.asList(entries)).get(0); } abstract ManifestFile getManifestFile(); @@ -105,7 +98,7 @@ protected void assertEquivalentEntries( List input, List merged) { List inputEntry = input.stream() - .flatMap(f -> getManifestFile().read(f.fileName()).stream()) + .flatMap(f -> getManifestFile().read(f.fileName(), f.fileSize()).stream()) .collect(Collectors.toList()); List entryBeforeMerge = FileEntry.mergeEntries(inputEntry).stream() @@ -115,7 +108,9 @@ protected void assertEquivalentEntries( List entryAfterMerge = new ArrayList<>(); for (ManifestFileMeta manifestFileMeta : merged) { - List entries = getManifestFile().read(manifestFileMeta.fileName()); + List entries = + getManifestFile() + .read(manifestFileMeta.fileName(), manifestFileMeta.fileSize()); for (ManifestEntry entry : entries) { entryAfterMerge.add(entry.kind() + "-" + entry.file().fileName()); } @@ -146,7 +141,10 @@ protected void containSameEntryFile( List mergedMainfest, List expecteded) { List actual = mergedMainfest.stream() - .flatMap(file -> getManifestFile().read(file.fileName()).stream()) + .flatMap( + file -> + getManifestFile().read(file.fileName(), file.fileSize()) + .stream()) .map(f -> f.kind() + "-" + f.file().fileName()) .collect(Collectors.toList()); assertThat(actual).hasSameElementsAs(expecteded); @@ -160,8 +158,8 @@ protected void assertSameContent( assertThat(actual.partitionStats()).isEqualTo(expected.partitionStats()); // check content - assertThat(manifestFile.read(actual.fileName())) - .isEqualTo(manifestFile.read(expected.fileName())); + assertThat(manifestFile.read(actual.fileName(), actual.fileSize())) + .isEqualTo(manifestFile.read(expected.fileName(), expected.fileSize())); } protected List createBaseManifestFileMetas(boolean hasPartition) { diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java index daa3b71a1f2a..d013a7ff0460 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java @@ -63,7 +63,7 @@ public void testWriteAndReadManifestFile() { checkRollingFiles(meta, actualMetas, manifestFile.suggestedFileSize()); List actualEntries = actualMetas.stream() - .flatMap(m -> manifestFile.read(m.fileName()).stream()) + .flatMap(m -> manifestFile.read(m.fileName(), m.fileSize()).stream()) .collect(Collectors.toList()); assertThat(actualEntries).isEqualTo(entries); } diff --git a/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java b/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java index b51537972841..f4bdca414864 100644 --- a/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java @@ -80,7 +80,8 @@ public void testPartitionStats() throws Exception { // should not have record stats because of NONE mode ManifestFile manifestFile = store.manifestFileFactory().create(); - DataFileMeta file = manifestFile.read(manifest.fileName()).get(0).file(); + DataFileMeta file = + manifestFile.read(manifest.fileName(), manifest.fileSize()).get(0).file(); BinaryTableStats recordStats = file.valueStats(); assertThat(recordStats.minValues().isNullAt(0)).isTrue(); assertThat(recordStats.minValues().isNullAt(1)).isTrue(); diff --git a/paimon-core/src/test/java/org/apache/paimon/stats/TestTableStatsExtractor.java b/paimon-core/src/test/java/org/apache/paimon/stats/TestTableStatsExtractor.java index 7c957265324d..f8b18f79a2bc 100644 --- a/paimon-core/src/test/java/org/apache/paimon/stats/TestTableStatsExtractor.java +++ b/paimon-core/src/test/java/org/apache/paimon/stats/TestTableStatsExtractor.java @@ -28,14 +28,16 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.statistics.FieldStatsCollector; import org.apache.paimon.types.RowType; -import org.apache.paimon.utils.FileUtils; import org.apache.paimon.utils.ObjectSerializer; import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.Preconditions; import java.io.IOException; +import java.util.ArrayList; import java.util.List; +import static org.apache.paimon.utils.FileUtils.createFormatReader; + /** * {@link TableStatsExtractor} for test. It reads all records from the file and use {@link * TableStatsCollector} to collect the stats. @@ -66,8 +68,7 @@ public Pair extractWithFileInfo(FileIO fileIO, Path path throws IOException { IdentityObjectSerializer serializer = new IdentityObjectSerializer(rowType); FormatReaderFactory readerFactory = format.createReaderFactory(rowType); - List records = - FileUtils.readListFromFile(fileIO, path, serializer, readerFactory); + List records = readListFromFile(fileIO, path, serializer, readerFactory); TableStatsCollector statsCollector = new TableStatsCollector(rowType, stats); for (InternalRow record : records) { @@ -76,6 +77,18 @@ public Pair extractWithFileInfo(FileIO fileIO, Path path return Pair.of(statsCollector.extract(), new FileInfo(records.size())); } + private static List readListFromFile( + FileIO fileIO, + Path path, + ObjectSerializer serializer, + FormatReaderFactory readerFactory) + throws IOException { + List result = new ArrayList<>(); + createFormatReader(fileIO, readerFactory, path, null) + .forEachRemaining(row -> result.add(serializer.fromRow(row))); + return result; + } + private static class IdentityObjectSerializer extends ObjectSerializer { public IdentityObjectSerializer(RowType rowType) { diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/ObjectsCacheTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/ObjectsCacheTest.java index a8d67272ad99..13271bd324a1 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/ObjectsCacheTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/ObjectsCacheTest.java @@ -46,7 +46,7 @@ public void test() throws IOException { new ObjectsCache<>( new SegmentsCache<>(1024, MemorySize.ofKibiBytes(5)), new StringSerializer(), - k -> + (k, size) -> CloseableIterator.adapterForIterator( map.get(k).stream() .map(BinaryString::fromString) @@ -56,36 +56,48 @@ public void test() throws IOException { // test empty map.put("k1", Collections.emptyList()); - List values = cache.read("k1", Filter.alwaysTrue(), Filter.alwaysTrue()); + List values = cache.read("k1", null, Filter.alwaysTrue(), Filter.alwaysTrue()); assertThat(values).isEmpty(); // test values List expect = Arrays.asList("v1", "v2", "v3"); map.put("k2", expect); - values = cache.read("k2", Filter.alwaysTrue(), Filter.alwaysTrue()); + values = cache.read("k2", null, Filter.alwaysTrue(), Filter.alwaysTrue()); assertThat(values).containsExactlyElementsOf(expect); // test cache - values = cache.read("k2", Filter.alwaysTrue(), Filter.alwaysTrue()); + values = cache.read("k2", null, Filter.alwaysTrue(), Filter.alwaysTrue()); assertThat(values).containsExactlyElementsOf(expect); // test filter values = - cache.read("k2", Filter.alwaysTrue(), r -> r.getString(0).toString().endsWith("2")); + cache.read( + "k2", + null, + Filter.alwaysTrue(), + r -> r.getString(0).toString().endsWith("2")); assertThat(values).containsExactly("v2"); // test load filter expect = Arrays.asList("v1", "v2", "v3"); map.put("k3", expect); values = - cache.read("k3", r -> r.getString(0).toString().endsWith("2"), Filter.alwaysTrue()); + cache.read( + "k3", + null, + r -> r.getString(0).toString().endsWith("2"), + Filter.alwaysTrue()); assertThat(values).containsExactly("v2"); // test load filter empty expect = Arrays.asList("v1", "v2", "v3"); map.put("k4", expect); values = - cache.read("k4", r -> r.getString(0).toString().endsWith("5"), Filter.alwaysTrue()); + cache.read( + "k4", + null, + r -> r.getString(0).toString().endsWith("5"), + Filter.alwaysTrue()); assertThat(values).isEmpty(); } 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 717f998959a8..abf82342a5aa 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 @@ -19,7 +19,6 @@ package org.apache.paimon.format.avro; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; @@ -50,9 +49,9 @@ public AvroBulkFormat(RowType projectedRowType) { } @Override - public RecordReader createReader(FormatReaderContext formatReaderContext) + public RecordReader createReader(FormatReaderFactory.Context context) throws IOException { - return new AvroReader(formatReaderContext.getFileIO(), formatReaderContext.getFile()); + return new AvroReader(context.fileIO(), context.filePath(), context.fileSize()); } private class AvroReader implements RecordReader { @@ -63,9 +62,9 @@ private class AvroReader implements RecordReader { private final long end; private final Pool pool; - private AvroReader(FileIO fileIO, Path path) throws IOException { + private AvroReader(FileIO fileIO, Path path, long fileSize) throws IOException { this.fileIO = fileIO; - this.end = fileIO.getFileSize(path); + this.end = fileSize; this.reader = createReaderFromPath(path, end); this.reader.sync(0); this.pool = new Pool<>(1); 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 cdc46139f790..55cff92980bf 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 @@ -23,12 +23,11 @@ import org.apache.paimon.data.columnar.ColumnarRow; import org.apache.paimon.data.columnar.ColumnarRowIterator; import org.apache.paimon.data.columnar.VectorizedColumnBatch; -import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.format.FormatReaderFactory; +import org.apache.paimon.format.OrcFormatReaderContext; import org.apache.paimon.format.fs.HadoopReadOnlyFileSystem; import org.apache.paimon.format.orc.filter.OrcFilters; import org.apache.paimon.fs.FileIO; -import org.apache.paimon.fs.Path; import org.apache.paimon.reader.RecordReader.RecordIterator; import org.apache.paimon.types.DataType; import org.apache.paimon.types.RowType; @@ -89,22 +88,23 @@ public OrcReaderFactory( // ------------------------------------------------------------------------ @Override - public OrcVectorizedReader createReader(FormatReaderContext context) throws IOException { - int poolSize = context.getPoolSize() == null ? 1 : context.getPoolSize(); + public OrcVectorizedReader createReader(FormatReaderFactory.Context context) + throws IOException { + int poolSize = + context instanceof OrcFormatReaderContext + ? ((OrcFormatReaderContext) context).poolSize() + : 1; Pool poolOfBatches = createPoolOfBatches(poolSize); - FileIO fileIO = context.getFileIO(); - Long fileSize = context.getFileSize(); - Path file = context.getFile(); RecordReader orcReader = createRecordReader( hadoopConfigWrapper.getHadoopConfig(), schema, conjunctPredicates, - fileIO, - file, + context.fileIO(), + context.filePath(), 0, - fileSize == null ? fileIO.getFileSize(file) : fileSize); + context.fileSize()); return new OrcVectorizedReader(orcReader, poolOfBatches); } 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 29cf45a65260..ed778c0bf018 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 @@ -24,13 +24,10 @@ import org.apache.paimon.data.columnar.ColumnarRowIterator; import org.apache.paimon.data.columnar.VectorizedColumnBatch; import org.apache.paimon.data.columnar.writable.WritableColumnVector; -import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.format.parquet.reader.ColumnReader; import org.apache.paimon.format.parquet.reader.ParquetDecimalVector; import org.apache.paimon.format.parquet.reader.ParquetTimestampVector; -import org.apache.paimon.fs.FileIO; -import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.reader.RecordReader.RecordIterator; @@ -88,19 +85,15 @@ public ParquetReaderFactory(Options conf, RowType projectedType, int batchSize) } @Override - public ParquetReader createReader(FormatReaderContext context) throws IOException { - Path filePath = context.getFile(); - FileIO fileIO = context.getFileIO(); - Long fileSize = context.getFileSize(); - final long splitOffset = 0; - final long splitLength = fileSize == null ? fileIO.getFileSize(filePath) : fileSize; - + public ParquetReader createReader(FormatReaderFactory.Context context) throws IOException { ParquetReadOptions.Builder builder = - ParquetReadOptions.builder().withRange(splitOffset, splitOffset + splitLength); + ParquetReadOptions.builder().withRange(0, context.fileSize()); setReadOptions(builder); ParquetFileReader reader = - new ParquetFileReader(ParquetInputFile.fromPath(fileIO, filePath), builder.build()); + new ParquetFileReader( + ParquetInputFile.fromPath(context.fileIO(), context.filePath()), + builder.build()); MessageType fileSchema = reader.getFileMetaData().getSchema(); MessageType requestedSchema = clipParquetSchema(fileSchema); reader.setRequestedSchema(requestedSchema); diff --git a/paimon-format/src/test/java/org/apache/paimon/format/BulkFileFormatTest.java b/paimon-format/src/test/java/org/apache/paimon/format/BulkFileFormatTest.java index a6225909c3f8..da852eb7003c 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/BulkFileFormatTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/BulkFileFormatTest.java @@ -80,7 +80,13 @@ public void testFormatWriteRead( // read RecordReader reader = - fileFormat.createReaderFactory(rowType).createReader(new LocalFileIO(), path); + fileFormat + .createReaderFactory(rowType) + .createReader( + new FormatReaderContext( + new LocalFileIO(), + path, + new LocalFileIO().getFileSize(path))); List result = new ArrayList<>(); reader.forEachRemaining( rowData -> result.add(GenericRow.of(rowData.getInt(0), rowData.getInt(0)))); diff --git a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcReaderFactoryTest.java b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcReaderFactoryTest.java index 5a0f4925ddbc..1efd984965bf 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcReaderFactoryTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcReaderFactoryTest.java @@ -20,6 +20,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.format.FormatReaderContext; +import org.apache.paimon.format.OrcFormatReaderContext; import org.apache.paimon.format.orc.filter.OrcFilters; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; @@ -148,7 +149,10 @@ void testReadRowPosition() throws IOException { AtomicInteger cnt = new AtomicInteger(0); AtomicLong totalF0 = new AtomicLong(0); - try (RecordReader reader = format.createReader(new LocalFileIO(), flatFile)) { + LocalFileIO fileIO = new LocalFileIO(); + try (RecordReader reader = + format.createReader( + new FormatReaderContext(fileIO, flatFile, fileIO.getFileSize(flatFile)))) { reader.forEachRemainingWithPosition( (rowPosition, row) -> { assertThat(row.isNullAt(0)).isFalse(); @@ -183,7 +187,11 @@ void testReadRowPositionWithRandomFilterAndPool() throws IOException { LocalFileIO localFileIO = new LocalFileIO(); try (RecordReader reader = format.createReader( - new FormatReaderContext(localFileIO, flatFile, randomPooSize, null))) { + new OrcFormatReaderContext( + localFileIO, + flatFile, + localFileIO.getFileSize(flatFile), + randomPooSize))) { reader.forEachRemainingWithPosition( (rowPosition, row) -> { // check filter: _col0 > randomStart @@ -208,7 +216,11 @@ void testReadRowPositionWithTransformAndFilter() throws IOException { LocalFileIO localFileIO = new LocalFileIO(); try (RecordReader reader = format.createReader( - new FormatReaderContext(localFileIO, flatFile, randomPooSize, null))) { + new OrcFormatReaderContext( + localFileIO, + flatFile, + localFileIO.getFileSize(flatFile), + randomPooSize))) { reader.transform(row -> row) .filter(row -> row.getInt(1) % 123 == 0) .forEachRemainingWithPosition( @@ -270,12 +282,17 @@ protected OrcReaderFactory createFormat( private RecordReader createReader(OrcReaderFactory format, Path split) throws IOException { - return format.createReader(new LocalFileIO(), split); + LocalFileIO fileIO = new LocalFileIO(); + return format.createReader( + new FormatReaderContext(fileIO, split, fileIO.getFileSize(split))); } private void forEach(OrcReaderFactory format, Path file, Consumer action) throws IOException { - RecordReader reader = format.createReader(new LocalFileIO(), file); + LocalFileIO fileIO = new LocalFileIO(); + RecordReader reader = + format.createReader( + new FormatReaderContext(fileIO, file, fileIO.getFileSize(file))); reader.forEachRemaining(action); } 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 bf2b7217db52..d56edea5959b 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 @@ -25,6 +25,7 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.Timestamp; +import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.format.FormatWriter; import org.apache.paimon.format.parquet.writer.RowDataParquetBuilder; import org.apache.paimon.fs.Path; @@ -232,7 +233,12 @@ void testProjection(int rowGroupSize) throws IOException { 500); AtomicInteger cnt = new AtomicInteger(0); - RecordReader reader = format.createReader(new LocalFileIO(), testPath); + RecordReader reader = + format.createReader( + new FormatReaderContext( + new LocalFileIO(), + testPath, + new LocalFileIO().getFileSize(testPath))); reader.forEachRemaining( row -> { int i = cnt.get(); @@ -270,7 +276,12 @@ void testProjectionReadUnknownField(int rowGroupSize) throws IOException { 500); AtomicInteger cnt = new AtomicInteger(0); - RecordReader reader = format.createReader(new LocalFileIO(), testPath); + RecordReader reader = + format.createReader( + new FormatReaderContext( + new LocalFileIO(), + testPath, + new LocalFileIO().getFileSize(testPath))); reader.forEachRemaining( row -> { int i = cnt.get(); @@ -303,7 +314,12 @@ void testReadRowPosition() throws IOException { batchSize); AtomicInteger cnt = new AtomicInteger(0); - try (RecordReader reader = format.createReader(new LocalFileIO(), testPath)) { + try (RecordReader reader = + format.createReader( + new FormatReaderContext( + new LocalFileIO(), + testPath, + new LocalFileIO().getFileSize(testPath)))) { reader.forEachRemainingWithPosition( (rowPosition, row) -> { assertThat(row.getDouble(0)).isEqualTo(cnt.get()); @@ -353,7 +369,10 @@ private int testReadingFile(List expected, Path path) throws IOExceptio throw new IOException(e); } - RecordReader reader = format.createReader(new LocalFileIO(), path); + RecordReader reader = + format.createReader( + new FormatReaderContext( + new LocalFileIO(), path, new LocalFileIO().getFileSize(path))); AtomicInteger cnt = new AtomicInteger(0); final AtomicReference previousRow = new AtomicReference<>(); From 907525dd5e5bba57a12cdb6b6e065ed9e805beaf Mon Sep 17 00:00:00 2001 From: Aitozi Date: Tue, 19 Mar 2024 17:32:25 +0800 Subject: [PATCH 28/79] [core] support create tag based on tag (#3044) --- .../paimon/table/AbstractFileStoreTable.java | 19 +++++- .../org/apache/paimon/utils/TagManager.java | 28 +++++---- .../paimon/table/FileStoreTableTestBase.java | 59 +++++++++++++++++++ 3 files changed, 93 insertions(+), 13 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java index 39b368982ae1..d2c840ba9526 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java @@ -65,6 +65,7 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.SortedMap; import java.util.function.BiConsumer; import static org.apache.paimon.CoreOptions.PATH; @@ -427,11 +428,25 @@ public void rollbackTo(long snapshotId) { @Override public void createTag(String tagName, long fromSnapshotId) { SnapshotManager snapshotManager = snapshotManager(); + Snapshot snapshot = null; + if (snapshotManager.snapshotExists(fromSnapshotId)) { + snapshot = snapshotManager.snapshot(fromSnapshotId); + } else { + SortedMap> tags = tagManager().tags(); + for (Snapshot snap : tags.keySet()) { + if (snap.id() == fromSnapshotId) { + snapshot = snap; + break; + } else if (snap.id() > fromSnapshotId) { + break; + } + } + } checkArgument( - snapshotManager.snapshotExists(fromSnapshotId), + snapshot != null, "Cannot create tag because given snapshot #%s doesn't exist.", fromSnapshotId); - createTag(tagName, snapshotManager.snapshot(fromSnapshotId)); + createTag(tagName, snapshot); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java b/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java index a29a3e151c76..90f690053160 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java @@ -77,18 +77,24 @@ public Path branchTagPath(String branchName, String tagName) { /** Create a tag from given snapshot and save it in the storage. */ public void createTag(Snapshot snapshot, String tagName, List callbacks) { checkArgument(!StringUtils.isBlank(tagName), "Tag name '%s' is blank.", tagName); - checkArgument(!tagExists(tagName), "Tag name '%s' already exists.", tagName); - Path newTagPath = tagPath(tagName); - try { - fileIO.writeFileUtf8(newTagPath, snapshot.toJson()); - } catch (IOException e) { - throw new RuntimeException( - String.format( - "Exception occurs when committing tag '%s' (path %s). " - + "Cannot clean up because we can't determine the success.", - tagName, newTagPath), - e); + // skip create tag for the same snapshot of the same name. + if (tagExists(tagName)) { + Snapshot tagged = taggedSnapshot(tagName); + Preconditions.checkArgument( + tagged.id() == snapshot.id(), "Tag name '%s' already exists.", tagName); + } else { + Path newTagPath = tagPath(tagName); + try { + fileIO.writeFileUtf8(newTagPath, snapshot.toJson()); + } catch (IOException e) { + throw new RuntimeException( + String.format( + "Exception occurs when committing tag '%s' (path %s). " + + "Cannot clean up because we can't determine the success.", + tagName, newTagPath), + e); + } } try { diff --git a/paimon-core/src/test/java/org/apache/paimon/table/FileStoreTableTestBase.java b/paimon-core/src/test/java/org/apache/paimon/table/FileStoreTableTestBase.java index 4cd019568c41..fabd61639207 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/FileStoreTableTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/FileStoreTableTestBase.java @@ -68,6 +68,7 @@ import org.apache.paimon.utils.TagManager; import org.apache.paimon.utils.TraceableFileIO; +import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -915,6 +916,64 @@ public void testCreateTag() throws Exception { assertThat(tagged.equals(snapshot2)).isTrue(); } + @Test + public void testCreateTagOnExpiredSnapshot() throws Exception { + FileStoreTable table = + createFileStoreTable( + conf -> { + conf.set(SNAPSHOT_NUM_RETAINED_MAX, 1); + conf.set(SNAPSHOT_NUM_RETAINED_MIN, 1); + }); + try (StreamTableWrite write = table.newWrite(commitUser); + StreamTableCommit commit = table.newCommit(commitUser)) { + // snapshot 1 + write.write(rowData(1, 10, 100L)); + commit.commit(0, write.prepareCommit(false, 1)); + table.createTag("test-tag", 1); + // verify that tag file exist + TagManager tagManager = new TagManager(new TraceableFileIO(), tablePath); + assertThat(tagManager.tagExists("test-tag")).isTrue(); + // verify that test-tag is equal to snapshot 1 + Snapshot tagged = tagManager.taggedSnapshot("test-tag"); + Snapshot snapshot1 = table.snapshotManager().snapshot(1); + assertThat(tagged.equals(snapshot1)).isTrue(); + // snapshot 2 + write.write(rowData(2, 20, 200L)); + commit.commit(1, write.prepareCommit(false, 2)); + SnapshotManager snapshotManager = new SnapshotManager(new TraceableFileIO(), tablePath); + // The snapshot 1 is expired. + assertThat(snapshotManager.snapshotExists(1)).isFalse(); + table.createTag("test-tag-2", 1); + // verify that tag file exist + assertThat(tagManager.tagExists("test-tag-2")).isTrue(); + // verify that test-tag is equal to snapshot 1 + Snapshot tag2 = tagManager.taggedSnapshot("test-tag-2"); + assertThat(tag2.equals(snapshot1)).isTrue(); + } + } + + @Test + public void testCreateSameTagName() throws Exception { + FileStoreTable table = createFileStoreTable(); + try (StreamTableWrite write = table.newWrite(commitUser); + StreamTableCommit commit = table.newCommit(commitUser)) { + // snapshot 1 + write.write(rowData(1, 10, 100L)); + commit.commit(0, write.prepareCommit(false, 1)); + // snapshot 2 + write.write(rowData(1, 10, 100L)); + commit.commit(1, write.prepareCommit(false, 2)); + TagManager tagManager = new TagManager(new TraceableFileIO(), tablePath); + table.createTag("test-tag", 1); + // verify that tag file exist + assertThat(tagManager.tagExists("test-tag")).isTrue(); + // Create again + table.createTag("test-tag", 1); + Assertions.assertThatThrownBy(() -> table.createTag("test-tag", 2)) + .hasMessageContaining("Tag name 'test-tag' already exists."); + } + } + @Test public void testCreateBranch() throws Exception { FileStoreTable table = createFileStoreTable(); From e589a422c6fe121ac171ae0d30a39a321de8d5b7 Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Wed, 20 Mar 2024 12:42:37 +0800 Subject: [PATCH 29/79] [flink] Remove compatibility utils that are used for flink-1.14 (#3054) --- .../sink/cdc/FlinkCdcMultiTableSink.java | 6 +- .../apache/paimon/flink/TableConfigUtils.java | 52 ---------------- .../paimon/flink/action/ActionBase.java | 3 +- .../paimon/flink/action/CompactAction.java | 3 +- .../flink/action/CompactDatabaseAction.java | 5 +- .../paimon/flink/action/TableActionBase.java | 34 ++++++++++- .../paimon/flink/procedure/ProcedureBase.java | 5 +- .../paimon/flink/service/QueryService.java | 3 +- .../apache/paimon/flink/sink/FlinkSink.java | 6 +- .../flink/sink/MultiTablesCompactorSink.java | 6 +- .../StreamExecutionEnvironmentUtils.java | 30 ---------- .../flink/utils/TableEnvironmentUtils.java | 60 ------------------- 12 files changed, 43 insertions(+), 170 deletions(-) delete mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/TableConfigUtils.java delete mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/StreamExecutionEnvironmentUtils.java delete mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableEnvironmentUtils.java 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 99d933d89c1b..c0156e841cfa 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 @@ -32,7 +32,6 @@ import org.apache.paimon.flink.sink.StoreSinkWrite; import org.apache.paimon.flink.sink.StoreSinkWriteImpl; import org.apache.paimon.flink.sink.WrappedManifestCommittableSerializer; -import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils; import org.apache.paimon.manifest.WrappedManifestCommittable; import org.apache.paimon.options.MemorySize; import org.apache.paimon.options.Options; @@ -133,10 +132,7 @@ public DataStreamSink sinkFrom( createCommittableStateManager())) .setParallelism(input.getParallelism()); configureGlobalCommitter( - committed, - commitCpuCores, - commitHeapMemory, - StreamExecutionEnvironmentUtils.getConfiguration(env)); + committed, commitCpuCores, commitHeapMemory, env.getConfiguration()); return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/TableConfigUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/TableConfigUtils.java deleted file mode 100644 index ffeb9b92ea43..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/TableConfigUtils.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.TableConfig; - -/** Utils for {@link TableConfig}. */ -public class TableConfigUtils { - - public static Configuration extractConfiguration(ReadableConfig readableConfig) { - Configuration to = new Configuration(); - copyConfiguration(readableConfig, to); - return to; - } - - private static void copyConfiguration(ReadableConfig from, Configuration to) { - if (from instanceof Configuration) { - to.addAll((Configuration) from); - return; - } - - if (!(from instanceof TableConfig)) { - throw new RuntimeException("Unknown readableConfig type: " + from.getClass()); - } - - TableConfig tableConfig = (TableConfig) from; - - // copy root configuration first - copyConfiguration(tableConfig.getRootConfiguration(), to); - - // copy table configuration - to.addAll(tableConfig.getConfiguration()); - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java index 7f4c03a38f29..3a9e39b4eda1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java @@ -23,7 +23,6 @@ import org.apache.paimon.flink.FlinkCatalog; import org.apache.paimon.flink.FlinkCatalogFactory; import org.apache.paimon.flink.LogicalTypeConversion; -import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils; import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; import org.apache.paimon.types.DataType; @@ -80,7 +79,7 @@ private void initFlinkEnv(StreamExecutionEnvironment env) { } protected void execute(String defaultName) throws Exception { - ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env); + ReadableConfig conf = env.getConfiguration(); String name = conf.getOptional(PipelineOptions.NAME).orElse(defaultName); env.execute(name); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java index 313a7a5d44d9..e24f8d9a9aab 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java @@ -22,7 +22,6 @@ import org.apache.paimon.flink.compact.UnawareBucketCompactionTopoBuilder; import org.apache.paimon.flink.sink.CompactorSinkBuilder; import org.apache.paimon.flink.source.CompactorSourceBuilder; -import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils; import org.apache.paimon.table.FileStoreTable; import org.apache.flink.api.common.RuntimeExecutionMode; @@ -75,7 +74,7 @@ public CompactAction withPartitions(List> partitions) { @Override public void build() { - ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env); + ReadableConfig conf = env.getConfiguration(); boolean isStreaming = conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING; FileStoreTable fileStoreTable = (FileStoreTable) table; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java index a97b3048e822..408d5768cf4c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java @@ -28,7 +28,6 @@ import org.apache.paimon.flink.sink.MultiTablesCompactorSink; import org.apache.paimon.flink.source.CompactorSourceBuilder; import org.apache.paimon.flink.source.MultiTablesCompactorSourceBuilder; -import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; @@ -161,7 +160,7 @@ private void buildForDividedMode() { !tableMap.isEmpty(), "no tables to be compacted. possible cause is that there are no tables detected after pattern matching"); - ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env); + ReadableConfig conf = env.getConfiguration(); boolean isStreaming = conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING; for (Map.Entry entry : tableMap.entrySet()) { @@ -186,7 +185,7 @@ private void buildForDividedMode() { private void buildForCombinedMode() { - ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env); + ReadableConfig conf = env.getConfiguration(); boolean isStreaming = conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING; // TODO: Currently, multi-tables compaction don't support tables which bucketmode is UNWARE. diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/TableActionBase.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/TableActionBase.java index 06d234469ea2..a97335cd2763 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/TableActionBase.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/TableActionBase.java @@ -22,7 +22,6 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.sink.FlinkSinkBuilder; -import org.apache.paimon.flink.utils.TableEnvironmentUtils; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.utils.Preconditions; @@ -30,8 +29,11 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.data.RowData; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -68,8 +70,34 @@ public TableResult batchSink(DataStream dataStream) { List sinkIdentifierNames = Collections.singletonList(identifier.getFullName()); - return TableEnvironmentUtils.executeInternal( - batchTEnv, transformations, sinkIdentifierNames); + return executeInternal(transformations, sinkIdentifierNames); + } + + /** + * Invoke {@code TableEnvironmentImpl#executeInternal(List>, List)} + * from a {@link StreamTableEnvironment} instance through reflecting. + */ + private TableResult executeInternal( + List> transformations, List sinkIdentifierNames) { + Class clazz = batchTEnv.getClass().getSuperclass().getSuperclass(); + try { + Method executeInternal = + clazz.getDeclaredMethod("executeInternal", List.class, List.class); + executeInternal.setAccessible(true); + + return (TableResult) + executeInternal.invoke(batchTEnv, transformations, sinkIdentifierNames); + } catch (NoSuchMethodException e) { + throw new RuntimeException( + "Failed to get 'TableEnvironmentImpl#executeInternal(List, List)' method " + + "from given StreamTableEnvironment instance by Java reflection. This is unexpected.", + e); + } catch (IllegalAccessException | InvocationTargetException e) { + throw new RuntimeException( + "Failed to invoke 'TableEnvironmentImpl#executeInternal(List, List)' method " + + "from given StreamTableEnvironment instance by Java reflection. This is unexpected.", + e); + } } /** diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ProcedureBase.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ProcedureBase.java index fd7f74148889..7d5542109d28 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ProcedureBase.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ProcedureBase.java @@ -22,7 +22,6 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.factories.Factory; import org.apache.paimon.flink.action.ActionBase; -import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils; import org.apache.paimon.table.Table; import org.apache.paimon.utils.StringUtils; @@ -69,13 +68,13 @@ protected String[] execute( protected String[] execute(ProcedureContext procedureContext, JobClient jobClient) { StreamExecutionEnvironment env = procedureContext.getExecutionEnvironment(); - ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env); + ReadableConfig conf = env.getConfiguration(); return execute(jobClient, conf.get(TABLE_DML_SYNC)); } protected String[] execute(StreamExecutionEnvironment env, String defaultJobName) throws Exception { - ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env); + ReadableConfig conf = env.getConfiguration(); String name = conf.getOptional(PipelineOptions.NAME).orElse(defaultJobName); return execute(env.executeAsync(name), conf.get(TABLE_DML_SYNC)); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java index 5b2c13c8488d..8a4814d0ae4c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java @@ -20,7 +20,6 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.flink.utils.InternalTypeInfo; -import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; @@ -38,7 +37,7 @@ public class QueryService { public static void build(StreamExecutionEnvironment env, Table table, int parallelism) { - ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env); + ReadableConfig conf = env.getConfiguration(); Preconditions.checkArgument( conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING, "Query Service only supports streaming mode."); 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 545bd7f07072..582fcfc35af2 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 @@ -20,7 +20,6 @@ import org.apache.paimon.CoreOptions.ChangelogProducer; import org.apache.paimon.CoreOptions.TagCreationMode; -import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils; import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.options.MemorySize; import org.apache.paimon.options.Options; @@ -190,8 +189,7 @@ public DataStream doWrite( DataStream input, String commitUser, @Nullable Integer parallelism) { StreamExecutionEnvironment env = input.getExecutionEnvironment(); boolean isStreaming = - StreamExecutionEnvironmentUtils.getConfiguration(env) - .get(ExecutionOptions.RUNTIME_MODE) + env.getConfiguration().get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING; boolean writeOnly = table.coreOptions().writeOnly(); @@ -222,7 +220,7 @@ public DataStream doWrite( protected DataStreamSink doCommit(DataStream written, String commitUser) { StreamExecutionEnvironment env = written.getExecutionEnvironment(); - ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env); + ReadableConfig conf = env.getConfiguration(); CheckpointConfig checkpointConfig = env.getCheckpointConfig(); boolean isStreaming = conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesCompactorSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesCompactorSink.java index d9ded153d82a..659296633155 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesCompactorSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesCompactorSink.java @@ -20,7 +20,6 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.flink.VersionedSerializerWrapper; -import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils; import org.apache.paimon.manifest.WrappedManifestCommittable; import org.apache.paimon.options.Options; @@ -87,8 +86,7 @@ public SingleOutputStreamOperator doWrite( DataStream input, String commitUser, Integer parallelism) { StreamExecutionEnvironment env = input.getExecutionEnvironment(); boolean isStreaming = - StreamExecutionEnvironmentUtils.getConfiguration(env) - .get(ExecutionOptions.RUNTIME_MODE) + env.getConfiguration().get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING; SingleOutputStreamOperator written = @@ -112,7 +110,7 @@ public SingleOutputStreamOperator doWrite( protected DataStreamSink doCommit( DataStream written, String commitUser) { StreamExecutionEnvironment env = written.getExecutionEnvironment(); - ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env); + ReadableConfig conf = env.getConfiguration(); CheckpointConfig checkpointConfig = env.getCheckpointConfig(); boolean isStreaming = conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/StreamExecutionEnvironmentUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/StreamExecutionEnvironmentUtils.java deleted file mode 100644 index 9f3b28bbe764..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/StreamExecutionEnvironmentUtils.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.utils; - -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; - -/** Utility methods for {@link StreamExecutionEnvironment}. */ -public class StreamExecutionEnvironmentUtils { - - public static ReadableConfig getConfiguration(StreamExecutionEnvironment env) { - return env.getConfiguration(); - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableEnvironmentUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableEnvironmentUtils.java deleted file mode 100644 index 6a5820ce84e7..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableEnvironmentUtils.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.flink.utils; - -import org.apache.flink.api.dag.Transformation; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableResult; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; - -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.util.List; - -/** Utility methods for {@link TableEnvironment} and its subclasses. */ -public class TableEnvironmentUtils { - - /** - * Invoke {@code TableEnvironmentImpl#executeInternal(List>, List)} - * from a {@link StreamTableEnvironment} instance through reflecting. - */ - public static TableResult executeInternal( - StreamTableEnvironment tEnv, - List> transformations, - List sinkIdentifierNames) { - Class clazz = tEnv.getClass().getSuperclass().getSuperclass(); - try { - Method executeInternal = - clazz.getDeclaredMethod("executeInternal", List.class, List.class); - executeInternal.setAccessible(true); - - return (TableResult) executeInternal.invoke(tEnv, transformations, sinkIdentifierNames); - } catch (NoSuchMethodException e) { - throw new RuntimeException( - "Failed to get 'TableEnvironmentImpl#executeInternal(List, List)' method " - + "from given StreamTableEnvironment instance by Java reflection. This is unexpected.", - e); - } catch (IllegalAccessException | InvocationTargetException e) { - throw new RuntimeException( - "Failed to invoke 'TableEnvironmentImpl#executeInternal(List, List)' method " - + "from given StreamTableEnvironment instance by Java reflection. This is unexpected.", - e); - } - } -} From da28bc27c8cccdd7d9c71fce53df917051b292fc Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Wed, 20 Mar 2024 13:20:24 +0800 Subject: [PATCH 30/79] [core] Add validation for fields prefix (#3052) --- .../paimon/schema/SchemaValidation.java | 19 +++++++++++++++++++ .../apache/paimon/schema/TableSchemaTest.java | 18 ++++++++++++++++++ .../paimon/flink/PartialUpdateITCase.java | 3 +-- 3 files changed, 38 insertions(+), 2 deletions(-) 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 cb52570cd2da..7e59bb7b51b8 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 @@ -90,6 +90,8 @@ public static void validateTableSchema(TableSchema schema) { validateStartupMode(options); + validateFieldsPrefix(schema, options); + validateSequenceField(schema, options); validateSequenceGroup(schema, options); @@ -347,6 +349,23 @@ private static String concatConfigKeys(List> configOptions) { return configOptions.stream().map(ConfigOption::key).collect(Collectors.joining(",")); } + private static void validateFieldsPrefix(TableSchema schema, CoreOptions options) { + List fieldNames = schema.fieldNames(); + options.toMap() + .keySet() + .forEach( + k -> { + if (k.startsWith(FIELDS_PREFIX)) { + String fieldName = k.split("\\.")[1]; + checkArgument( + fieldNames.contains(fieldName), + String.format( + "Field %s can not be found in table schema.", + fieldName)); + } + }); + } + private static void validateSequenceGroup(TableSchema schema, CoreOptions options) { Map> fields2Group = new HashMap<>(); for (Map.Entry entry : options.toMap().entrySet()) { diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java b/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java index a39ce083bf22..0927c9e9165c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java @@ -161,6 +161,24 @@ public void testSequenceField() { .hasMessageContaining("Should not define aggregation on sequence field: 'f3'."); } + @Test + public void testFieldsPrefix() { + List fields = + Arrays.asList( + new DataField(0, "f0", DataTypes.INT()), + new DataField(1, "f1", DataTypes.INT()), + new DataField(2, "f2", DataTypes.INT())); + List primaryKeys = Collections.singletonList("f0"); + Map options = new HashMap<>(); + options.put(MERGE_ENGINE.key(), CoreOptions.MergeEngine.AGGREGATE.toString()); + options.put(FIELDS_PREFIX + ".f1." + AGG_FUNCTION, "max"); + options.put(FIELDS_PREFIX + ".fake_col." + AGG_FUNCTION, "max"); + TableSchema schema = + new TableSchema(1, fields, 10, Collections.emptyList(), primaryKeys, options, ""); + assertThatThrownBy(() -> validateTableSchema(schema)) + .hasMessageContaining("Field fake_col can not be found in table schema."); + } + static RowType newRowType(boolean isNullable, int fieldId) { return new RowType( isNullable, diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PartialUpdateITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PartialUpdateITCase.java index e59c6dc8fd99..f4cf0b67f24b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PartialUpdateITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PartialUpdateITCase.java @@ -217,8 +217,7 @@ public void testInvalidSequenceGroup() { + "'merge-engine'='partial-update', " + "'fields.g_0.sequence-group'='a,b', " + "'fields.g_2.sequence-group'='c,d');")) - .hasRootCauseMessage( - "The sequence field group: g_0 can not be found in table schema."); + .hasRootCauseMessage("Field g_0 can not be found in table schema."); Assertions.assertThatThrownBy( () -> From 7a44d53c2ff963e6291b493df78b054484b88a13 Mon Sep 17 00:00:00 2001 From: Stephen0421 <60921147+Stephen0421@users.noreply.github.com> Date: Wed, 20 Mar 2024 13:38:00 +0800 Subject: [PATCH 31/79] [core] add initial map size to solve load hash index slowly (#3051) --- .../org/apache/paimon/utils/Int2ShortHashMap.java | 4 ++++ .../org/apache/paimon/index/PartitionIndex.java | 13 ++++++++++++- 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/Int2ShortHashMap.java b/paimon-common/src/main/java/org/apache/paimon/utils/Int2ShortHashMap.java index a271bee6636f..453f04974ab5 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/Int2ShortHashMap.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/Int2ShortHashMap.java @@ -29,6 +29,10 @@ public Int2ShortHashMap() { this.map = new Int2ShortOpenHashMap(); } + public Int2ShortHashMap(int capacity) { + this.map = new Int2ShortOpenHashMap(capacity); + } + public void put(int key, short value) { map.put(key, value); } diff --git a/paimon-core/src/main/java/org/apache/paimon/index/PartitionIndex.java b/paimon-core/src/main/java/org/apache/paimon/index/PartitionIndex.java index 28428aa2d22f..e9667cfe380e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/index/PartitionIndex.java +++ b/paimon-core/src/main/java/org/apache/paimon/index/PartitionIndex.java @@ -22,6 +22,7 @@ import org.apache.paimon.manifest.IndexManifestEntry; import org.apache.paimon.utils.Int2ShortHashMap; import org.apache.paimon.utils.IntIterator; +import org.apache.paimon.utils.MathUtils; import java.io.EOFException; import java.io.IOException; @@ -111,8 +112,8 @@ public static PartitionIndex loadIndex( long targetBucketRowNumber, IntPredicate loadFilter, IntPredicate bucketFilter) { - Int2ShortHashMap map = new Int2ShortHashMap(); List files = indexFileHandler.scan(HASH_INDEX, partition); + Int2ShortHashMap map = new Int2ShortHashMap(calculateInitialMapSize(files)); Map buckets = new HashMap<>(); for (IndexManifestEntry file : files) { try (IntIterator iterator = indexFileHandler.readHashIndex(file.indexFile())) { @@ -137,4 +138,14 @@ public static PartitionIndex loadIndex( } return new PartitionIndex(map, buckets, targetBucketRowNumber); } + + private static int calculateInitialMapSize(List files) { + long size = 16; + for (IndexManifestEntry file : files) { + size = Math.max(size, file.indexFile().rowCount()); + } + return MathUtils.isPowerOf2(size) + ? (int) size + : MathUtils.roundDownToPowerOf2((int) size) * 2; + } } From e1afe3cce720af98a5196297f8e8d331bdb1986a Mon Sep 17 00:00:00 2001 From: Jingsong Date: Wed, 20 Mar 2024 13:48:25 +0800 Subject: [PATCH 32/79] [core] Introduce Int2ShortHashMap.Builder to to accelerate init --- .../apache/paimon/utils/Int2ShortHashMap.java | 26 +++++++++++++++++++ .../apache/paimon/index/PartitionIndex.java | 17 +++--------- 2 files changed, 29 insertions(+), 14 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/Int2ShortHashMap.java b/paimon-common/src/main/java/org/apache/paimon/utils/Int2ShortHashMap.java index 453f04974ab5..0338d6d61c72 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/Int2ShortHashMap.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/Int2ShortHashMap.java @@ -19,6 +19,8 @@ package org.apache.paimon.utils; import it.unimi.dsi.fastutil.ints.Int2ShortOpenHashMap; +import it.unimi.dsi.fastutil.ints.IntArrayList; +import it.unimi.dsi.fastutil.shorts.ShortArrayList; /** Int to short hash map. */ public class Int2ShortHashMap { @@ -48,4 +50,28 @@ public short get(int key) { public int size() { return map.size(); } + + public static Builder builder() { + return new Builder(); + } + + /** Builder of {@link Int2ShortHashMap}. */ + public static class Builder { + + private final IntArrayList keyList = new IntArrayList(); + private final ShortArrayList valueList = new ShortArrayList(); + + public void put(int key, short value) { + keyList.add(key); + valueList.add(value); + } + + public Int2ShortHashMap build() { + Int2ShortHashMap map = new Int2ShortHashMap(keyList.size()); + for (int i = 0; i < keyList.size(); i++) { + map.put(keyList.getInt(i), valueList.getShort(i)); + } + return map; + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/index/PartitionIndex.java b/paimon-core/src/main/java/org/apache/paimon/index/PartitionIndex.java index e9667cfe380e..79ff72656216 100644 --- a/paimon-core/src/main/java/org/apache/paimon/index/PartitionIndex.java +++ b/paimon-core/src/main/java/org/apache/paimon/index/PartitionIndex.java @@ -22,7 +22,6 @@ import org.apache.paimon.manifest.IndexManifestEntry; import org.apache.paimon.utils.Int2ShortHashMap; import org.apache.paimon.utils.IntIterator; -import org.apache.paimon.utils.MathUtils; import java.io.EOFException; import java.io.IOException; @@ -113,7 +112,7 @@ public static PartitionIndex loadIndex( IntPredicate loadFilter, IntPredicate bucketFilter) { List files = indexFileHandler.scan(HASH_INDEX, partition); - Int2ShortHashMap map = new Int2ShortHashMap(calculateInitialMapSize(files)); + Int2ShortHashMap.Builder mapBuilder = Int2ShortHashMap.builder(); Map buckets = new HashMap<>(); for (IndexManifestEntry file : files) { try (IntIterator iterator = indexFileHandler.readHashIndex(file.indexFile())) { @@ -121,7 +120,7 @@ public static PartitionIndex loadIndex( try { int hash = iterator.next(); if (loadFilter.test(hash)) { - map.put(hash, (short) file.bucket()); + mapBuilder.put(hash, (short) file.bucket()); } if (bucketFilter.test(file.bucket())) { buckets.compute( @@ -136,16 +135,6 @@ public static PartitionIndex loadIndex( throw new UncheckedIOException(e); } } - return new PartitionIndex(map, buckets, targetBucketRowNumber); - } - - private static int calculateInitialMapSize(List files) { - long size = 16; - for (IndexManifestEntry file : files) { - size = Math.max(size, file.indexFile().rowCount()); - } - return MathUtils.isPowerOf2(size) - ? (int) size - : MathUtils.roundDownToPowerOf2((int) size) * 2; + return new PartitionIndex(mapBuilder.build(), buckets, targetBucketRowNumber); } } From 0e570dc26719f2b5d9757fd7b1be7230eb7475c4 Mon Sep 17 00:00:00 2001 From: Aitozi Date: Wed, 20 Mar 2024 13:50:56 +0800 Subject: [PATCH 33/79] [core] support to drop partition when delete tag (#3042) --- .../metastore/AddPartitionTagCallback.java | 11 ++++++++ .../paimon/metastore/MetastoreClient.java | 2 ++ .../paimon/table/AbstractFileStoreTable.java | 7 ++++- .../apache/paimon/table/sink/TagCallback.java | 2 ++ .../apache/paimon/tag/TagAutoCreation.java | 5 +++- .../org/apache/paimon/utils/TagManager.java | 20 +++++++++++--- .../paimon/operation/FileDeletionTest.java | 6 +++-- .../UncleanedFileStoreExpireTest.java | 6 ++++- .../AutoTagForSavepointCommitterOperator.java | 2 +- .../sink/BatchWriteGeneratorTagOperator.java | 12 ++++++--- .../BatchWriteGeneratorTagOperatorTest.java | 1 + .../paimon/hive/HiveMetastoreClient.java | 14 ++++++++++ .../paimon/hive/HiveCatalogITCaseBase.java | 26 +++++++++++++++++++ 13 files changed, 102 insertions(+), 12 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionTagCallback.java b/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionTagCallback.java index 33f5ed5a94f7..70efe68e83f3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionTagCallback.java +++ b/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionTagCallback.java @@ -44,6 +44,17 @@ public void notifyCreation(String tagName) { } } + @Override + public void notifyDeletion(String tagName) { + LinkedHashMap partitionSpec = new LinkedHashMap<>(); + partitionSpec.put(partitionField, tagName); + try { + client.deletePartition(partitionSpec); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + @Override public void close() throws Exception { client.close(); diff --git a/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java b/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java index 615e783303c9..9247d49232c5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java @@ -33,6 +33,8 @@ public interface MetastoreClient extends AutoCloseable { void addPartition(LinkedHashMap partitionSpec) throws Exception; + void deletePartition(LinkedHashMap partitionSpec) throws Exception; + /** Factory to create {@link MetastoreClient}. */ interface Factory extends Serializable { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java index d2c840ba9526..ac0f798a4b4a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java @@ -462,7 +462,12 @@ private void createTag(String tagName, Snapshot fromSnapshot) { @Override public void deleteTag(String tagName) { - tagManager().deleteTag(tagName, store().newTagDeletion(), snapshotManager()); + tagManager() + .deleteTag( + tagName, + store().newTagDeletion(), + snapshotManager(), + store().createTagCallbacks()); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/TagCallback.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/TagCallback.java index 397b341d9590..1d20bb89db88 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/TagCallback.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/TagCallback.java @@ -26,4 +26,6 @@ public interface TagCallback extends AutoCloseable { void notifyCreation(String tagName); + + void notifyDeletion(String tagName); } diff --git a/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoCreation.java b/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoCreation.java index 31a9b19974f3..505454313bd2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoCreation.java +++ b/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoCreation.java @@ -155,7 +155,10 @@ private void tryToTag(Snapshot snapshot) { int i = 0; for (List tag : tags.values()) { tagManager.deleteTag( - checkAndGetOneAutoTag(tag), tagDeletion, snapshotManager); + checkAndGetOneAutoTag(tag), + tagDeletion, + snapshotManager, + callbacks); i++; if (i == toDelete) { break; diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java b/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java index 90f690053160..134dea459f02 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java @@ -126,7 +126,10 @@ public void deleteAllTagsOfOneSnapshot( } public void deleteTag( - String tagName, TagDeletion tagDeletion, SnapshotManager snapshotManager) { + String tagName, + TagDeletion tagDeletion, + SnapshotManager snapshotManager, + List callbacks) { checkArgument(!StringUtils.isBlank(tagName), "Tag name '%s' is blank.", tagName); checkArgument(tagExists(tagName), "Tag '%s' doesn't exist.", tagName); @@ -135,12 +138,12 @@ public void deleteTag( // skip file deletion if snapshot exists if (snapshotManager.snapshotExists(taggedSnapshot.id())) { - fileIO.deleteQuietly(tagPath(tagName)); + deleteTagMetaFile(tagName, callbacks); return; } else { // FileIO discovers tags by tag file, so we should read all tags before we delete tag SortedMap> tags = tags(); - fileIO.deleteQuietly(tagPath(tagName)); + deleteTagMetaFile(tagName, callbacks); // skip data file clean if more than 1 tags are created based on this snapshot if (tags.get(taggedSnapshot).size() > 1) { @@ -152,6 +155,17 @@ public void deleteTag( doClean(taggedSnapshot, taggedSnapshots, snapshotManager, tagDeletion); } + private void deleteTagMetaFile(String tagName, List callbacks) { + fileIO.deleteQuietly(tagPath(tagName)); + try { + callbacks.forEach(callback -> callback.notifyDeletion(tagName)); + } finally { + for (TagCallback tagCallback : callbacks) { + IOUtils.closeQuietly(tagCallback); + } + } + } + private void doClean( Snapshot taggedSnapshot, List taggedSnapshots, diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java index 9994c0809a88..ec73eb317e4a 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java @@ -425,7 +425,8 @@ public void testDeleteTagWithSnapshot() throws Exception { assertPathExists(fileIO, pathFactory.toManifestListPath(manifestListName)); } - tagManager.deleteTag("tag1", store.newTagDeletion(), snapshotManager); + tagManager.deleteTag( + "tag1", store.newTagDeletion(), snapshotManager, Collections.emptyList()); // check data files assertPathNotExists(fileIO, pathFactory.bucketPath(partition, 0)); @@ -501,7 +502,8 @@ public void testDeleteTagWithOtherTag() throws Exception { assertPathExists(fileIO, pathFactory.toManifestListPath(manifestListName)); } - tagManager.deleteTag("tag2", store.newTagDeletion(), snapshotManager); + tagManager.deleteTag( + "tag2", store.newTagDeletion(), snapshotManager, Collections.emptyList()); // check data files assertPathExists(fileIO, pathFactory.bucketPath(partition, 0)); diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/UncleanedFileStoreExpireTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/UncleanedFileStoreExpireTest.java index ce93166a5bcd..9f5ccb81c8a2 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/UncleanedFileStoreExpireTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/UncleanedFileStoreExpireTest.java @@ -109,7 +109,11 @@ public void testMixedSnapshotAndTagDeletion() throws Exception { // randomly delete tags for (int id = 1; id <= latestSnapshotId; id++) { if (random.nextBoolean()) { - tagManager.deleteTag("tag" + id, store.newTagDeletion(), snapshotManager); + tagManager.deleteTag( + "tag" + id, + store.newTagDeletion(), + snapshotManager, + Collections.emptyList()); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java index da3425e9b0d3..dcf2c8b0045c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java @@ -156,7 +156,7 @@ public void notifyCheckpointAborted(long checkpointId) throws Exception { identifiersForTags.remove(checkpointId); String tagName = SAVEPOINT_TAG_PREFIX + checkpointId; if (tagManager.tagExists(tagName)) { - tagManager.deleteTag(tagName, tagDeletion, snapshotManager); + tagManager.deleteTag(tagName, tagDeletion, snapshotManager, callbacks); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java index d65ab74140fe..2c898831ec2c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java @@ -114,7 +114,8 @@ private void createTag() { try { // If the tag already exists, delete the tag if (tagManager.tagExists(tagName)) { - tagManager.deleteTag(tagName, tagDeletion, snapshotManager); + tagManager.deleteTag( + tagName, tagDeletion, snapshotManager, table.store().createTagCallbacks()); } // Create a new tag tagManager.createTag(snapshot, tagName, table.store().createTagCallbacks()); @@ -122,7 +123,8 @@ private void createTag() { expireTag(); } catch (Exception e) { if (tagManager.tagExists(tagName)) { - tagManager.deleteTag(tagName, tagDeletion, snapshotManager); + tagManager.deleteTag( + tagName, tagDeletion, snapshotManager, table.store().createTagCallbacks()); } } } @@ -147,7 +149,11 @@ private void expireTag() { } else { List sortedTagNames = tagManager.sortTagsOfOneSnapshot(tagNames); for (String toBeDeleted : sortedTagNames) { - tagManager.deleteTag(toBeDeleted, tagDeletion, snapshotManager); + tagManager.deleteTag( + toBeDeleted, + tagDeletion, + snapshotManager, + table.store().createTagCallbacks()); tagCount--; if (tagCount == tagNumRetainedMax) { break; 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 f020f65bd427..d6d7f434bac5 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 @@ -63,6 +63,7 @@ public void testBatchWriteGeneratorTag() throws Exception { () -> new VersionedSerializerWrapper<>( new ManifestCommittableSerializer()))); + committerOperator.open(); TableCommitImpl tableCommit = table.newCommit(initialCommitUser); diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java index d24944b34f5d..031b1848a01e 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java @@ -91,6 +91,20 @@ public void addPartition(LinkedHashMap partitionSpec) throws Exc } } + @Override + public void deletePartition(LinkedHashMap partitionSpec) throws Exception { + List partitionValues = new ArrayList<>(partitionSpec.values()); + try { + client.dropPartition( + identifier.getDatabaseName(), + identifier.getObjectName(), + partitionValues, + false); + } catch (NoSuchObjectException e) { + // do nothing if the partition not exists + } + } + @Override public void close() throws Exception { client.close(); diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java index 80d6c546035f..aacd9087c746 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java @@ -933,6 +933,32 @@ public void testAddPartitionsForTag() throws Exception { "4\t40\t2023-10-17"); } + @Test + public void testDeletePartitionForTag() throws Exception { + tEnv.executeSql( + "CREATE TABLE t (\n" + + " k INT,\n" + + " v BIGINT,\n" + + " PRIMARY KEY (k) NOT ENFORCED\n" + + ") WITH (\n" + + " 'bucket' = '2',\n" + + " 'metastore.tag-to-partition' = 'dt'\n" + + ")"); + tEnv.executeSql("INSERT INTO t VALUES (1, 10), (2, 20)").await(); + tEnv.executeSql("CALL sys.create_tag('test_db.t', '2023-10-16', 1)"); + tEnv.executeSql("INSERT INTO t VALUES (3, 30)").await(); + tEnv.executeSql("CALL sys.create_tag('test_db.t', '2023-10-17', 2)"); + + assertThat(hiveShell.executeQuery("SHOW PARTITIONS t")) + .containsExactlyInAnyOrder("dt=2023-10-16", "dt=2023-10-17"); + + tEnv.executeSql("CALL sys.delete_tag('test_db.t', '2023-10-16')"); + assertThat(hiveShell.executeQuery("SHOW PARTITIONS t")) + .containsExactlyInAnyOrder("dt=2023-10-17"); + + assertThat(hiveShell.executeQuery("SELECT k, v FROM t WHERE dt='2023-10-16'")).isEmpty(); + } + @Test public void testHistoryPartitionsCascadeToUpdate() throws Exception { tEnv.executeSql( From 21e016a53c72e995d5b0a39a188e89032dd5d4e9 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Wed, 20 Mar 2024 15:12:10 +0800 Subject: [PATCH 34/79] [core] Optimize first_row batch read (#3055) --- .../java/org/apache/paimon/KeyValueFileStore.java | 3 ++- .../paimon/operation/KeyValueFileStoreRead.java | 2 +- .../paimon/operation/KeyValueFileStoreScan.java | 12 ++++++++++-- .../apache/paimon/table/AbstractFileStoreTable.java | 5 ++++- .../paimon/table/PrimaryKeyFileStoreTable.java | 8 +++++--- .../paimon/table/source/InnerTableScanImpl.java | 5 ++++- .../paimon/table/source/MergeTreeSplitGenerator.java | 11 +++++++++-- .../table/source/snapshot/SnapshotReaderImpl.java | 6 +++++- .../paimon/table/system/ReadOptimizedTable.java | 1 + .../paimon/table/source/SplitGeneratorTest.java | 5 +++-- 10 files changed, 44 insertions(+), 14 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java index d80bd6d3979d..956b615d7884 100644 --- a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java @@ -229,7 +229,8 @@ public void pushdown(Predicate keyFilter) { forWrite, options.scanManifestParallelism(), branchName, - options.deletionVectorsEnabled()); + options.deletionVectorsEnabled(), + options.mergeEngine()); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java index 0e115fdddc87..b0ab4338f117 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java @@ -213,7 +213,7 @@ public RecordReader createReader(DataSplit split) throws IOException { private RecordReader createReaderWithoutOuterProjection(DataSplit split) throws IOException { if (split.beforeFiles().isEmpty()) { - if (split.isStreaming() || split.deletionFiles().isPresent()) { + if (split.isStreaming() || split.convertToRawFiles().isPresent()) { return noMergeRead( split.partition(), split.bucket(), diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java index 0f34cac5a138..b4c4909aed3f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java @@ -18,6 +18,7 @@ package org.apache.paimon.operation; +import org.apache.paimon.CoreOptions.MergeEngine; import org.apache.paimon.KeyValueFileStore; import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestFile; @@ -36,6 +37,8 @@ import java.util.Collections; import java.util.List; +import static org.apache.paimon.CoreOptions.MergeEngine.FIRST_ROW; + /** {@link FileStoreScan} for {@link KeyValueFileStore}. */ public class KeyValueFileStoreScan extends AbstractFileStoreScan { @@ -45,6 +48,7 @@ public class KeyValueFileStoreScan extends AbstractFileStoreScan { private Predicate keyFilter; private Predicate valueFilter; private final boolean deletionVectorsEnabled; + private final MergeEngine mergeEngine; public KeyValueFileStoreScan( RowType partitionType, @@ -59,7 +63,8 @@ public KeyValueFileStoreScan( boolean checkNumOfBuckets, Integer scanManifestParallelism, String branchName, - boolean deletionVectorsEnabled) { + boolean deletionVectorsEnabled, + MergeEngine mergeEngine) { super( partitionType, bucketFilter, @@ -81,6 +86,7 @@ public KeyValueFileStoreScan( sid -> keyValueFieldsExtractor.valueFields(scanTableSchema(sid)), schema.id()); this.deletionVectorsEnabled = deletionVectorsEnabled; + this.mergeEngine = mergeEngine; } public KeyValueFileStoreScan withKeyFilter(Predicate predicate) { @@ -100,7 +106,9 @@ protected boolean filterByStats(ManifestEntry entry) { Predicate filter = null; FieldStatsArraySerializer serializer = null; BinaryTableStats stats = null; - if (deletionVectorsEnabled && entry.level() > 0 && valueFilter != null) { + if ((deletionVectorsEnabled || mergeEngine == FIRST_ROW) + && entry.level() > 0 + && valueFilter != null) { filter = valueFilter; serializer = fieldValueStatsConverters.getOrCreate(entry.file().schemaId()); stats = entry.file().valueStats(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java index ac0f798a4b4a..f7215dc574fa 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java @@ -157,7 +157,10 @@ public SnapshotReader newSnapshotReader(String branchName) { @Override public InnerTableScan newScan() { return new InnerTableScanImpl( - coreOptions(), newSnapshotReader(), DefaultValueAssigner.create(tableSchema)); + tableSchema.primaryKeys().size() > 0, + coreOptions(), + newSnapshotReader(), + DefaultValueAssigner.create(tableSchema)); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java index f35afc64d5c9..fea783259241 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java @@ -116,11 +116,13 @@ public KeyValueFileStore store() { @Override protected SplitGenerator splitGenerator() { + CoreOptions options = store().options(); return new MergeTreeSplitGenerator( store().newKeyComparator(), - store().options().splitTargetSize(), - store().options().splitOpenFileCost(), - store().options().deletionVectorsEnabled()); + options.splitTargetSize(), + options.splitOpenFileCost(), + options.deletionVectorsEnabled(), + options.mergeEngine()); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScanImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScanImpl.java index 375ef7e0ac6c..b307279d0f31 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScanImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScanImpl.java @@ -28,6 +28,8 @@ import java.util.ArrayList; import java.util.List; +import static org.apache.paimon.CoreOptions.MergeEngine.FIRST_ROW; + /** {@link TableScan} implementation for batch planning. */ public class InnerTableScanImpl extends AbstractInnerTableScan { @@ -39,13 +41,14 @@ public class InnerTableScanImpl extends AbstractInnerTableScan { private Integer pushDownLimit; public InnerTableScanImpl( + boolean pkTable, CoreOptions options, SnapshotReader snapshotReader, DefaultValueAssigner defaultValueAssigner) { super(options, snapshotReader); this.hasNext = true; this.defaultValueAssigner = defaultValueAssigner; - if (options.deletionVectorsEnabled()) { + if (pkTable && (options.deletionVectorsEnabled() || options.mergeEngine() == FIRST_ROW)) { snapshotReader.withLevelFilter(level -> level > 0); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/MergeTreeSplitGenerator.java b/paimon-core/src/main/java/org/apache/paimon/table/source/MergeTreeSplitGenerator.java index 7cf1ed24d9c8..9a06a53f4ce6 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/MergeTreeSplitGenerator.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/MergeTreeSplitGenerator.java @@ -18,6 +18,7 @@ package org.apache.paimon.table.source; +import org.apache.paimon.CoreOptions.MergeEngine; import org.apache.paimon.data.InternalRow; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.mergetree.SortedRun; @@ -31,6 +32,8 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static org.apache.paimon.CoreOptions.MergeEngine.FIRST_ROW; + /** Merge tree implementation of {@link SplitGenerator}. */ public class MergeTreeSplitGenerator implements SplitGenerator { @@ -42,20 +45,24 @@ public class MergeTreeSplitGenerator implements SplitGenerator { private final boolean deletionVectorsEnabled; + private final MergeEngine mergeEngine; + public MergeTreeSplitGenerator( Comparator keyComparator, long targetSplitSize, long openFileCost, - boolean deletionVectorsEnabled) { + boolean deletionVectorsEnabled, + MergeEngine mergeEngine) { this.keyComparator = keyComparator; this.targetSplitSize = targetSplitSize; this.openFileCost = openFileCost; this.deletionVectorsEnabled = deletionVectorsEnabled; + this.mergeEngine = mergeEngine; } @Override public List> splitForBatch(List files) { - if (deletionVectorsEnabled) { + if (deletionVectorsEnabled || mergeEngine == FIRST_ROW) { Function weightFunc = file -> Math.max(file.fileSize(), openFileCost); return BinPacking.packForOrdered(files, weightFunc, targetSplitSize); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java index 06836d481aa5..aa28fa667467 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java @@ -19,6 +19,7 @@ package org.apache.paimon.table.source.snapshot; import org.apache.paimon.CoreOptions; +import org.apache.paimon.CoreOptions.MergeEngine; import org.apache.paimon.Snapshot; import org.apache.paimon.codegen.CodeGenUtils; import org.apache.paimon.codegen.RecordComparator; @@ -63,6 +64,7 @@ import java.util.function.BiConsumer; import java.util.stream.Collectors; +import static org.apache.paimon.CoreOptions.MergeEngine.FIRST_ROW; import static org.apache.paimon.deletionvectors.DeletionVectorsIndexFile.DELETION_VECTORS_INDEX; import static org.apache.paimon.operation.FileStoreScan.Plan.groupByPartFiles; import static org.apache.paimon.predicate.PredicateBuilder.transformFieldMapping; @@ -73,6 +75,7 @@ public class SnapshotReaderImpl implements SnapshotReader { private final FileStoreScan scan; private final TableSchema tableSchema; private final CoreOptions options; + private final MergeEngine mergeEngine; private final boolean deletionVectors; private final SnapshotManager snapshotManager; private final ConsumerManager consumerManager; @@ -100,6 +103,7 @@ public SnapshotReaderImpl( this.scan = scan; this.tableSchema = tableSchema; this.options = options; + this.mergeEngine = options.mergeEngine(); this.deletionVectors = options.deletionVectorsEnabled(); this.snapshotManager = snapshotManager; this.consumerManager = @@ -435,7 +439,7 @@ private List convertToRawFiles( String bucketPath = pathFactory.bucketPath(partition, bucket).toString(); // append only or deletionVectors files can be returned - if (tableSchema.primaryKeys().isEmpty() || deletionVectors) { + if (tableSchema.primaryKeys().isEmpty() || deletionVectors || mergeEngine == FIRST_ROW) { return makeRawTableFiles(bucketPath, dataFiles); } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java index 35ac209a91d0..3c6910fbec16 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java @@ -105,6 +105,7 @@ public SnapshotReader newSnapshotReader(String branchName) { @Override public InnerTableScan newScan() { return new InnerTableScanImpl( + dataTable.schema().primaryKeys().size() > 0, coreOptions(), newSnapshotReader(), DefaultValueAssigner.create(dataTable.schema())); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java index 6d97eda5f69a..1278339210de 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.stream.Collectors; +import static org.apache.paimon.CoreOptions.MergeEngine.DEDUPLICATE; import static org.apache.paimon.data.BinaryRow.EMPTY_ROW; import static org.apache.paimon.io.DataFileTestUtils.fromMinMax; import static org.assertj.core.api.Assertions.assertThat; @@ -108,14 +109,14 @@ public void testMergeTree() { Comparator comparator = Comparator.comparingInt(o -> o.getInt(0)); assertThat( toNames( - new MergeTreeSplitGenerator(comparator, 100, 2, false) + new MergeTreeSplitGenerator(comparator, 100, 2, false, DEDUPLICATE) .splitForBatch(files))) .containsExactlyInAnyOrder( Arrays.asList("1", "2", "4", "3", "5"), Collections.singletonList("6")); assertThat( toNames( - new MergeTreeSplitGenerator(comparator, 100, 30, false) + new MergeTreeSplitGenerator(comparator, 100, 30, false, DEDUPLICATE) .splitForBatch(files))) .containsExactlyInAnyOrder( Arrays.asList("1", "2", "4", "3"), From 47bc30fd13f4e05798f35ff87a0dc83cb3ae67d6 Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Wed, 20 Mar 2024 17:36:07 +0800 Subject: [PATCH 35/79] [flink] Bump flink version to 1.19 (#3049) --- .github/workflows/e2e-tests-1.18-jdk11.yml | 4 +- .github/workflows/e2e-tests-1.18.yml | 2 +- .github/workflows/e2e-tests-1.19-jdk11.yml | 58 +++++++++++++ .github/workflows/e2e-tests-1.19.yml | 57 +++++++++++++ .github/workflows/unitcase-flink-jdk11.yml | 7 +- .github/workflows/utitcase-flink.yml | 2 +- paimon-e2e-tests/pom.xml | 12 ++- .../flink/utils/ManagedMemoryUtils.java | 49 +++++++++++ .../flink/ContinuousFileStoreITCase.java | 20 +++++ paimon-flink/paimon-flink-1.16/pom.xml | 9 +- .../flink/utils/ManagedMemoryUtils.java | 49 +++++++++++ .../flink/ContinuousFileStoreITCase.java | 59 +++++++++++++ paimon-flink/paimon-flink-1.17/pom.xml | 38 +++++++++ .../flink/utils/ManagedMemoryUtils.java | 49 +++++++++++ .../flink/ContinuousFileStoreITCase.java | 59 +++++++++++++ paimon-flink/paimon-flink-1.18/pom.xml | 44 ++++++++++ .../flink/utils/ManagedMemoryUtils.java | 49 +++++++++++ .../flink/ContinuousFileStoreITCase.java | 59 +++++++++++++ paimon-flink/paimon-flink-1.19/pom.xml | 85 +++++++++++++++++++ .../sink/cdc/FlinkCdcMultiTableSink.java | 3 +- paimon-flink/paimon-flink-common/pom.xml | 2 +- .../apache/paimon/flink/sink/FlinkSink.java | 14 +-- .../flink/sink/RowDataStoreWriteOperator.java | 8 +- .../flink/utils/ManagedMemoryUtils.java | 1 + .../paimon/flink/RescaleBucketITCase.java | 14 +-- paimon-flink/pom.xml | 6 ++ paimon-hive/paimon-hive-catalog/pom.xml | 25 ------ paimon-hive/paimon-hive-connector-2.3/pom.xml | 29 +------ paimon-hive/paimon-hive-connector-3.1/pom.xml | 29 +------ .../paimon-hive-connector-common/pom.xml | 43 +++------- .../paimon/hive/HiveCatalogITCaseBase.java | 4 + paimon-hive/pom.xml | 42 ++++++++- pom.xml | 5 +- 33 files changed, 799 insertions(+), 137 deletions(-) create mode 100644 .github/workflows/e2e-tests-1.19-jdk11.yml create mode 100644 .github/workflows/e2e-tests-1.19.yml create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java create mode 100644 paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java create mode 100644 paimon-flink/paimon-flink-1.17/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java create mode 100644 paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java create mode 100644 paimon-flink/paimon-flink-1.19/pom.xml diff --git a/.github/workflows/e2e-tests-1.18-jdk11.yml b/.github/workflows/e2e-tests-1.18-jdk11.yml index a8b42a6df641..b924a3b07628 100644 --- a/.github/workflows/e2e-tests-1.18-jdk11.yml +++ b/.github/workflows/e2e-tests-1.18-jdk11.yml @@ -46,13 +46,13 @@ jobs: distribution: 'adopt' - name: Build Flink 1.18 run: mvn -T 1C -B clean install -DskipTests - - name: Test Flink 1.17 + - name: Test Flink 1.18 timeout-minutes: 60 run: | # run tests with random timezone to find out timezone related bugs . .github/workflows/utils.sh jvm_timezone=$(random_timezone) echo "JVM timezone is set to $jvm_timezone" - mvn -T 1C -B test -pl paimon-e2e-tests -Duser.timezone=$jvm_timezone + mvn -T 1C -B test -pl paimon-e2e-tests -Duser.timezone=$jvm_timezone -Pflink-1.18 env: MAVEN_OPTS: -Xmx4096m \ No newline at end of file diff --git a/.github/workflows/e2e-tests-1.18.yml b/.github/workflows/e2e-tests-1.18.yml index 2985b45c46e9..2f566004a241 100644 --- a/.github/workflows/e2e-tests-1.18.yml +++ b/.github/workflows/e2e-tests-1.18.yml @@ -52,6 +52,6 @@ jobs: . .github/workflows/utils.sh jvm_timezone=$(random_timezone) echo "JVM timezone is set to $jvm_timezone" - mvn -T 1C -B test -pl paimon-e2e-tests -Duser.timezone=$jvm_timezone + mvn -T 1C -B test -pl paimon-e2e-tests -Duser.timezone=$jvm_timezone -Pflink-1.18 env: MAVEN_OPTS: -Xmx4096m \ No newline at end of file diff --git a/.github/workflows/e2e-tests-1.19-jdk11.yml b/.github/workflows/e2e-tests-1.19-jdk11.yml new file mode 100644 index 000000000000..bc917f453f0f --- /dev/null +++ b/.github/workflows/e2e-tests-1.19-jdk11.yml @@ -0,0 +1,58 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +name: End to End Tests Flink 1.19 on JDK 11 + +on: + issue_comment: + types: [created, edited, deleted] + + # daily run + schedule: + - cron: "0 0 * * *" + +env: + JDK_VERSION: 11 + +jobs: + build: + if: | + github.event_name == 'schedule' || + (contains(github.event.comment.html_url, '/pull/') && contains(github.event.comment.body, '/jdk11')) + runs-on: ubuntu-latest + + steps: + - name: Checkout code + uses: actions/checkout@v2 + - name: Set up JDK ${{ env.JDK_VERSION }} + uses: actions/setup-java@v2 + with: + java-version: ${{ env.JDK_VERSION }} + distribution: 'adopt' + - name: Build Flink 1.19 + run: mvn -T 1C -B clean install -DskipTests + - name: Test Flink 1.19 + timeout-minutes: 60 + run: | + # run tests with random timezone to find out timezone related bugs + . .github/workflows/utils.sh + jvm_timezone=$(random_timezone) + echo "JVM timezone is set to $jvm_timezone" + mvn -T 1C -B test -pl paimon-e2e-tests -Duser.timezone=$jvm_timezone + env: + MAVEN_OPTS: -Xmx4096m \ No newline at end of file diff --git a/.github/workflows/e2e-tests-1.19.yml b/.github/workflows/e2e-tests-1.19.yml new file mode 100644 index 000000000000..b451d6385a9f --- /dev/null +++ b/.github/workflows/e2e-tests-1.19.yml @@ -0,0 +1,57 @@ +################################################################################ +# 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. +################################################################################ + +name: End to End Tests Flink 1.19 + +on: + push: + pull_request: + paths-ignore: + - 'docs/**' + - '**/*.md' + +env: + JDK_VERSION: 8 + +concurrency: + group: ${{ github.workflow }}-${{ github.event_name }}-${{ github.event.number || github.run_id }} + cancel-in-progress: true + +jobs: + build: + runs-on: ubuntu-latest + steps: + - name: Checkout code + uses: actions/checkout@v2 + - name: Set up JDK ${{ env.JDK_VERSION }} + uses: actions/setup-java@v2 + with: + java-version: ${{ env.JDK_VERSION }} + distribution: 'adopt' + - name: Build Flink 1.19 + run: mvn -T 1C -B clean install -DskipTests + - name: Test Flink 1.19 + timeout-minutes: 60 + run: | + # run tests with random timezone to find out timezone related bugs + . .github/workflows/utils.sh + jvm_timezone=$(random_timezone) + echo "JVM timezone is set to $jvm_timezone" + mvn -T 1C -B test -pl paimon-e2e-tests -Duser.timezone=$jvm_timezone + env: + MAVEN_OPTS: -Xmx4096m \ No newline at end of file diff --git a/.github/workflows/unitcase-flink-jdk11.yml b/.github/workflows/unitcase-flink-jdk11.yml index 59dd1457ce0f..135dc5718f23 100644 --- a/.github/workflows/unitcase-flink-jdk11.yml +++ b/.github/workflows/unitcase-flink-jdk11.yml @@ -52,6 +52,11 @@ jobs: . .github/workflows/utils.sh jvm_timezone=$(random_timezone) echo "JVM timezone is set to $jvm_timezone" - mvn -T 1C -B clean install -pl 'org.apache.paimon:paimon-flink-common' -Duser.timezone=$jvm_timezone + test_modules="" + for suffix in 1.15 1.16 1.17 1.18 1.19 common; do + test_modules+="org.apache.paimon:paimon-flink-${suffix}," + done + test_modules="${test_modules%,}" + mvn -T 1C -B clean install -pl "${test_modules}" -Duser.timezone=$jvm_timezone env: MAVEN_OPTS: -Xmx4096m \ No newline at end of file diff --git a/.github/workflows/utitcase-flink.yml b/.github/workflows/utitcase-flink.yml index b6cc9cbd128b..c7455e8143b4 100644 --- a/.github/workflows/utitcase-flink.yml +++ b/.github/workflows/utitcase-flink.yml @@ -52,7 +52,7 @@ jobs: jvm_timezone=$(random_timezone) echo "JVM timezone is set to $jvm_timezone" test_modules="" - for suffix in 1.15 1.16 1.17 1.18 common; do + for suffix in 1.15 1.16 1.17 1.18 1.19 common; do test_modules+="org.apache.paimon:paimon-flink-${suffix}," done test_modules="${test_modules%,}" diff --git a/paimon-e2e-tests/pom.xml b/paimon-e2e-tests/pom.xml index 604eb04c7b8b..fdcdbda7ab28 100644 --- a/paimon-e2e-tests/pom.xml +++ b/paimon-e2e-tests/pom.xml @@ -34,7 +34,6 @@ under the License. 2.8.3-10.0 2.3.0 - flink-sql-connector-kafka flink-sql-connector-hive-2.3.9_${scala.binary.version} @@ -210,7 +209,7 @@ under the License. org.apache.flink - ${flink.sql.connector.kafka} + flink-sql-connector-kafka ${test.flink.connector.kafka.version} flink-sql-connector-kafka.jar jar @@ -276,6 +275,14 @@ under the License. + + flink-1.18 + + 1.18 + 1.18.1 + + + flink-1.17 @@ -300,7 +307,6 @@ under the License. 1.15 1.15.3 ${test.flink.version} - flink-sql-connector-kafka flink-sql-connector-hive-2.3.6_${scala.binary.version} diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java new file mode 100644 index 000000000000..a51d9e02e743 --- /dev/null +++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java @@ -0,0 +1,49 @@ +/* + * 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.utils; + +import org.apache.paimon.options.MemorySize; + +import org.apache.flink.core.memory.ManagedMemoryUseCase; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; + +/** Utils for using Flink managed memory. */ +public class ManagedMemoryUtils { + + public static void declareManagedMemory(DataStream dataStream, MemorySize memorySize) { + dataStream + .getTransformation() + .declareManagedMemoryUseCaseAtOperatorScope( + ManagedMemoryUseCase.OPERATOR, memorySize.getMebiBytes()); + } + + public static long computeManagedMemory(AbstractStreamOperator operator) { + final Environment environment = operator.getContainingTask().getEnvironment(); + return environment + .getMemoryManager() + .computeMemorySize( + operator.getOperatorConfig() + .getManagedMemoryFractionOperatorUseCaseOfSlot( + ManagedMemoryUseCase.OPERATOR, + environment.getTaskManagerInfo().getConfiguration(), + environment.getUserCodeClassLoader().asClassLoader())); + } +} diff --git a/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java b/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java index e1c62e28f599..5e9b7c7ceb53 100644 --- a/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java +++ b/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java @@ -329,4 +329,24 @@ public void testUnsupportedEventual() { "SELECT * FROM T1 /*+ OPTIONS('log.consistency'='eventual') */"), "File store continuous reading does not support eventual consistency mode"); } + + @Test + public void testFlinkMemoryPool() { + // Check if the configuration is effective + assertThatThrownBy( + () -> + batchSql( + "INSERT INTO %s /*+ OPTIONS('sink.use-managed-memory-allocator'='true', 'sink.managed.writer-buffer-memory'='0M') */ " + + "VALUES ('1', '2', '3'), ('4', '5', '6')", + "T1")) + .hasCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage( + "Weights for operator scope use cases must be greater than 0."); + + batchSql( + "INSERT INTO %s /*+ OPTIONS('sink.use-managed-memory-allocator'='true', 'sink.managed.writer-buffer-memory'='1M') */ " + + "VALUES ('1', '2', '3'), ('4', '5', '6')", + "T1"); + assertThat(batchSql("SELECT * FROM T1").size()).isEqualTo(2); + } } diff --git a/paimon-flink/paimon-flink-1.16/pom.xml b/paimon-flink/paimon-flink-1.16/pom.xml index 9aaa19228ed5..3558309e6a55 100644 --- a/paimon-flink/paimon-flink-1.16/pom.xml +++ b/paimon-flink/paimon-flink-1.16/pom.xml @@ -64,7 +64,7 @@ under the License. org.apache.flink - flink-table-common + flink-table-api-java-bridge ${flink.version} provided @@ -93,6 +93,13 @@ under the License. test + + org.apache.flink + flink-connector-files + ${flink.version} + test + + diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java new file mode 100644 index 000000000000..a51d9e02e743 --- /dev/null +++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java @@ -0,0 +1,49 @@ +/* + * 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.utils; + +import org.apache.paimon.options.MemorySize; + +import org.apache.flink.core.memory.ManagedMemoryUseCase; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; + +/** Utils for using Flink managed memory. */ +public class ManagedMemoryUtils { + + public static void declareManagedMemory(DataStream dataStream, MemorySize memorySize) { + dataStream + .getTransformation() + .declareManagedMemoryUseCaseAtOperatorScope( + ManagedMemoryUseCase.OPERATOR, memorySize.getMebiBytes()); + } + + public static long computeManagedMemory(AbstractStreamOperator operator) { + final Environment environment = operator.getContainingTask().getEnvironment(); + return environment + .getMemoryManager() + .computeMemorySize( + operator.getOperatorConfig() + .getManagedMemoryFractionOperatorUseCaseOfSlot( + ManagedMemoryUseCase.OPERATOR, + environment.getTaskManagerInfo().getConfiguration(), + environment.getUserCodeClassLoader().asClassLoader())); + } +} diff --git a/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java b/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java new file mode 100644 index 000000000000..84c84d1c68f9 --- /dev/null +++ b/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java @@ -0,0 +1,59 @@ +/* + * 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.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** SQL ITCase for continuous file store. */ +public class ContinuousFileStoreITCase extends CatalogITCaseBase { + + @Override + protected List ddl() { + return Arrays.asList( + "CREATE TABLE IF NOT EXISTS T1 (a STRING, b STRING, c STRING) WITH ('bucket' = '1')", + "CREATE TABLE IF NOT EXISTS T2 (a STRING, b STRING, c STRING, PRIMARY KEY (a) NOT ENFORCED)" + + " WITH ('changelog-producer'='input', 'bucket' = '1')"); + } + + @Test + public void testFlinkMemoryPool() { + // Check if the configuration is effective + assertThatThrownBy( + () -> + batchSql( + "INSERT INTO %s /*+ OPTIONS('sink.use-managed-memory-allocator'='true', 'sink.managed.writer-buffer-memory'='0M') */ " + + "VALUES ('1', '2', '3'), ('4', '5', '6')", + "T1")) + .hasCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage( + "Weights for operator scope use cases must be greater than 0."); + + batchSql( + "INSERT INTO %s /*+ OPTIONS('sink.use-managed-memory-allocator'='true', 'sink.managed.writer-buffer-memory'='1M') */ " + + "VALUES ('1', '2', '3'), ('4', '5', '6')", + "T1"); + assertThat(batchSql("SELECT * FROM T1").size()).isEqualTo(2); + } +} diff --git a/paimon-flink/paimon-flink-1.17/pom.xml b/paimon-flink/paimon-flink-1.17/pom.xml index 454386d59217..11318dbab686 100644 --- a/paimon-flink/paimon-flink-1.17/pom.xml +++ b/paimon-flink/paimon-flink-1.17/pom.xml @@ -68,6 +68,44 @@ under the License. + + + org.apache.flink + flink-table-api-java-bridge + ${flink.version} + provided + + + + + + org.apache.paimon + paimon-flink-common + ${project.version} + test-jar + test + + + + org.apache.flink + flink-test-utils + ${flink.version} + test + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${flink.version} + test + + + + org.apache.flink + flink-connector-files + ${flink.version} + test + diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java new file mode 100644 index 000000000000..a51d9e02e743 --- /dev/null +++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java @@ -0,0 +1,49 @@ +/* + * 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.utils; + +import org.apache.paimon.options.MemorySize; + +import org.apache.flink.core.memory.ManagedMemoryUseCase; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; + +/** Utils for using Flink managed memory. */ +public class ManagedMemoryUtils { + + public static void declareManagedMemory(DataStream dataStream, MemorySize memorySize) { + dataStream + .getTransformation() + .declareManagedMemoryUseCaseAtOperatorScope( + ManagedMemoryUseCase.OPERATOR, memorySize.getMebiBytes()); + } + + public static long computeManagedMemory(AbstractStreamOperator operator) { + final Environment environment = operator.getContainingTask().getEnvironment(); + return environment + .getMemoryManager() + .computeMemorySize( + operator.getOperatorConfig() + .getManagedMemoryFractionOperatorUseCaseOfSlot( + ManagedMemoryUseCase.OPERATOR, + environment.getTaskManagerInfo().getConfiguration(), + environment.getUserCodeClassLoader().asClassLoader())); + } +} diff --git a/paimon-flink/paimon-flink-1.17/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java b/paimon-flink/paimon-flink-1.17/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java new file mode 100644 index 000000000000..84c84d1c68f9 --- /dev/null +++ b/paimon-flink/paimon-flink-1.17/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java @@ -0,0 +1,59 @@ +/* + * 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.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** SQL ITCase for continuous file store. */ +public class ContinuousFileStoreITCase extends CatalogITCaseBase { + + @Override + protected List ddl() { + return Arrays.asList( + "CREATE TABLE IF NOT EXISTS T1 (a STRING, b STRING, c STRING) WITH ('bucket' = '1')", + "CREATE TABLE IF NOT EXISTS T2 (a STRING, b STRING, c STRING, PRIMARY KEY (a) NOT ENFORCED)" + + " WITH ('changelog-producer'='input', 'bucket' = '1')"); + } + + @Test + public void testFlinkMemoryPool() { + // Check if the configuration is effective + assertThatThrownBy( + () -> + batchSql( + "INSERT INTO %s /*+ OPTIONS('sink.use-managed-memory-allocator'='true', 'sink.managed.writer-buffer-memory'='0M') */ " + + "VALUES ('1', '2', '3'), ('4', '5', '6')", + "T1")) + .hasCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage( + "Weights for operator scope use cases must be greater than 0."); + + batchSql( + "INSERT INTO %s /*+ OPTIONS('sink.use-managed-memory-allocator'='true', 'sink.managed.writer-buffer-memory'='1M') */ " + + "VALUES ('1', '2', '3'), ('4', '5', '6')", + "T1"); + assertThat(batchSql("SELECT * FROM T1").size()).isEqualTo(2); + } +} diff --git a/paimon-flink/paimon-flink-1.18/pom.xml b/paimon-flink/paimon-flink-1.18/pom.xml index a0f83fa0d274..31e85df3b9f0 100644 --- a/paimon-flink/paimon-flink-1.18/pom.xml +++ b/paimon-flink/paimon-flink-1.18/pom.xml @@ -42,6 +42,12 @@ under the License. org.apache.paimon paimon-flink-common ${project.version} + + + * + * + + @@ -55,6 +61,44 @@ under the License. + + + org.apache.flink + flink-table-api-java-bridge + ${flink.version} + provided + + + + + + org.apache.paimon + paimon-flink-common + ${project.version} + test-jar + test + + + + org.apache.flink + flink-test-utils + ${flink.version} + test + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${flink.version} + test + + + + org.apache.flink + flink-connector-files + ${flink.version} + test + diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java new file mode 100644 index 000000000000..a51d9e02e743 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java @@ -0,0 +1,49 @@ +/* + * 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.utils; + +import org.apache.paimon.options.MemorySize; + +import org.apache.flink.core.memory.ManagedMemoryUseCase; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; + +/** Utils for using Flink managed memory. */ +public class ManagedMemoryUtils { + + public static void declareManagedMemory(DataStream dataStream, MemorySize memorySize) { + dataStream + .getTransformation() + .declareManagedMemoryUseCaseAtOperatorScope( + ManagedMemoryUseCase.OPERATOR, memorySize.getMebiBytes()); + } + + public static long computeManagedMemory(AbstractStreamOperator operator) { + final Environment environment = operator.getContainingTask().getEnvironment(); + return environment + .getMemoryManager() + .computeMemorySize( + operator.getOperatorConfig() + .getManagedMemoryFractionOperatorUseCaseOfSlot( + ManagedMemoryUseCase.OPERATOR, + environment.getTaskManagerInfo().getConfiguration(), + environment.getUserCodeClassLoader().asClassLoader())); + } +} diff --git a/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java b/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java new file mode 100644 index 000000000000..84c84d1c68f9 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java @@ -0,0 +1,59 @@ +/* + * 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.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** SQL ITCase for continuous file store. */ +public class ContinuousFileStoreITCase extends CatalogITCaseBase { + + @Override + protected List ddl() { + return Arrays.asList( + "CREATE TABLE IF NOT EXISTS T1 (a STRING, b STRING, c STRING) WITH ('bucket' = '1')", + "CREATE TABLE IF NOT EXISTS T2 (a STRING, b STRING, c STRING, PRIMARY KEY (a) NOT ENFORCED)" + + " WITH ('changelog-producer'='input', 'bucket' = '1')"); + } + + @Test + public void testFlinkMemoryPool() { + // Check if the configuration is effective + assertThatThrownBy( + () -> + batchSql( + "INSERT INTO %s /*+ OPTIONS('sink.use-managed-memory-allocator'='true', 'sink.managed.writer-buffer-memory'='0M') */ " + + "VALUES ('1', '2', '3'), ('4', '5', '6')", + "T1")) + .hasCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage( + "Weights for operator scope use cases must be greater than 0."); + + batchSql( + "INSERT INTO %s /*+ OPTIONS('sink.use-managed-memory-allocator'='true', 'sink.managed.writer-buffer-memory'='1M') */ " + + "VALUES ('1', '2', '3'), ('4', '5', '6')", + "T1"); + assertThat(batchSql("SELECT * FROM T1").size()).isEqualTo(2); + } +} diff --git a/paimon-flink/paimon-flink-1.19/pom.xml b/paimon-flink/paimon-flink-1.19/pom.xml new file mode 100644 index 000000000000..60a33df24bf2 --- /dev/null +++ b/paimon-flink/paimon-flink-1.19/pom.xml @@ -0,0 +1,85 @@ + + + + 4.0.0 + + + org.apache.paimon + paimon-flink + 0.8-SNAPSHOT + + + jar + + paimon-flink-1.19 + Paimon : Flink : 1.19 + + + 1.19.0 + + + + + org.apache.paimon + paimon-flink-common + ${project.version} + + + + org.apache.paimon + paimon-flink-cdc + ${project.version} + + + * + * + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-paimon + package + + shade + + + + + org.apache.paimon:paimon-flink-common + org.apache.paimon:paimon-flink-cdc + + + + + + + + + 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 c0156e841cfa..9cba78f213fd 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 @@ -131,8 +131,7 @@ public DataStreamSink sinkFrom( createCommitterFactory(), createCommittableStateManager())) .setParallelism(input.getParallelism()); - configureGlobalCommitter( - committed, commitCpuCores, commitHeapMemory, env.getConfiguration()); + configureGlobalCommitter(committed, commitCpuCores, commitHeapMemory); return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); } diff --git a/paimon-flink/paimon-flink-common/pom.xml b/paimon-flink/paimon-flink-common/pom.xml index efca0b1bde64..eea8b9b581a7 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.18.1 + 1.19.0 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 582fcfc35af2..97c426ee5685 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 @@ -52,7 +52,6 @@ import java.util.Set; import java.util.UUID; -import static org.apache.flink.configuration.ClusterOptions.ENABLE_FINE_GRAINED_RESOURCE_MANAGEMENT; 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.SINK_AUTO_TAG_FOR_SAVEPOINT; @@ -261,27 +260,18 @@ protected DataStreamSink doCommit(DataStream written, String com .setMaxParallelism(1); Options options = Options.fromMap(table.options()); configureGlobalCommitter( - committed, - options.get(SINK_COMMITTER_CPU), - options.get(SINK_COMMITTER_MEMORY), - conf); + committed, options.get(SINK_COMMITTER_CPU), options.get(SINK_COMMITTER_MEMORY)); return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); } public static void configureGlobalCommitter( SingleOutputStreamOperator committed, double cpuCores, - @Nullable MemorySize heapMemory, - ReadableConfig conf) { + @Nullable MemorySize heapMemory) { if (heapMemory == null) { return; } - if (!conf.get(ENABLE_FINE_GRAINED_RESOURCE_MANAGEMENT)) { - throw new RuntimeException( - "To support the 'sink.committer-cpu' and 'sink.committer-memory' configurations, you must enable fine-grained resource management. Please set 'cluster.fine-grained-resource-management.enabled' to 'true' in your Flink configuration."); - } - SlotSharingGroup slotSharingGroup = SlotSharingGroup.newBuilder(committed.getName()) .setCpuCores(cpuCores) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java index a5c8e5557b94..b61fecab5d34 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java @@ -23,6 +23,7 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.SinkRecord; +import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.configuration.Configuration; @@ -96,7 +97,12 @@ public void open() throws Exception { this.sinkContext = new SimpleContext(getProcessingTimeService()); if (logSinkFunction != null) { - FunctionUtils.openFunction(logSinkFunction, new Configuration()); + // to stay compatible with Flink 1.18- + if (logSinkFunction instanceof RichFunction) { + RichFunction richFunction = (RichFunction) logSinkFunction; + richFunction.open(new Configuration()); + } + logCallback = new LogWriteCallback(); logSinkFunction.setWriteCallback(logCallback); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java index a51d9e02e743..82964e41cc05 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/ManagedMemoryUtils.java @@ -43,6 +43,7 @@ public static long computeManagedMemory(AbstractStreamOperator operator) { operator.getOperatorConfig() .getManagedMemoryFractionOperatorUseCaseOfSlot( ManagedMemoryUseCase.OPERATOR, + environment.getJobConfiguration(), environment.getTaskManagerInfo().getConfiguration(), environment.getUserCodeClassLoader().asClassLoader())); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java index 841a24ba65e1..a559d3350099 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java @@ -34,6 +34,7 @@ import java.util.Arrays; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import static org.apache.paimon.CoreOptions.BUCKET; @@ -80,13 +81,11 @@ public void testSuspendAndRecoverAfterRescaleOverwrite() throws Exception { + "INSERT INTO `T4` SELECT * FROM `S0`;\n" + "END"; - sEnv.getConfig().getConfiguration().set(SavepointConfigOptions.SAVEPOINT_PATH, path); - // step1: run streaming insert JobClient jobClient = startJobAndCommitSnapshot(streamSql, null); // step2: stop with savepoint - stopJobSafely(jobClient); + String savepointPath = stopJobSafely(jobClient); final Snapshot snapshotBeforeRescale = findLatestSnapshot("T3"); assertThat(snapshotBeforeRescale).isNotNull(); @@ -107,6 +106,9 @@ public void testSuspendAndRecoverAfterRescaleOverwrite() throws Exception { assertThat(batchSql("SELECT * FROM T3")).containsExactlyInAnyOrderElementsOf(committedData); // step5: resume streaming job + sEnv.getConfig() + .getConfiguration() + .set(SavepointConfigOptions.SAVEPOINT_PATH, savepointPath); JobClient resumedJobClient = startJobAndCommitSnapshot(streamSql, snapshotAfterRescale.id()); // stop job @@ -144,11 +146,13 @@ private JobClient startJobAndCommitSnapshot(String sql, @Nullable Long initSnaps return jobClient; } - private void stopJobSafely(JobClient client) throws ExecutionException, InterruptedException { - client.stopWithSavepoint(true, path, SavepointFormatType.DEFAULT); + private String stopJobSafely(JobClient client) throws ExecutionException, InterruptedException { + CompletableFuture savepointPath = + client.stopWithSavepoint(true, path, SavepointFormatType.DEFAULT); while (!client.getJobStatus().get().isGloballyTerminalState()) { Thread.sleep(2000L); } + return savepointPath.get(); } private void assertLatestSchema( diff --git a/paimon-flink/pom.xml b/paimon-flink/pom.xml index 30ef22d8c573..caeb2fcea5c6 100644 --- a/paimon-flink/pom.xml +++ b/paimon-flink/pom.xml @@ -39,6 +39,7 @@ under the License. paimon-flink-1.16 paimon-flink-1.17 paimon-flink-1.18 + paimon-flink-1.19 paimon-flink-action paimon-flink-cdc @@ -95,6 +96,10 @@ under the License. com.google.protobuf protobuf-java + + commons-io + commons-io + @@ -162,6 +167,7 @@ under the License. paimon-flink-1.16 paimon-flink-1.17 paimon-flink-1.18 + paimon-flink-1.19 paimon-flink-cdc diff --git a/paimon-hive/paimon-hive-catalog/pom.xml b/paimon-hive/paimon-hive-catalog/pom.xml index b50e58563750..f4c9a810bd63 100644 --- a/paimon-hive/paimon-hive-catalog/pom.xml +++ b/paimon-hive/paimon-hive-catalog/pom.xml @@ -87,31 +87,6 @@ under the License. - - org.apache.hadoop - hadoop-common - ${hadoop.version} - provided - - - org.apache.avro - avro - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - jdk.tools - jdk.tools - - - - org.apache.hadoop hadoop-hdfs diff --git a/paimon-hive/paimon-hive-connector-2.3/pom.xml b/paimon-hive/paimon-hive-connector-2.3/pom.xml index 0f4733f001e3..8f2005e5678b 100644 --- a/paimon-hive/paimon-hive-connector-2.3/pom.xml +++ b/paimon-hive/paimon-hive-connector-2.3/pom.xml @@ -127,31 +127,6 @@ under the License. - - org.apache.hadoop - hadoop-common - ${hadoop.version} - test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.pentaho - * - - - com.google.protobuf - protobuf-java - - - - org.apache.hadoop hadoop-client @@ -554,6 +529,10 @@ under the License. org.apache.calcite calcite-avatica + + commons-io + commons-io + diff --git a/paimon-hive/paimon-hive-connector-3.1/pom.xml b/paimon-hive/paimon-hive-connector-3.1/pom.xml index 1792f032f01b..d52bc0b0b028 100644 --- a/paimon-hive/paimon-hive-connector-3.1/pom.xml +++ b/paimon-hive/paimon-hive-connector-3.1/pom.xml @@ -141,31 +141,6 @@ under the License. test - - org.apache.hadoop - hadoop-common - ${hadoop.version} - test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.pentaho - * - - - com.google.protobuf - protobuf-java - - - - org.apache.hadoop hadoop-client @@ -580,6 +555,10 @@ under the License. org.apache.calcite calcite-avatica + + commons-io + commons-io + diff --git a/paimon-hive/paimon-hive-connector-common/pom.xml b/paimon-hive/paimon-hive-connector-common/pom.xml index a9939854ce0e..7d5f1e551492 100644 --- a/paimon-hive/paimon-hive-connector-common/pom.xml +++ b/paimon-hive/paimon-hive-connector-common/pom.xml @@ -46,35 +46,6 @@ under the License. ${project.version} - - org.apache.hadoop - hadoop-common - ${hadoop.version} - provided - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.pentaho - * - - - jdk.tools - jdk.tools - - - com.google.protobuf - protobuf-java - - - - org.apache.hadoop hadoop-client @@ -519,6 +490,12 @@ under the License. avro ${avro.version} test + + + org.apache.commons + commons-compress + + **/*Test.* - 1.18 - 1.18.1 + 1.19 + 1.19.0 + 3.0.1-1.18 3.0.11 From 532cd8f6e85618947375901f7955dce266fc04e2 Mon Sep 17 00:00:00 2001 From: Jason Zhang Date: Wed, 20 Mar 2024 17:49:15 +0800 Subject: [PATCH 36/79] [orc] ORC Support ZStandard compression using zstd-jni (#3056) --- paimon-format/pom.xml | 33 +- .../java/org/apache/orc/CompressionKind.java | 32 + .../src/main/java/org/apache/orc/OrcConf.java | 519 +++++++ .../src/main/java/org/apache/orc/OrcFile.java | 1336 +++++++++++++++++ .../org/apache/orc/impl/PhysicalFsWriter.java | 793 ++++++++++ .../java/org/apache/orc/impl/WriterImpl.java | 1063 +++++++++++++ .../java/org/apache/orc/impl/ZstdCodec.java | 287 ++++ .../paimon/format/orc/OrcFileFormat.java | 2 +- .../src/main/resources/META-INF/NOTICE | 6 +- .../paimon/format/orc/writer/OrcZstdTest.java | 166 ++ 10 files changed, 4232 insertions(+), 5 deletions(-) create mode 100644 paimon-format/src/main/java/org/apache/orc/CompressionKind.java create mode 100644 paimon-format/src/main/java/org/apache/orc/OrcConf.java create mode 100644 paimon-format/src/main/java/org/apache/orc/OrcFile.java create mode 100644 paimon-format/src/main/java/org/apache/orc/impl/PhysicalFsWriter.java create mode 100644 paimon-format/src/main/java/org/apache/orc/impl/WriterImpl.java create mode 100644 paimon-format/src/main/java/org/apache/orc/impl/ZstdCodec.java create mode 100644 paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcZstdTest.java diff --git a/paimon-format/pom.xml b/paimon-format/pom.xml index b9e755996011..1b38aad99bf5 100644 --- a/paimon-format/pom.xml +++ b/paimon-format/pom.xml @@ -33,10 +33,13 @@ under the License. 1.13.1 - 1.8.3 + 1.9.2 2.5 1.6 3.12.0 + 1.5.5-11 + 2.8.1 + 3.17.3 @@ -53,6 +56,12 @@ under the License. ${snappy.version} + + com.github.luben + zstd-jni + ${zstd-jni.version} + + @@ -116,6 +125,24 @@ under the License. ${commons.lang3.version} + + org.apache.hive + hive-storage-api + ${storage-api.version} + + + org.apache.hadoop + hadoop-hdfs + + + + + + com.google.protobuf + protobuf-java + ${protobuf-java.version} + + @@ -150,6 +177,10 @@ under the License. org.xerial.snappy snappy-java + + zstd-jni + com.github.luben + diff --git a/paimon-format/src/main/java/org/apache/orc/CompressionKind.java b/paimon-format/src/main/java/org/apache/orc/CompressionKind.java new file mode 100644 index 000000000000..9841f5a2919e --- /dev/null +++ b/paimon-format/src/main/java/org/apache/orc/CompressionKind.java @@ -0,0 +1,32 @@ +/* + * 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.orc; + +/** + * An enumeration that lists the generic compression algorithms that can be applied to ORC files. + */ +public enum CompressionKind { + NONE, + ZLIB, + SNAPPY, + LZO, + LZ4, + ZSTD, + BROTLI +} diff --git a/paimon-format/src/main/java/org/apache/orc/OrcConf.java b/paimon-format/src/main/java/org/apache/orc/OrcConf.java new file mode 100644 index 000000000000..2e1507eedf82 --- /dev/null +++ b/paimon-format/src/main/java/org/apache/orc/OrcConf.java @@ -0,0 +1,519 @@ +/* + * 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.orc; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; + +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +/** Define the configuration properties that Orc understands. */ +public enum OrcConf { + STRIPE_SIZE( + "orc.stripe.size", + "hive.exec.orc.default.stripe.size", + 64L * 1024 * 1024, + "Define the default ORC stripe size, in bytes."), + STRIPE_ROW_COUNT( + "orc.stripe.row.count", + "orc.stripe.row.count", + Integer.MAX_VALUE, + "This value limit the row count in one stripe. \n" + + "The number of stripe rows can be controlled at \n" + + "(0, \"orc.stripe.row.count\" + max(batchSize, \"orc.rows.between.memory.checks\"))"), + BLOCK_SIZE( + "orc.block.size", + "hive.exec.orc.default.block.size", + 256L * 1024 * 1024, + "Define the default file system block size for ORC files."), + ENABLE_INDEXES( + "orc.create.index", + "orc.create.index", + true, + "Should the ORC writer create indexes as part of the file."), + ROW_INDEX_STRIDE( + "orc.row.index.stride", + "hive.exec.orc.default.row.index.stride", + 10000, + "Define the default ORC index stride in number of rows. (Stride is the\n" + + " number of rows an index entry represents.)"), + BUFFER_SIZE( + "orc.compress.size", + "hive.exec.orc.default.buffer.size", + 256 * 1024, + "Define the default ORC buffer size, in bytes."), + BASE_DELTA_RATIO( + "orc.base.delta.ratio", + "hive.exec.orc.base.delta.ratio", + 8, + "The ratio of base writer and delta writer in terms of STRIPE_SIZE and BUFFER_SIZE."), + BLOCK_PADDING( + "orc.block.padding", + "hive.exec.orc.default.block.padding", + true, + "Define whether stripes should be padded to the HDFS block boundaries."), + COMPRESS( + "orc.compress", + "hive.exec.orc.default.compress", + "ZLIB", + "Define the default compression codec for ORC file"), + WRITE_FORMAT( + "orc.write.format", + "hive.exec.orc.write.format", + "0.12", + "Define the version of the file to write. Possible values are 0.11 and\n" + + " 0.12. If this parameter is not defined, ORC will use the run\n" + + " length encoding (RLE) introduced in Hive 0.12."), + ENFORCE_COMPRESSION_BUFFER_SIZE( + "orc.buffer.size.enforce", + "hive.exec.orc.buffer.size.enforce", + false, + "Defines whether to enforce ORC compression buffer size."), + ENCODING_STRATEGY( + "orc.encoding.strategy", + "hive.exec.orc.encoding.strategy", + "SPEED", + "Define the encoding strategy to use while writing data. Changing this\n" + + "will only affect the light weight encoding for integers. This\n" + + "flag will not change the compression level of higher level\n" + + "compression codec (like ZLIB)."), + COMPRESSION_STRATEGY( + "orc.compression.strategy", + "hive.exec.orc.compression.strategy", + "SPEED", + "Define the compression strategy to use while writing data.\n" + + "This changes the compression level of higher level compression\n" + + "codec (like ZLIB)."), + COMPRESSION_ZSTD_LEVEL( + "orc.compression.zstd.level", + "hive.exec.orc.compression.zstd.level", + 1, + "Define the compression level to use with ZStandard codec " + + "while writing data. The valid range is 1~22"), + COMPRESSION_ZSTD_WINDOWLOG( + "orc.compression.zstd.windowlog", + "hive.exec.orc.compression.zstd.windowlog", + 0, + "Set the maximum allowed back-reference distance for " + + "ZStandard codec, expressed as power of 2."), + BLOCK_PADDING_TOLERANCE( + "orc.block.padding.tolerance", + "hive.exec.orc.block.padding.tolerance", + 0.05, + "Define the tolerance for block padding as a decimal fraction of\n" + + "stripe size (for example, the default value 0.05 is 5% of the\n" + + "stripe size). For the defaults of 64Mb ORC stripe and 256Mb HDFS\n" + + "blocks, the default block padding tolerance of 5% will\n" + + "reserve a maximum of 3.2Mb for padding within the 256Mb block.\n" + + "In that case, if the available size within the block is more than\n" + + "3.2Mb, a new smaller stripe will be inserted to fit within that\n" + + "space. This will make sure that no stripe written will block\n" + + " boundaries and cause remote reads within a node local task."), + BLOOM_FILTER_FPP( + "orc.bloom.filter.fpp", + "orc.default.bloom.fpp", + 0.01, + "Define the default false positive probability for bloom filters."), + USE_ZEROCOPY( + "orc.use.zerocopy", + "hive.exec.orc.zerocopy", + false, + "Use zerocopy reads with ORC. (This requires Hadoop 2.3 or later.)"), + SKIP_CORRUPT_DATA( + "orc.skip.corrupt.data", + "hive.exec.orc.skip.corrupt.data", + false, + "If ORC reader encounters corrupt data, this value will be used to\n" + + "determine whether to skip the corrupt data or throw exception.\n" + + "The default behavior is to throw exception."), + TOLERATE_MISSING_SCHEMA( + "orc.tolerate.missing.schema", + "hive.exec.orc.tolerate.missing.schema", + true, + "Writers earlier than HIVE-4243 may have inaccurate schema metadata.\n" + + "This setting will enable best effort schema evolution rather\n" + + "than rejecting mismatched schemas"), + MEMORY_POOL( + "orc.memory.pool", + "hive.exec.orc.memory.pool", + 0.5, + "Maximum fraction of heap that can be used by ORC file writers"), + DICTIONARY_KEY_SIZE_THRESHOLD( + "orc.dictionary.key.threshold", + "hive.exec.orc.dictionary.key.size.threshold", + 0.8, + "If the number of distinct keys in a dictionary is greater than this\n" + + "fraction of the total number of non-null rows, turn off \n" + + "dictionary encoding. Use 1 to always use dictionary encoding."), + ROW_INDEX_STRIDE_DICTIONARY_CHECK( + "orc.dictionary.early.check", + "hive.orc.row.index.stride.dictionary.check", + true, + "If enabled dictionary check will happen after first row index stride\n" + + "(default 10000 rows) else dictionary check will happen before\n" + + "writing first stripe. In both cases, the decision to use\n" + + "dictionary or not will be retained thereafter."), + DICTIONARY_IMPL( + "orc.dictionary.implementation", + "orc.dictionary.implementation", + "rbtree", + "the implementation for the dictionary used for string-type column encoding.\n" + + "The choices are:\n" + + " rbtree - use red-black tree as the implementation for the dictionary.\n" + + " hash - use hash table as the implementation for the dictionary."), + BLOOM_FILTER_COLUMNS( + "orc.bloom.filter.columns", + "orc.bloom.filter.columns", + "", + "List of columns to create bloom filters for when writing."), + BLOOM_FILTER_WRITE_VERSION( + "orc.bloom.filter.write.version", + "orc.bloom.filter.write.version", + OrcFile.BloomFilterVersion.UTF8.toString(), + "Which version of the bloom filters should we write.\n" + + "The choices are:\n" + + " original - writes two versions of the bloom filters for use by\n" + + " both old and new readers.\n" + + " utf8 - writes just the new bloom filters."), + IGNORE_NON_UTF8_BLOOM_FILTERS( + "orc.bloom.filter.ignore.non-utf8", + "orc.bloom.filter.ignore.non-utf8", + false, + "Should the reader ignore the obsolete non-UTF8 bloom filters."), + MAX_FILE_LENGTH( + "orc.max.file.length", + "orc.max.file.length", + Long.MAX_VALUE, + "The maximum size of the file to read for finding the file tail. This\n" + + "is primarily used for streaming ingest to read intermediate\n" + + "footers while the file is still open"), + MAPRED_INPUT_SCHEMA( + "orc.mapred.input.schema", + null, + null, + "The schema that the user desires to read. The values are\n" + + "interpreted using TypeDescription.fromString."), + MAPRED_SHUFFLE_KEY_SCHEMA( + "orc.mapred.map.output.key.schema", + null, + null, + "The schema of the MapReduce shuffle key. The values are\n" + + "interpreted using TypeDescription.fromString."), + MAPRED_SHUFFLE_VALUE_SCHEMA( + "orc.mapred.map.output.value.schema", + null, + null, + "The schema of the MapReduce shuffle value. The values are\n" + + "interpreted using TypeDescription.fromString."), + MAPRED_OUTPUT_SCHEMA( + "orc.mapred.output.schema", + null, + null, + "The schema that the user desires to write. The values are\n" + + "interpreted using TypeDescription.fromString."), + INCLUDE_COLUMNS( + "orc.include.columns", + "hive.io.file.readcolumn.ids", + null, + "The list of comma separated column ids that should be read with 0\n" + + "being the first column, 1 being the next, and so on. ."), + KRYO_SARG( + "orc.kryo.sarg", + "orc.kryo.sarg", + null, + "The kryo and base64 encoded SearchArgument for predicate pushdown."), + KRYO_SARG_BUFFER( + "orc.kryo.sarg.buffer", + null, + 8192, + "The kryo buffer size for SearchArgument for predicate pushdown."), + SARG_COLUMNS( + "orc.sarg.column.names", + "orc.sarg.column.names", + null, + "The list of column names for the SearchArgument."), + FORCE_POSITIONAL_EVOLUTION( + "orc.force.positional.evolution", + "orc.force.positional.evolution", + false, + "Require schema evolution to match the top level columns using position\n" + + "rather than column names. This provides backwards compatibility with\n" + + "Hive 2.1."), + FORCE_POSITIONAL_EVOLUTION_LEVEL( + "orc.force.positional.evolution.level", + "orc.force.positional.evolution.level", + 1, + "Require schema evolution to match the the defined no. of level columns using position\n" + + "rather than column names. This provides backwards compatibility with Hive 2.1."), + ROWS_BETWEEN_CHECKS( + "orc.rows.between.memory.checks", + "orc.rows.between.memory.checks", + 5000, + "How often should MemoryManager check the memory sizes? Measured in rows\n" + + "added to all of the writers. Valid range is [1,10000] and is primarily meant for" + + "testing. Setting this too low may negatively affect performance." + + " Use orc.stripe.row.count instead if the value larger than orc.stripe.row.count."), + OVERWRITE_OUTPUT_FILE( + "orc.overwrite.output.file", + "orc.overwrite.output.file", + false, + "A boolean flag to enable overwriting of the output file if it already exists.\n"), + IS_SCHEMA_EVOLUTION_CASE_SENSITIVE( + "orc.schema.evolution.case.sensitive", + "orc.schema.evolution.case.sensitive", + true, + "A boolean flag to determine if the comparision of field names " + + "in schema evolution is case sensitive .\n"), + ALLOW_SARG_TO_FILTER( + "orc.sarg.to.filter", + "orc.sarg.to.filter", + false, + "A boolean flag to determine if a SArg is allowed to become a filter"), + READER_USE_SELECTED( + "orc.filter.use.selected", + "orc.filter.use.selected", + false, + "A boolean flag to determine if the selected vector is supported by\n" + + "the reading application. If false, the output of the ORC reader " + + "must have the filter\n" + + "reapplied to avoid using unset values in the unselected rows.\n" + + "If unsure please leave this as false."), + ALLOW_PLUGIN_FILTER( + "orc.filter.plugin", + "orc.filter.plugin", + false, + "Enables the use of plugin filters during read. The plugin filters " + + "are discovered against the service " + + "org.apache.orc.filter.PluginFilterService, if multiple filters are " + + "determined, they are combined using AND. The order of application is " + + "non-deterministic and the filter functionality should not depend on the " + + "order of application."), + WRITE_VARIABLE_LENGTH_BLOCKS( + "orc.write.variable.length.blocks", + null, + false, + "A boolean flag as to whether the ORC writer should write variable length\n" + + "HDFS blocks."), + DIRECT_ENCODING_COLUMNS( + "orc.column.encoding.direct", + "orc.column.encoding.direct", + "", + "Comma-separated list of columns for which dictionary encoding is to be skipped."), + // some JVM doesn't allow array creation of size Integer.MAX_VALUE, so chunk size is slightly + // less than max int + ORC_MAX_DISK_RANGE_CHUNK_LIMIT( + "orc.max.disk.range.chunk.limit", + "hive.exec.orc.max.disk.range.chunk.limit", + Integer.MAX_VALUE - 1024, + "When reading stripes >2GB, specify max limit for the chunk size."), + ORC_MIN_DISK_SEEK_SIZE( + "orc.min.disk.seek.size", + "orc.min.disk.seek.size", + 0, + "When determining contiguous reads, gaps within this size are " + + "read contiguously and not seeked. Default value of zero disables this " + + "optimization"), + ORC_MIN_DISK_SEEK_SIZE_TOLERANCE( + "orc.min.disk.seek.size.tolerance", + "orc.min.disk.seek.size.tolerance", + 0.00, + "Define the tolerance for for extra bytes read as a result of " + + "orc.min.disk.seek.size. If the " + + "(bytesRead - bytesNeeded) / bytesNeeded is greater than this " + + "threshold then extra work is performed to drop the extra bytes from " + + "memory after the read."), + ENCRYPTION("orc.encrypt", "orc.encrypt", null, "The list of keys and columns to encrypt with"), + DATA_MASK("orc.mask", "orc.mask", null, "The masks to apply to the encrypted columns"), + KEY_PROVIDER( + "orc.key.provider", + "orc.key.provider", + "hadoop", + "The kind of KeyProvider to use for encryption."), + PROLEPTIC_GREGORIAN( + "orc.proleptic.gregorian", + "orc.proleptic.gregorian", + false, + "Should we read and write dates & times using the proleptic Gregorian calendar\n" + + "instead of the hybrid Julian Gregorian? Hive before 3.1 and Spark before 3.0\n" + + "used hybrid."), + PROLEPTIC_GREGORIAN_DEFAULT( + "orc.proleptic.gregorian.default", + "orc.proleptic.gregorian.default", + false, + "This value controls whether pre-ORC 27 files are using the hybrid or proleptic\n" + + "calendar. Only Hive 3.1 and the C++ library wrote using the proleptic, so hybrid\n" + + "is the default."), + ROW_BATCH_SIZE( + "orc.row.batch.size", + "orc.row.batch.size", + 1024, + "The number of rows to include in a orc vectorized reader batch. " + + "The value should be carefully chosen to minimize overhead and avoid OOMs in reading data."), + ROW_BATCH_CHILD_LIMIT( + "orc.row.child.limit", + "orc.row.child.limit", + 1024 * 32, + "The maximum number of child elements to buffer before " + + "the ORC row writer writes the batch to the file."); + + private final String attribute; + private final String hiveConfName; + private final Object defaultValue; + private final String description; + + OrcConf(String attribute, String hiveConfName, Object defaultValue, String description) { + this.attribute = attribute; + this.hiveConfName = hiveConfName; + this.defaultValue = defaultValue; + this.description = description; + } + + public String getAttribute() { + return attribute; + } + + public String getHiveConfName() { + return hiveConfName; + } + + public Object getDefaultValue() { + return defaultValue; + } + + public String getDescription() { + return description; + } + + private String lookupValue(Properties tbl, Configuration conf) { + String result = null; + if (tbl != null) { + result = tbl.getProperty(attribute); + } + if (result == null && conf != null) { + result = conf.get(attribute); + if (result == null && hiveConfName != null) { + result = conf.get(hiveConfName); + } + } + return result; + } + + public int getInt(Properties tbl, Configuration conf) { + String value = lookupValue(tbl, conf); + if (value != null) { + return Integer.parseInt(value); + } + return ((Number) defaultValue).intValue(); + } + + public int getInt(Configuration conf) { + return getInt(null, conf); + } + + /** + * @deprecated Use {@link #getInt(Configuration)} instead. This method was incorrectly added and + * shouldn't be used anymore. + */ + @Deprecated + public void getInt(Configuration conf, int value) { + // noop + } + + public void setInt(Configuration conf, int value) { + conf.setInt(attribute, value); + } + + public long getLong(Properties tbl, Configuration conf) { + String value = lookupValue(tbl, conf); + if (value != null) { + return Long.parseLong(value); + } + return ((Number) defaultValue).longValue(); + } + + public long getLong(Configuration conf) { + return getLong(null, conf); + } + + public void setLong(Configuration conf, long value) { + conf.setLong(attribute, value); + } + + public String getString(Properties tbl, Configuration conf) { + String value = lookupValue(tbl, conf); + return value == null ? (String) defaultValue : value; + } + + public String getString(Configuration conf) { + return getString(null, conf); + } + + public List getStringAsList(Configuration conf) { + String value = getString(null, conf); + List confList = new ArrayList<>(); + if (StringUtils.isEmpty(value)) { + return confList; + } + for (String str : value.split(",")) { + String trimStr = StringUtils.trim(str); + if (StringUtils.isNotEmpty(trimStr)) { + confList.add(trimStr); + } + } + return confList; + } + + public void setString(Configuration conf, String value) { + conf.set(attribute, value); + } + + public boolean getBoolean(Properties tbl, Configuration conf) { + String value = lookupValue(tbl, conf); + if (value != null) { + return Boolean.parseBoolean(value); + } + return (Boolean) defaultValue; + } + + public boolean getBoolean(Configuration conf) { + return getBoolean(null, conf); + } + + public void setBoolean(Configuration conf, boolean value) { + conf.setBoolean(attribute, value); + } + + public double getDouble(Properties tbl, Configuration conf) { + String value = lookupValue(tbl, conf); + if (value != null) { + return Double.parseDouble(value); + } + return ((Number) defaultValue).doubleValue(); + } + + public double getDouble(Configuration conf) { + return getDouble(null, conf); + } + + public void setDouble(Configuration conf, double value) { + conf.setDouble(attribute, value); + } +} diff --git a/paimon-format/src/main/java/org/apache/orc/OrcFile.java b/paimon-format/src/main/java/org/apache/orc/OrcFile.java new file mode 100644 index 000000000000..a903ba9e7545 --- /dev/null +++ b/paimon-format/src/main/java/org/apache/orc/OrcFile.java @@ -0,0 +1,1336 @@ +/* + * 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.orc; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.orc.impl.HadoopShims; +import org.apache.orc.impl.HadoopShimsFactory; +import org.apache.orc.impl.KeyProvider; +import org.apache.orc.impl.MemoryManagerImpl; +import org.apache.orc.impl.OrcTail; +import org.apache.orc.impl.ReaderImpl; +import org.apache.orc.impl.WriterImpl; +import org.apache.orc.impl.WriterInternal; +import org.apache.orc.impl.writer.WriterImplV2; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** Contains factory methods to read or write ORC files. */ +public class OrcFile { + private static final Logger LOG = LoggerFactory.getLogger(OrcFile.class); + public static final String MAGIC = "ORC"; + + /** + * Create a version number for the ORC file format, so that we can add non-forward compatible + * changes in the future. To make it easier for users to understand the version numbers, we use + * the Hive release number that first wrote that version of ORC files. + * + *

Thus, if you add new encodings or other non-forward compatible changes to ORC files, which + * prevent the old reader from reading the new format, you should change these variable to + * reflect the next Hive release number. Non-forward compatible changes should never be added in + * patch releases. + * + *

Do not make any changes that break backwards compatibility, which would prevent the new + * reader from reading ORC files generated by any released version of Hive. + */ + public enum Version { + V_0_11("0.11", 0, 11), + V_0_12("0.12", 0, 12), + + /** + * Do not use this format except for testing. It will not be compatible with other versions + * of the software. While we iterate on the ORC 2.0 format, we will make incompatible format + * changes under this version without providing any forward or backward compatibility. + * + *

When 2.0 is released, this version identifier will be completely removed. + */ + UNSTABLE_PRE_2_0("UNSTABLE-PRE-2.0", 1, 9999), + + /** The generic identifier for all unknown versions. */ + FUTURE("future", Integer.MAX_VALUE, Integer.MAX_VALUE); + + public static final Version CURRENT = V_0_12; + + private final String name; + private final int major; + private final int minor; + + Version(String name, int major, int minor) { + this.name = name; + this.major = major; + this.minor = minor; + } + + public static Version byName(String name) { + for (Version version : values()) { + if (version.name.equals(name)) { + return version; + } + } + throw new IllegalArgumentException("Unknown ORC version " + name); + } + + /** Get the human readable name for the version. */ + public String getName() { + return name; + } + + /** Get the major version number. */ + public int getMajor() { + return major; + } + + /** Get the minor version number. */ + public int getMinor() { + return minor; + } + } + + /** WriterImplementation Enum. */ + public enum WriterImplementation { + /** ORC_JAVA. */ + ORC_JAVA(0), // ORC Java writer + /** ORC_CPP. */ + ORC_CPP(1), // ORC C++ writer + /** PRESTO. */ + PRESTO(2), // Presto writer + /** SCRITCHLEY_GO. */ + SCRITCHLEY_GO(3), // Go writer from https://github.com/scritchley/orc + /** TRINO. */ + TRINO(4), // Trino writer + /** CUDF. */ + CUDF(5), // CUDF writer + /** UNKNOWN. */ + UNKNOWN(Integer.MAX_VALUE); + + private final int id; + + WriterImplementation(int id) { + this.id = id; + } + + public int getId() { + return id; + } + + public static WriterImplementation from(int id) { + WriterImplementation[] values = values(); + if (id >= 0 && id < values.length - 1) { + return values[id]; + } + return UNKNOWN; + } + } + + /** + * Records the version of the writer in terms of which bugs have been fixed. When you fix bugs + * in the writer (or make substantial changes) that don't change the file format, add a new + * version here instead of Version. + * + *

The ids are assigned sequentially from 6 per a WriterImplementation so that readers that + * predate ORC-202 treat the other writers correctly. + */ + public enum WriterVersion { + // Java ORC Writer. + ORIGINAL(WriterImplementation.ORC_JAVA, 0), + HIVE_8732(WriterImplementation.ORC_JAVA, 1), + /** fixed stripe/file maximum statistics and string statistics to use utf8 for min/max. */ + HIVE_4243(WriterImplementation.ORC_JAVA, 2), // use real column names from Hive tables + HIVE_12055(WriterImplementation.ORC_JAVA, 3), // vectorized writer + HIVE_13083(WriterImplementation.ORC_JAVA, 4), // decimals write present stream correctly + ORC_101(WriterImplementation.ORC_JAVA, 5), // bloom filters use utf8 + ORC_135(WriterImplementation.ORC_JAVA, 6), // timestamp stats use utc + ORC_517(WriterImplementation.ORC_JAVA, 7), // decimal64 min/max are fixed + ORC_203(WriterImplementation.ORC_JAVA, 8), // trim long strings & record they were trimmed + ORC_14(WriterImplementation.ORC_JAVA, 9), // column encryption added + + // C++ ORC Writer + ORC_CPP_ORIGINAL(WriterImplementation.ORC_CPP, 6), + + // Presto Writer + PRESTO_ORIGINAL(WriterImplementation.PRESTO, 6), + + // Scritchley Go Writer + SCRITCHLEY_GO_ORIGINAL(WriterImplementation.SCRITCHLEY_GO, 6), + + // Trino Writer + TRINO_ORIGINAL(WriterImplementation.TRINO, 6), + + // CUDF Writer + CUDF_ORIGINAL(WriterImplementation.CUDF, 6), + + // Don't use any magic numbers here except for the below: + FUTURE(WriterImplementation.UNKNOWN, Integer.MAX_VALUE); // a version from a future writer + + private final int id; + private final WriterImplementation writer; + + public WriterImplementation getWriterImplementation() { + return writer; + } + + public int getId() { + return id; + } + + WriterVersion(WriterImplementation writer, int id) { + this.writer = writer; + this.id = id; + } + + private static final WriterVersion[][] values = + new WriterVersion[WriterImplementation.values().length][]; + + static { + for (WriterVersion v : WriterVersion.values()) { + WriterImplementation writer = v.writer; + if (writer != WriterImplementation.UNKNOWN) { + if (values[writer.id] == null) { + values[writer.id] = new WriterVersion[WriterVersion.values().length]; + } + if (values[writer.id][v.id] != null) { + throw new IllegalArgumentException("Duplicate WriterVersion id " + v); + } + values[writer.id][v.id] = v; + } + } + } + + /** + * Convert the integer from OrcProto.PostScript.writerVersion to the enumeration with + * unknown versions being mapped to FUTURE. + * + * @param writer the writer implementation + * @param val the serialized writer version + * @return the corresponding enumeration value + */ + public static WriterVersion from(WriterImplementation writer, int val) { + if (writer == WriterImplementation.UNKNOWN) { + return FUTURE; + } + if (writer != WriterImplementation.ORC_JAVA && val < 6) { + throw new IllegalArgumentException( + "ORC File with illegal version " + val + " for writer " + writer); + } + WriterVersion[] versions = values[writer.id]; + if (val < 0 || versions.length <= val) { + return FUTURE; + } + WriterVersion result = versions[val]; + return result == null ? FUTURE : result; + } + + /** + * Does this file include the given fix or come from a different writer? + * + * @param fix the required fix + * @return true if the required fix is present + */ + public boolean includes(WriterVersion fix) { + return writer != fix.writer || id >= fix.id; + } + } + + /** The WriterVersion for this version of the software. */ + public static final WriterVersion CURRENT_WRITER = WriterVersion.ORC_14; + + /** EncodingStrategy Enum. */ + public enum EncodingStrategy { + /** SPEED. */ + SPEED, + /** COMPRESSION. */ + COMPRESSION + } + + /** CompressionStrategy Enum. */ + public enum CompressionStrategy { + /** SPEED. */ + SPEED, + /** COMPRESSION. */ + COMPRESSION + } + + // unused + protected OrcFile() {} + + /** Orc ReaderOptions. */ + public static class ReaderOptions { + private final Configuration conf; + private FileSystem filesystem; + private long maxLength = Long.MAX_VALUE; + private OrcTail orcTail; + private KeyProvider keyProvider; + // TODO: We can generalize FileMetadata interface. Make OrcTail implement FileMetadata + // interface + // and remove this class altogether. Both footer caching and llap caching just needs + // OrcTail. + // For now keeping this around to avoid complex surgery + private FileMetadata fileMetadata; + private boolean useUTCTimestamp; + private boolean useProlepticGregorian; + + public ReaderOptions(Configuration conf) { + this.conf = conf; + this.useProlepticGregorian = OrcConf.PROLEPTIC_GREGORIAN.getBoolean(conf); + } + + public ReaderOptions filesystem(FileSystem fs) { + this.filesystem = fs; + return this; + } + + public ReaderOptions maxLength(long val) { + maxLength = val; + return this; + } + + public ReaderOptions orcTail(OrcTail tail) { + this.orcTail = tail; + return this; + } + + /** + * Set the KeyProvider to override the default for getting keys. + * + * @param provider + * @return + */ + public ReaderOptions setKeyProvider(KeyProvider provider) { + this.keyProvider = provider; + return this; + } + + /** + * Should the reader convert dates and times to the proleptic Gregorian calendar? + * + * @param newValue should it use the proleptic Gregorian calendar? + * @return this + */ + public ReaderOptions convertToProlepticGregorian(boolean newValue) { + this.useProlepticGregorian = newValue; + return this; + } + + public Configuration getConfiguration() { + return conf; + } + + public FileSystem getFilesystem() { + return filesystem; + } + + public long getMaxLength() { + return maxLength; + } + + public OrcTail getOrcTail() { + return orcTail; + } + + public KeyProvider getKeyProvider() { + return keyProvider; + } + + /** @deprecated Use {@link #orcTail(OrcTail)} instead. */ + public ReaderOptions fileMetadata(final FileMetadata metadata) { + fileMetadata = metadata; + return this; + } + + public FileMetadata getFileMetadata() { + return fileMetadata; + } + + public ReaderOptions useUTCTimestamp(boolean value) { + useUTCTimestamp = value; + return this; + } + + public boolean getUseUTCTimestamp() { + return useUTCTimestamp; + } + + public boolean getConvertToProlepticGregorian() { + return useProlepticGregorian; + } + } + + public static ReaderOptions readerOptions(Configuration conf) { + return new ReaderOptions(conf); + } + + public static Reader createReader(Path path, ReaderOptions options) throws IOException { + return new ReaderImpl(path, options); + } + + /** WriterContext. */ + public interface WriterContext { + Writer getWriter(); + } + + /** WriterCallback. */ + public interface WriterCallback { + void preStripeWrite(WriterContext context) throws IOException; + + void preFooterWrite(WriterContext context) throws IOException; + } + + /** BloomFilterVersion. */ + public enum BloomFilterVersion { + // Include both the BLOOM_FILTER and BLOOM_FILTER_UTF8 streams to support + // both old and new readers. + ORIGINAL("original"), + // Only include the BLOOM_FILTER_UTF8 streams that consistently use UTF8. + // See ORC-101 + UTF8("utf8"); + + private final String id; + + BloomFilterVersion(String id) { + this.id = id; + } + + @Override + public String toString() { + return id; + } + + public static BloomFilterVersion fromString(String s) { + for (BloomFilterVersion version : values()) { + if (version.id.equals(s)) { + return version; + } + } + throw new IllegalArgumentException("Unknown BloomFilterVersion " + s); + } + } + + /** ZstdCompressOptions. */ + public static class ZstdCompressOptions { + private int compressionZstdLevel; + private int compressionZstdWindowLog; + + public int getCompressionZstdLevel() { + return compressionZstdLevel; + } + + public void setCompressionZstdLevel(int compressionZstdLevel) { + this.compressionZstdLevel = compressionZstdLevel; + } + + public int getCompressionZstdWindowLog() { + return compressionZstdWindowLog; + } + + public void setCompressionZstdWindowLog(int compressionZstdWindowLog) { + this.compressionZstdWindowLog = compressionZstdWindowLog; + } + } + + /** Options for creating ORC file writers. */ + public static class WriterOptions implements Cloneable { + private final Configuration configuration; + private FileSystem fileSystemValue = null; + private TypeDescription schema = null; + private long stripeSizeValue; + private long stripeRowCountValue; + private long blockSizeValue; + private boolean buildIndex; + private int rowIndexStrideValue; + private int bufferSizeValue; + private boolean enforceBufferSize = false; + private boolean blockPaddingValue; + private CompressionKind compressValue; + private MemoryManager memoryManagerValue; + private Version versionValue; + private WriterCallback callback; + private EncodingStrategy encodingStrategy; + private CompressionStrategy compressionStrategy; + private ZstdCompressOptions zstdCompressOptions; + private double paddingTolerance; + private String bloomFilterColumns; + private double bloomFilterFpp; + private BloomFilterVersion bloomFilterVersion; + private PhysicalWriter physicalWriter; + private WriterVersion writerVersion = CURRENT_WRITER; + private boolean useUTCTimestamp; + private boolean overwrite; + private boolean writeVariableLengthBlocks; + private HadoopShims shims; + private String directEncodingColumns; + private String encryption; + private String masks; + private KeyProvider provider; + private boolean useProlepticGregorian; + private Map keyOverrides = new HashMap<>(); + + protected WriterOptions(Properties tableProperties, Configuration conf) { + configuration = conf; + memoryManagerValue = getStaticMemoryManager(conf); + overwrite = OrcConf.OVERWRITE_OUTPUT_FILE.getBoolean(tableProperties, conf); + stripeSizeValue = OrcConf.STRIPE_SIZE.getLong(tableProperties, conf); + stripeRowCountValue = OrcConf.STRIPE_ROW_COUNT.getLong(tableProperties, conf); + blockSizeValue = OrcConf.BLOCK_SIZE.getLong(tableProperties, conf); + buildIndex = OrcConf.ENABLE_INDEXES.getBoolean(tableProperties, conf); + rowIndexStrideValue = (int) OrcConf.ROW_INDEX_STRIDE.getLong(tableProperties, conf); + bufferSizeValue = (int) OrcConf.BUFFER_SIZE.getLong(tableProperties, conf); + blockPaddingValue = OrcConf.BLOCK_PADDING.getBoolean(tableProperties, conf); + compressValue = + CompressionKind.valueOf( + OrcConf.COMPRESS.getString(tableProperties, conf).toUpperCase()); + enforceBufferSize = + OrcConf.ENFORCE_COMPRESSION_BUFFER_SIZE.getBoolean(tableProperties, conf); + String versionName = OrcConf.WRITE_FORMAT.getString(tableProperties, conf); + versionValue = Version.byName(versionName); + String enString = OrcConf.ENCODING_STRATEGY.getString(tableProperties, conf); + encodingStrategy = EncodingStrategy.valueOf(enString); + + String compString = OrcConf.COMPRESSION_STRATEGY.getString(tableProperties, conf); + compressionStrategy = CompressionStrategy.valueOf(compString); + + zstdCompressOptions = new ZstdCompressOptions(); + zstdCompressOptions.setCompressionZstdLevel( + OrcConf.COMPRESSION_ZSTD_LEVEL.getInt(tableProperties, conf)); + zstdCompressOptions.setCompressionZstdWindowLog( + OrcConf.COMPRESSION_ZSTD_WINDOWLOG.getInt(tableProperties, conf)); + + paddingTolerance = OrcConf.BLOCK_PADDING_TOLERANCE.getDouble(tableProperties, conf); + + bloomFilterColumns = OrcConf.BLOOM_FILTER_COLUMNS.getString(tableProperties, conf); + bloomFilterFpp = OrcConf.BLOOM_FILTER_FPP.getDouble(tableProperties, conf); + bloomFilterVersion = + BloomFilterVersion.fromString( + OrcConf.BLOOM_FILTER_WRITE_VERSION.getString(tableProperties, conf)); + shims = HadoopShimsFactory.get(); + writeVariableLengthBlocks = + OrcConf.WRITE_VARIABLE_LENGTH_BLOCKS.getBoolean(tableProperties, conf); + directEncodingColumns = + OrcConf.DIRECT_ENCODING_COLUMNS.getString(tableProperties, conf); + useProlepticGregorian = OrcConf.PROLEPTIC_GREGORIAN.getBoolean(conf); + } + + /** @return a SHALLOW clone */ + @Override + public WriterOptions clone() { + try { + return (WriterOptions) super.clone(); + } catch (CloneNotSupportedException ex) { + throw new AssertionError("Expected super.clone() to work"); + } + } + + /** + * Provide the filesystem for the path, if the client has it available. If it is not + * provided, it will be found from the path. + */ + public WriterOptions fileSystem(FileSystem value) { + fileSystemValue = value; + return this; + } + + /** + * If the output file already exists, should it be overwritten? If it is not provided, write + * operation will fail if the file already exists. + */ + public WriterOptions overwrite(boolean value) { + overwrite = value; + return this; + } + + /** + * Set the stripe size for the file. The writer stores the contents of the stripe in memory + * until this memory limit is reached and the stripe is flushed to the HDFS file and the + * next stripe started. + */ + public WriterOptions stripeSize(long value) { + stripeSizeValue = value; + return this; + } + + /** + * Set the file system block size for the file. For optimal performance, set the block size + * to be multiple factors of stripe size. + */ + public WriterOptions blockSize(long value) { + blockSizeValue = value; + return this; + } + + /** + * Set the distance between entries in the row index. The minimum value is 1000 to prevent + * the index from overwhelming the data. If the stride is set to 0, no indexes will be + * included in the file. + */ + public WriterOptions rowIndexStride(int value) { + rowIndexStrideValue = value; + return this; + } + + /** + * The size of the memory buffers used for compressing and storing the stripe in memory. + * NOTE: ORC writer may choose to use smaller buffer size based on stripe size and number of + * columns for efficient stripe writing and memory utilization. To enforce writer to use the + * requested buffer size use enforceBufferSize(). + */ + public WriterOptions bufferSize(int value) { + bufferSizeValue = value; + return this; + } + + /** + * Enforce writer to use requested buffer size instead of estimating buffer size based on + * stripe size and number of columns. See bufferSize() method for more info. Default: false + */ + public WriterOptions enforceBufferSize() { + enforceBufferSize = true; + return this; + } + + /** + * Sets whether the HDFS blocks are padded to prevent stripes from straddling blocks. + * Padding improves locality and thus the speed of reading, but costs space. + */ + public WriterOptions blockPadding(boolean value) { + blockPaddingValue = value; + return this; + } + + /** Sets the encoding strategy that is used to encode the data. */ + public WriterOptions encodingStrategy(EncodingStrategy strategy) { + encodingStrategy = strategy; + return this; + } + + /** Sets the tolerance for block padding as a percentage of stripe size. */ + public WriterOptions paddingTolerance(double value) { + paddingTolerance = value; + return this; + } + + /** Comma separated values of column names for which bloom filter is to be created. */ + public WriterOptions bloomFilterColumns(String columns) { + bloomFilterColumns = columns; + return this; + } + + /** + * Specify the false positive probability for bloom filter. + * + * @param fpp - false positive probability + * @return this + */ + public WriterOptions bloomFilterFpp(double fpp) { + bloomFilterFpp = fpp; + return this; + } + + /** Sets the generic compression that is used to compress the data. */ + public WriterOptions compress(CompressionKind value) { + compressValue = value; + return this; + } + + /** + * Set the schema for the file. This is a required parameter. + * + * @param schema the schema for the file. + * @return this + */ + public WriterOptions setSchema(TypeDescription schema) { + this.schema = schema; + return this; + } + + /** Sets the version of the file that will be written. */ + public WriterOptions version(Version value) { + versionValue = value; + return this; + } + + /** + * Add a listener for when the stripe and file are about to be closed. + * + * @param callback the object to be called when the stripe is closed + * @return this + */ + public WriterOptions callback(WriterCallback callback) { + this.callback = callback; + return this; + } + + /** Set the version of the bloom filters to write. */ + public WriterOptions bloomFilterVersion(BloomFilterVersion version) { + this.bloomFilterVersion = version; + return this; + } + + /** + * Change the physical writer of the ORC file. + * + *

SHOULD ONLY BE USED BY LLAP. + * + * @param writer the writer to control the layout and persistence + * @return this + */ + public WriterOptions physicalWriter(PhysicalWriter writer) { + this.physicalWriter = writer; + return this; + } + + /** A public option to set the memory manager. */ + public WriterOptions memory(MemoryManager value) { + memoryManagerValue = value; + return this; + } + + /** + * Should the ORC file writer use HDFS variable length blocks, if they are available? + * + * @param value the new value + * @return this + */ + public WriterOptions writeVariableLengthBlocks(boolean value) { + writeVariableLengthBlocks = value; + return this; + } + + /** + * Set the HadoopShims to use. This is only for testing. + * + * @param value the new value + * @return this + */ + public WriterOptions setShims(HadoopShims value) { + this.shims = value; + return this; + } + + /** + * Manually set the writer version. This is an internal API. + * + * @param version the version to write + * @return this + */ + protected WriterOptions writerVersion(WriterVersion version) { + if (version == WriterVersion.FUTURE) { + throw new IllegalArgumentException("Can't write a future version."); + } + this.writerVersion = version; + return this; + } + + /** + * Manually set the time zone for the writer to utc. If not defined, system time zone is + * assumed. + */ + public WriterOptions useUTCTimestamp(boolean value) { + useUTCTimestamp = value; + return this; + } + + /** + * Set the comma-separated list of columns that should be direct encoded. + * + * @param value the value to set + * @return this + */ + public WriterOptions directEncodingColumns(String value) { + directEncodingColumns = value; + return this; + } + + /** + * Encrypt a set of columns with a key. + * + *

Format of the string is a key-list. + * + *

    + *
  • key-list = key (';' key-list)? + *
  • key = key-name ':' field-list + *
  • field-list = field-name ( ',' field-list )? + *
  • field-name = number | field-part ('.' field-name)? + *
  • field-part = quoted string | simple name + *
+ * + * @param value a key-list of which columns to encrypt + * @return this + */ + public WriterOptions encrypt(String value) { + encryption = value; + return this; + } + + /** + * Set the masks for the unencrypted data. + * + *

Format of the string is a mask-list. + * + *

    + *
  • mask-list = mask (';' mask-list)? + *
  • mask = mask-name (',' parameter)* ':' field-list + *
  • field-list = field-name ( ',' field-list )? + *
  • field-name = number | field-part ('.' field-name)? + *
  • field-part = quoted string | simple name + *
+ * + * @param value a list of the masks and column names + * @return this + */ + public WriterOptions masks(String value) { + masks = value; + return this; + } + + /** + * For users that need to override the current version of a key, this method allows them to + * define the version and algorithm for a given key. + * + *

This will mostly be used for ORC file merging where the writer has to use the same + * version of the key that the original files used. + * + * @param keyName the key name + * @param version the version of the key to use + * @param algorithm the algorithm for the given key version + * @return this + */ + public WriterOptions setKeyVersion( + String keyName, int version, EncryptionAlgorithm algorithm) { + HadoopShims.KeyMetadata meta = new HadoopShims.KeyMetadata(keyName, version, algorithm); + keyOverrides.put(keyName, meta); + return this; + } + + /** + * Set the key provider for column encryption. + * + * @param provider the object that holds the master secrets + * @return this + */ + public WriterOptions setKeyProvider(KeyProvider provider) { + this.provider = provider; + return this; + } + + /** + * Should the writer use the proleptic Gregorian calendar for times and dates. + * + * @param newValue true if we should use the proleptic calendar + * @return this + */ + public WriterOptions setProlepticGregorian(boolean newValue) { + this.useProlepticGregorian = newValue; + return this; + } + + public KeyProvider getKeyProvider() { + return provider; + } + + public boolean getBlockPadding() { + return blockPaddingValue; + } + + public long getBlockSize() { + return blockSizeValue; + } + + public String getBloomFilterColumns() { + return bloomFilterColumns; + } + + public boolean getOverwrite() { + return overwrite; + } + + public FileSystem getFileSystem() { + return fileSystemValue; + } + + public Configuration getConfiguration() { + return configuration; + } + + public TypeDescription getSchema() { + return schema; + } + + public long getStripeSize() { + return stripeSizeValue; + } + + public long getStripeRowCountValue() { + return stripeRowCountValue; + } + + public CompressionKind getCompress() { + return compressValue; + } + + public WriterCallback getCallback() { + return callback; + } + + public Version getVersion() { + return versionValue; + } + + public MemoryManager getMemoryManager() { + return memoryManagerValue; + } + + public int getBufferSize() { + return bufferSizeValue; + } + + public boolean isEnforceBufferSize() { + return enforceBufferSize; + } + + public int getRowIndexStride() { + return rowIndexStrideValue; + } + + public boolean isBuildIndex() { + return buildIndex; + } + + public CompressionStrategy getCompressionStrategy() { + return compressionStrategy; + } + + public EncodingStrategy getEncodingStrategy() { + return encodingStrategy; + } + + public ZstdCompressOptions getZstdCompressOptions() { + return zstdCompressOptions; + } + + public double getPaddingTolerance() { + return paddingTolerance; + } + + public double getBloomFilterFpp() { + return bloomFilterFpp; + } + + public BloomFilterVersion getBloomFilterVersion() { + return bloomFilterVersion; + } + + public PhysicalWriter getPhysicalWriter() { + return physicalWriter; + } + + public WriterVersion getWriterVersion() { + return writerVersion; + } + + public boolean getWriteVariableLengthBlocks() { + return writeVariableLengthBlocks; + } + + public HadoopShims getHadoopShims() { + return shims; + } + + public boolean getUseUTCTimestamp() { + return useUTCTimestamp; + } + + public String getDirectEncodingColumns() { + return directEncodingColumns; + } + + public String getEncryption() { + return encryption; + } + + public String getMasks() { + return masks; + } + + public Map getKeyOverrides() { + return keyOverrides; + } + + public boolean getProlepticGregorian() { + return useProlepticGregorian; + } + } + + /** + * Create a set of writer options based on a configuration. + * + * @param conf the configuration to use for values + * @return A WriterOptions object that can be modified + */ + public static WriterOptions writerOptions(Configuration conf) { + return new WriterOptions(null, conf); + } + + /** + * Create a set of write options based on a set of table properties and configuration. + * + * @param tableProperties the properties of the table + * @param conf the configuration of the query + * @return a WriterOptions object that can be modified + */ + public static WriterOptions writerOptions(Properties tableProperties, Configuration conf) { + return new WriterOptions(tableProperties, conf); + } + + private static MemoryManager memoryManager = null; + + private static synchronized MemoryManager getStaticMemoryManager(Configuration conf) { + if (memoryManager == null) { + memoryManager = new MemoryManagerImpl(conf); + } + return memoryManager; + } + + /** + * Create an ORC file writer. This is the public interface for creating writers going forward + * and new options will only be added to this method. + * + * @param path filename to write to + * @param opts the options + * @return a new ORC file writer + * @throws IOException + */ + public static Writer createWriter(Path path, WriterOptions opts) throws IOException { + FileSystem fs = + opts.getFileSystem() == null + ? path.getFileSystem(opts.getConfiguration()) + : opts.getFileSystem(); + switch (opts.getVersion()) { + case V_0_11: + case V_0_12: + return new WriterImpl(fs, path, opts); + case UNSTABLE_PRE_2_0: + return new WriterImplV2(fs, path, opts); + default: + throw new IllegalArgumentException("Unknown version " + opts.getVersion()); + } + } + + /** + * Do we understand the version in the reader? + * + * @param path the path of the file + * @param reader the ORC file reader + * @return is the version understood by this writer? + */ + static boolean understandFormat(Path path, Reader reader) { + if (reader.getFileVersion() == Version.FUTURE) { + LOG.info("Can't merge {} because it has a future version.", path); + return false; + } + if (reader.getWriterVersion() == WriterVersion.FUTURE) { + LOG.info("Can't merge {} because it has a future writerVersion.", path); + return false; + } + return true; + } + + private static boolean sameKeys(EncryptionKey[] first, EncryptionKey[] next) { + if (first.length != next.length) { + return false; + } + for (int k = 0; k < first.length; ++k) { + if (!first[k].getKeyName().equals(next[k].getKeyName()) + || first[k].getKeyVersion() != next[k].getKeyVersion() + || first[k].getAlgorithm() != next[k].getAlgorithm()) { + return false; + } + } + return true; + } + + private static boolean sameMasks(DataMaskDescription[] first, DataMaskDescription[] next) { + if (first.length != next.length) { + return false; + } + for (int k = 0; k < first.length; ++k) { + if (!first[k].getName().equals(next[k].getName())) { + return false; + } + String[] firstParam = first[k].getParameters(); + String[] nextParam = next[k].getParameters(); + if (firstParam.length != nextParam.length) { + return false; + } + for (int p = 0; p < firstParam.length; ++p) { + if (!firstParam[p].equals(nextParam[p])) { + return false; + } + } + TypeDescription[] firstRoots = first[k].getColumns(); + TypeDescription[] nextRoots = next[k].getColumns(); + if (firstRoots.length != nextRoots.length) { + return false; + } + for (int r = 0; r < firstRoots.length; ++r) { + if (firstRoots[r].getId() != nextRoots[r].getId()) { + return false; + } + } + } + return true; + } + + private static boolean sameVariants(EncryptionVariant[] first, EncryptionVariant[] next) { + if (first.length != next.length) { + return false; + } + for (int k = 0; k < first.length; ++k) { + if ((first[k].getKeyDescription() == null) != (next[k].getKeyDescription() == null) + || !first[k].getKeyDescription() + .getKeyName() + .equals(next[k].getKeyDescription().getKeyName()) + || first[k].getRoot().getId() != next[k].getRoot().getId()) { + return false; + } + } + return true; + } + + /** + * Is the new reader compatible with the file that is being written? + * + * @param firstReader the first reader that others must match + * @param userMetadata the user metadata + * @param path the new path name for warning messages + * @param reader the new reader + * @return is the reader compatible with the previous ones? + */ + static boolean readerIsCompatible( + Reader firstReader, Map userMetadata, Path path, Reader reader) { + // now we have to check compatibility + TypeDescription schema = firstReader.getSchema(); + if (!reader.getSchema().equals(schema)) { + LOG.info( + "Can't merge {} because of different schemas {} vs {}", + path, + reader.getSchema(), + schema); + return false; + } + CompressionKind compression = firstReader.getCompressionKind(); + if (reader.getCompressionKind() != compression) { + LOG.info( + "Can't merge {} because of different compression {} vs {}", + path, + reader.getCompressionKind(), + compression); + return false; + } + Version fileVersion = firstReader.getFileVersion(); + if (reader.getFileVersion() != fileVersion) { + LOG.info( + "Can't merge {} because of different file versions {} vs {}", + path, + reader.getFileVersion(), + fileVersion); + return false; + } + WriterVersion writerVersion = firstReader.getWriterVersion(); + if (reader.getWriterVersion() != writerVersion) { + LOG.info( + "Can't merge {} because of different writer versions {} vs {}", + path, + reader.getFileVersion(), + fileVersion); + return false; + } + int rowIndexStride = firstReader.getRowIndexStride(); + if (reader.getRowIndexStride() != rowIndexStride) { + LOG.info( + "Can't merge {} because of different row index strides {} vs {}", + path, + reader.getRowIndexStride(), + rowIndexStride); + return false; + } + for (String key : reader.getMetadataKeys()) { + ByteBuffer currentValue = userMetadata.get(key); + if (currentValue != null) { + ByteBuffer newValue = reader.getMetadataValue(key); + if (!newValue.equals(currentValue)) { + LOG.info("Can't merge {} because of different user metadata {}", path, key); + return false; + } + } + } + if (!sameKeys(firstReader.getColumnEncryptionKeys(), reader.getColumnEncryptionKeys())) { + LOG.info("Can't merge {} because it has different encryption keys", path); + return false; + } + if (!sameMasks(firstReader.getDataMasks(), reader.getDataMasks())) { + LOG.info("Can't merge {} because it has different encryption masks", path); + return false; + } + if (!sameVariants(firstReader.getEncryptionVariants(), reader.getEncryptionVariants())) { + LOG.info("Can't merge {} because it has different encryption variants", path); + return false; + } + if (firstReader.writerUsedProlepticGregorian() != reader.writerUsedProlepticGregorian()) { + LOG.info("Can't merge {} because it uses a different calendar", path); + return false; + } + return true; + } + + static void mergeMetadata(Map metadata, Reader reader) { + for (String key : reader.getMetadataKeys()) { + metadata.put(key, reader.getMetadataValue(key)); + } + } + + /** + * Merges multiple ORC files that all have the same schema to produce a single ORC file. The + * merge will reject files that aren't compatible with the merged file so the output list may be + * shorter than the input list. The stripes are copied as serialized byte buffers. The user + * metadata are merged and files that disagree on the value associated with a key will be + * rejected. + * + * @param outputPath the output file + * @param options the options for writing with although the options related to the input files' + * encodings are overridden + * @param inputFiles the list of files to merge + * @return the list of files that were successfully merged + * @throws IOException + */ + public static List mergeFiles( + Path outputPath, WriterOptions options, List inputFiles) throws IOException { + Writer output = null; + final Configuration conf = options.getConfiguration(); + KeyProvider keyProvider = options.getKeyProvider(); + try { + byte[] buffer = new byte[0]; + Reader firstFile = null; + List result = new ArrayList<>(inputFiles.size()); + Map userMetadata = new HashMap<>(); + int bufferSize = 0; + + for (Path input : inputFiles) { + FileSystem fs = input.getFileSystem(conf); + Reader reader = + createReader( + input, + readerOptions(options.getConfiguration()) + .filesystem(fs) + .setKeyProvider(keyProvider)); + + if (!understandFormat(input, reader)) { + continue; + } else if (firstFile == null) { + // if this is the first file that we are including, grab the values + firstFile = reader; + bufferSize = reader.getCompressionSize(); + CompressionKind compression = reader.getCompressionKind(); + options.bufferSize(bufferSize) + .version(reader.getFileVersion()) + .writerVersion(reader.getWriterVersion()) + .compress(compression) + .rowIndexStride(reader.getRowIndexStride()) + .setSchema(reader.getSchema()); + if (compression != CompressionKind.NONE) { + options.enforceBufferSize().bufferSize(bufferSize); + } + mergeMetadata(userMetadata, reader); + // ensure that the merged file uses the same key versions + for (EncryptionKey key : reader.getColumnEncryptionKeys()) { + options.setKeyVersion( + key.getKeyName(), key.getKeyVersion(), key.getAlgorithm()); + } + output = createWriter(outputPath, options); + } else if (!readerIsCompatible(firstFile, userMetadata, input, reader)) { + continue; + } else { + mergeMetadata(userMetadata, reader); + if (bufferSize < reader.getCompressionSize()) { + bufferSize = reader.getCompressionSize(); + ((WriterInternal) output).increaseCompressionSize(bufferSize); + } + } + EncryptionVariant[] variants = reader.getEncryptionVariants(); + List[] completeList = new List[variants.length + 1]; + for (int v = 0; v < variants.length; ++v) { + completeList[v] = reader.getVariantStripeStatistics(variants[v]); + } + completeList[completeList.length - 1] = reader.getVariantStripeStatistics(null); + StripeStatistics[] stripeStats = new StripeStatistics[completeList.length]; + try (FSDataInputStream inputStream = ((ReaderImpl) reader).takeFile()) { + result.add(input); + + for (StripeInformation stripe : reader.getStripes()) { + int length = (int) stripe.getLength(); + if (buffer.length < length) { + buffer = new byte[length]; + } + long offset = stripe.getOffset(); + inputStream.readFully(offset, buffer, 0, length); + int stripeId = (int) stripe.getStripeId(); + for (int v = 0; v < completeList.length; ++v) { + stripeStats[v] = completeList[v].get(stripeId); + } + output.appendStripe(buffer, 0, length, stripe, stripeStats); + } + } + } + if (output != null) { + for (Map.Entry entry : userMetadata.entrySet()) { + output.addUserMetadata(entry.getKey(), entry.getValue()); + } + output.close(); + } + return result; + } catch (Throwable t) { + if (output != null) { + try { + output.close(); + } catch (Throwable ignore) { + // PASS + } + try { + FileSystem fs = + options.getFileSystem() == null + ? outputPath.getFileSystem(conf) + : options.getFileSystem(); + fs.delete(outputPath, false); + } catch (Throwable ignore) { + // PASS + } + } + throw new IOException("Problem merging files into " + outputPath, t); + } + } +} diff --git a/paimon-format/src/main/java/org/apache/orc/impl/PhysicalFsWriter.java b/paimon-format/src/main/java/org/apache/orc/impl/PhysicalFsWriter.java new file mode 100644 index 000000000000..f887e860a830 --- /dev/null +++ b/paimon-format/src/main/java/org/apache/orc/impl/PhysicalFsWriter.java @@ -0,0 +1,793 @@ +/* + * 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.orc.impl; + +import com.google.protobuf.ByteString; +import com.google.protobuf.CodedOutputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.orc.CompressionCodec; +import org.apache.orc.EncryptionVariant; +import org.apache.orc.OrcFile; +import org.apache.orc.OrcProto; +import org.apache.orc.PhysicalWriter; +import org.apache.orc.TypeDescription; +import org.apache.orc.impl.writer.StreamOptions; +import org.apache.orc.impl.writer.WriterEncryptionKey; +import org.apache.orc.impl.writer.WriterEncryptionVariant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +/** A orc PhysicalFsWriter. */ +public class PhysicalFsWriter implements PhysicalWriter { + private static final Logger LOG = LoggerFactory.getLogger(PhysicalFsWriter.class); + + private static final int HDFS_BUFFER_SIZE = 256 * 1024; + + private FSDataOutputStream rawWriter; + private final DirectStream rawStream; + + // the compressed metadata information outStream + private OutStream compressStream; + // a protobuf outStream around streamFactory + private CodedOutputStream codedCompressStream; + + private Path path; + private final HadoopShims shims; + private final long blockSize; + private final int maxPadding; + private final StreamOptions compress; + private final OrcFile.CompressionStrategy compressionStrategy; + private final boolean addBlockPadding; + private final boolean writeVariableLengthBlocks; + private final VariantTracker unencrypted; + + private long headerLength; + private long stripeStart; + // The position of the last time we wrote a short block, which becomes the + // natural blocks + private long blockOffset; + private int metadataLength; + private int stripeStatisticsLength = 0; + private int footerLength; + private int stripeNumber = 0; + + private final Map variants = new TreeMap<>(); + + public PhysicalFsWriter(FileSystem fs, Path path, OrcFile.WriterOptions opts) + throws IOException { + this(fs, path, opts, new WriterEncryptionVariant[0]); + } + + public PhysicalFsWriter( + FileSystem fs, + Path path, + OrcFile.WriterOptions opts, + WriterEncryptionVariant[] encryption) + throws IOException { + this( + fs.create( + path, + opts.getOverwrite(), + HDFS_BUFFER_SIZE, + fs.getDefaultReplication(path), + opts.getBlockSize()), + opts, + encryption); + this.path = path; + LOG.info( + "ORC writer created for path: {} with stripeSize: {} blockSize: {}" + + " compression: {}", + path, + opts.getStripeSize(), + blockSize, + compress); + } + + public PhysicalFsWriter( + FSDataOutputStream outputStream, + OrcFile.WriterOptions opts, + WriterEncryptionVariant[] encryption) + throws IOException { + this.rawWriter = outputStream; + long defaultStripeSize = opts.getStripeSize(); + this.addBlockPadding = opts.getBlockPadding(); + if (opts.isEnforceBufferSize()) { + this.compress = new StreamOptions(opts.getBufferSize()); + } else { + this.compress = + new StreamOptions( + WriterImpl.getEstimatedBufferSize( + defaultStripeSize, + opts.getSchema().getMaximumId() + 1, + opts.getBufferSize())); + } + CompressionCodec codec = OrcCodecPool.getCodec(opts.getCompress()); + if (codec != null) { + CompressionCodec.Options tempOptions = codec.getDefaultOptions(); + if (codec instanceof ZstdCodec + && codec.getDefaultOptions() instanceof ZstdCodec.ZstdOptions) { + ZstdCodec.ZstdOptions options = (ZstdCodec.ZstdOptions) codec.getDefaultOptions(); + OrcFile.ZstdCompressOptions zstdCompressOptions = opts.getZstdCompressOptions(); + if (zstdCompressOptions != null) { + options.setLevel(zstdCompressOptions.getCompressionZstdLevel()); + options.setWindowLog(zstdCompressOptions.getCompressionZstdWindowLog()); + } + } + compress.withCodec(codec, tempOptions); + } + this.compressionStrategy = opts.getCompressionStrategy(); + this.maxPadding = (int) (opts.getPaddingTolerance() * defaultStripeSize); + this.blockSize = opts.getBlockSize(); + blockOffset = 0; + unencrypted = new VariantTracker(opts.getSchema(), compress); + writeVariableLengthBlocks = opts.getWriteVariableLengthBlocks(); + shims = opts.getHadoopShims(); + rawStream = new DirectStream(rawWriter); + compressStream = new OutStream("stripe footer", compress, rawStream); + codedCompressStream = CodedOutputStream.newInstance(compressStream); + for (WriterEncryptionVariant variant : encryption) { + WriterEncryptionKey key = variant.getKeyDescription(); + StreamOptions encryptOptions = + new StreamOptions(unencrypted.options) + .withEncryption(key.getAlgorithm(), variant.getFileFooterKey()); + variants.put(variant, new VariantTracker(variant.getRoot(), encryptOptions)); + } + } + + /** + * Record the information about each column encryption variant. The unencrypted data and each + * encrypted column root are variants. + */ + protected static class VariantTracker { + // the streams that make up the current stripe + protected final Map streams = new TreeMap<>(); + private final int rootColumn; + private final int lastColumn; + protected final StreamOptions options; + // a list for each column covered by this variant + // the elements in the list correspond to each stripe in the file + protected final List[] stripeStats; + protected final List stripeStatsStreams = new ArrayList<>(); + protected final OrcProto.ColumnStatistics[] fileStats; + + VariantTracker(TypeDescription schema, StreamOptions options) { + rootColumn = schema.getId(); + lastColumn = schema.getMaximumId(); + this.options = options; + stripeStats = new List[schema.getMaximumId() - schema.getId() + 1]; + for (int i = 0; i < stripeStats.length; ++i) { + stripeStats[i] = new ArrayList<>(); + } + fileStats = new OrcProto.ColumnStatistics[stripeStats.length]; + } + + public BufferedStream createStream(StreamName name) { + BufferedStream result = new BufferedStream(); + streams.put(name, result); + return result; + } + + /** + * Place the streams in the appropriate area while updating the sizes with the number of + * bytes in the area. + * + * @param area the area to write + * @param sizes the sizes of the areas + * @return the list of stream descriptions to add + */ + public List placeStreams(StreamName.Area area, SizeCounters sizes) { + List result = new ArrayList<>(streams.size()); + for (Map.Entry stream : streams.entrySet()) { + StreamName name = stream.getKey(); + BufferedStream bytes = stream.getValue(); + if (name.getArea() == area && !bytes.isSuppressed) { + OrcProto.Stream.Builder builder = OrcProto.Stream.newBuilder(); + long size = bytes.getOutputSize(); + if (area == StreamName.Area.INDEX) { + sizes.index += size; + } else { + sizes.data += size; + } + builder.setColumn(name.getColumn()).setKind(name.getKind()).setLength(size); + result.add(builder.build()); + } + } + return result; + } + + /** + * Write the streams in the appropriate area. + * + * @param area the area to write + * @param raw the raw stream to write to + */ + public void writeStreams(StreamName.Area area, FSDataOutputStream raw) throws IOException { + for (Map.Entry stream : streams.entrySet()) { + if (stream.getKey().getArea() == area) { + stream.getValue().spillToDiskAndClear(raw); + } + } + } + + /** + * Computed the size of the given column on disk for this stripe. It excludes the index + * streams. + * + * @param column a column id + * @return the total number of bytes + */ + public long getFileBytes(int column) { + long result = 0; + if (column >= rootColumn && column <= lastColumn) { + for (Map.Entry entry : streams.entrySet()) { + StreamName name = entry.getKey(); + if (name.getColumn() == column && name.getArea() != StreamName.Area.INDEX) { + result += entry.getValue().getOutputSize(); + } + } + } + return result; + } + } + + VariantTracker getVariant(EncryptionVariant column) { + if (column == null) { + return unencrypted; + } + return variants.get(column); + } + + /** + * Get the number of bytes for a file in a given column by finding all the streams (not + * suppressed) for a given column and returning the sum of their sizes. excludes index + * + * @param column column from which to get file size + * @return number of bytes for the given column + */ + @Override + public long getFileBytes(int column, WriterEncryptionVariant variant) { + return getVariant(variant).getFileBytes(column); + } + + @Override + public StreamOptions getStreamOptions() { + return unencrypted.options; + } + + private static final byte[] ZEROS = new byte[64 * 1024]; + + private static void writeZeros(OutputStream output, long remaining) throws IOException { + while (remaining > 0) { + long size = Math.min(ZEROS.length, remaining); + output.write(ZEROS, 0, (int) size); + remaining -= size; + } + } + + /** + * Do any required shortening of the HDFS block or padding to avoid stradling HDFS blocks. This + * is called before writing the current stripe. + * + * @param stripeSize the number of bytes in the current stripe + */ + private void padStripe(long stripeSize) throws IOException { + this.stripeStart = rawWriter.getPos(); + long previousBytesInBlock = (stripeStart - blockOffset) % blockSize; + // We only have options if this isn't the first stripe in the block + if (previousBytesInBlock > 0) { + if (previousBytesInBlock + stripeSize >= blockSize) { + // Try making a short block + if (writeVariableLengthBlocks && shims.endVariableLengthBlock(rawWriter)) { + blockOffset = stripeStart; + } else if (addBlockPadding) { + // if we cross the block boundary, figure out what we should do + long padding = blockSize - previousBytesInBlock; + if (padding <= maxPadding) { + writeZeros(rawWriter, padding); + stripeStart += padding; + } + } + } + } + } + + /** An output receiver that writes the ByteBuffers to the output stream as they are received. */ + private static class DirectStream implements OutputReceiver { + private final FSDataOutputStream output; + + DirectStream(FSDataOutputStream output) { + this.output = output; + } + + @Override + public void output(ByteBuffer buffer) throws IOException { + output.write( + buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining()); + } + + @Override + public void suppress() { + throw new UnsupportedOperationException("Can't suppress direct stream"); + } + } + + private void writeStripeFooter( + OrcProto.StripeFooter footer, + SizeCounters sizes, + OrcProto.StripeInformation.Builder dirEntry) + throws IOException { + footer.writeTo(codedCompressStream); + codedCompressStream.flush(); + compressStream.flush(); + dirEntry.setOffset(stripeStart); + dirEntry.setFooterLength(rawWriter.getPos() - stripeStart - sizes.total()); + } + + /** + * Write the saved encrypted stripe statistic in a variant out to the file. The streams that are + * written are added to the tracker.stripeStatsStreams. + * + * @param output the file we are writing to + * @param stripeNumber the number of stripes in the file + * @param tracker the variant to write out + */ + static void writeEncryptedStripeStatistics( + DirectStream output, int stripeNumber, VariantTracker tracker) throws IOException { + StreamOptions options = new StreamOptions(tracker.options); + tracker.stripeStatsStreams.clear(); + for (int col = tracker.rootColumn; + col < tracker.rootColumn + tracker.stripeStats.length; + ++col) { + options.modifyIv( + CryptoUtils.modifyIvForStream( + col, OrcProto.Stream.Kind.STRIPE_STATISTICS, stripeNumber + 1)); + OutStream stream = new OutStream("stripe stats for " + col, options, output); + OrcProto.ColumnarStripeStatistics stats = + OrcProto.ColumnarStripeStatistics.newBuilder() + .addAllColStats(tracker.stripeStats[col - tracker.rootColumn]) + .build(); + long start = output.output.getPos(); + stats.writeTo(stream); + stream.flush(); + OrcProto.Stream description = + OrcProto.Stream.newBuilder() + .setColumn(col) + .setKind(OrcProto.Stream.Kind.STRIPE_STATISTICS) + .setLength(output.output.getPos() - start) + .build(); + tracker.stripeStatsStreams.add(description); + } + } + + /** + * Merge the saved unencrypted stripe statistics into the Metadata section of the footer. + * + * @param builder the Metadata section of the file + * @param stripeCount the number of stripes in the file + * @param stats the stripe statistics + */ + static void setUnencryptedStripeStatistics( + OrcProto.Metadata.Builder builder, + int stripeCount, + List[] stats) { + // Make the unencrypted stripe stats into lists of StripeStatistics. + builder.clearStripeStats(); + for (int s = 0; s < stripeCount; ++s) { + OrcProto.StripeStatistics.Builder stripeStats = OrcProto.StripeStatistics.newBuilder(); + for (List col : stats) { + stripeStats.addColStats(col.get(s)); + } + builder.addStripeStats(stripeStats.build()); + } + } + + static void setEncryptionStatistics( + OrcProto.Encryption.Builder encryption, + int stripeNumber, + Collection variants) + throws IOException { + int v = 0; + for (VariantTracker variant : variants) { + OrcProto.EncryptionVariant.Builder variantBuilder = encryption.getVariantsBuilder(v++); + + // Add the stripe statistics streams to the variant description. + variantBuilder.clearStripeStatistics(); + variantBuilder.addAllStripeStatistics(variant.stripeStatsStreams); + + // Serialize and encrypt the file statistics. + OrcProto.FileStatistics.Builder file = OrcProto.FileStatistics.newBuilder(); + for (OrcProto.ColumnStatistics col : variant.fileStats) { + file.addColumn(col); + } + StreamOptions options = new StreamOptions(variant.options); + options.modifyIv( + CryptoUtils.modifyIvForStream( + variant.rootColumn, + OrcProto.Stream.Kind.FILE_STATISTICS, + stripeNumber + 1)); + BufferedStream buffer = new BufferedStream(); + OutStream stream = new OutStream("stats for " + variant, options, buffer); + file.build().writeTo(stream); + stream.flush(); + variantBuilder.setFileStatistics(buffer.getBytes()); + } + } + + @Override + public void writeFileMetadata(OrcProto.Metadata.Builder builder) throws IOException { + long stripeStatisticsStart = rawWriter.getPos(); + for (VariantTracker variant : variants.values()) { + writeEncryptedStripeStatistics(rawStream, stripeNumber, variant); + } + setUnencryptedStripeStatistics(builder, stripeNumber, unencrypted.stripeStats); + long metadataStart = rawWriter.getPos(); + builder.build().writeTo(codedCompressStream); + codedCompressStream.flush(); + compressStream.flush(); + this.stripeStatisticsLength = (int) (metadataStart - stripeStatisticsStart); + this.metadataLength = (int) (rawWriter.getPos() - metadataStart); + } + + static void addUnencryptedStatistics( + OrcProto.Footer.Builder builder, OrcProto.ColumnStatistics[] stats) { + for (OrcProto.ColumnStatistics stat : stats) { + builder.addStatistics(stat); + } + } + + @Override + public void writeFileFooter(OrcProto.Footer.Builder builder) throws IOException { + if (variants.size() > 0) { + OrcProto.Encryption.Builder encryption = builder.getEncryptionBuilder(); + setEncryptionStatistics(encryption, stripeNumber, variants.values()); + } + addUnencryptedStatistics(builder, unencrypted.fileStats); + long bodyLength = rawWriter.getPos() - metadataLength - stripeStatisticsLength; + builder.setContentLength(bodyLength); + builder.setHeaderLength(headerLength); + long startPosn = rawWriter.getPos(); + OrcProto.Footer footer = builder.build(); + footer.writeTo(codedCompressStream); + codedCompressStream.flush(); + compressStream.flush(); + this.footerLength = (int) (rawWriter.getPos() - startPosn); + } + + @Override + public long writePostScript(OrcProto.PostScript.Builder builder) throws IOException { + builder.setFooterLength(footerLength); + builder.setMetadataLength(metadataLength); + if (variants.size() > 0) { + builder.setStripeStatisticsLength(stripeStatisticsLength); + } + OrcProto.PostScript ps = builder.build(); + // need to write this uncompressed + long startPosn = rawWriter.getPos(); + ps.writeTo(rawWriter); + long length = rawWriter.getPos() - startPosn; + if (length > 255) { + throw new IllegalArgumentException("PostScript too large at " + length); + } + rawWriter.writeByte((int) length); + return rawWriter.getPos(); + } + + @Override + public void close() throws IOException { + // We don't use the codec directly but do give it out codec in getCompressionCodec; + // that is used in tests, for boolean checks, and in StreamFactory. Some of the changes that + // would get rid of this pattern require cross-project interface changes, so just return the + // codec for now. + CompressionCodec codec = compress.getCodec(); + if (codec != null) { + OrcCodecPool.returnCodec(codec.getKind(), codec); + } + compress.withCodec(null, null); + rawWriter.close(); + rawWriter = null; + } + + @Override + public void flush() throws IOException { + rawWriter.hflush(); + } + + @Override + public void appendRawStripe(ByteBuffer buffer, OrcProto.StripeInformation.Builder dirEntry) + throws IOException { + long start = rawWriter.getPos(); + int length = buffer.remaining(); + long availBlockSpace = blockSize - (start % blockSize); + + // see if stripe can fit in the current hdfs block, else pad the remaining + // space in the block + if (length < blockSize && length > availBlockSpace && addBlockPadding) { + byte[] pad = new byte[(int) Math.min(HDFS_BUFFER_SIZE, availBlockSpace)]; + LOG.info("Padding ORC by {} bytes while merging", availBlockSpace); + start += availBlockSpace; + while (availBlockSpace > 0) { + int writeLen = (int) Math.min(availBlockSpace, pad.length); + rawWriter.write(pad, 0, writeLen); + availBlockSpace -= writeLen; + } + } + rawWriter.write(buffer.array(), buffer.arrayOffset() + buffer.position(), length); + dirEntry.setOffset(start); + stripeNumber += 1; + } + + /** + * This class is used to hold the contents of streams as they are buffered. The TreeWriters + * write to the outStream and the codec compresses the data as buffers fill up and stores them + * in the output list. When the stripe is being written, the whole stream is written to the + * file. + */ + static final class BufferedStream implements OutputReceiver { + private boolean isSuppressed = false; + private final List output = new ArrayList<>(); + + @Override + public void output(ByteBuffer buffer) { + if (!isSuppressed) { + output.add(buffer); + } + } + + @Override + public void suppress() { + isSuppressed = true; + output.clear(); + } + + /** + * Write any saved buffers to the OutputStream if needed, and clears all the buffers. + * + * @return true if the stream was written + */ + boolean spillToDiskAndClear(FSDataOutputStream raw) throws IOException { + if (!isSuppressed) { + for (ByteBuffer buffer : output) { + raw.write( + buffer.array(), + buffer.arrayOffset() + buffer.position(), + buffer.remaining()); + } + output.clear(); + return true; + } + isSuppressed = false; + return false; + } + + /** + * Get the buffer as a protobuf ByteString and clears the BufferedStream. + * + * @return the bytes + */ + ByteString getBytes() { + int len = output.size(); + if (len == 0) { + return ByteString.EMPTY; + } else { + ByteString result = ByteString.copyFrom(output.get(0)); + for (int i = 1; i < output.size(); ++i) { + result = result.concat(ByteString.copyFrom(output.get(i))); + } + output.clear(); + return result; + } + } + + /** + * Get the stream as a ByteBuffer and clear it. + * + * @return a single ByteBuffer with the contents of the stream + */ + ByteBuffer getByteBuffer() { + ByteBuffer result; + if (output.size() == 1) { + result = output.get(0); + } else { + result = ByteBuffer.allocate((int) getOutputSize()); + for (ByteBuffer buffer : output) { + result.put(buffer); + } + output.clear(); + result.flip(); + } + return result; + } + + /** + * Get the number of bytes that will be written to the output. + * + *

Assumes the stream writing into this receiver has already been flushed. + * + * @return number of bytes + */ + public long getOutputSize() { + long result = 0; + for (ByteBuffer buffer : output) { + result += buffer.remaining(); + } + return result; + } + } + + static class SizeCounters { + long index = 0; + long data = 0; + + long total() { + return index + data; + } + } + + void buildStreamList(OrcProto.StripeFooter.Builder footerBuilder, SizeCounters sizes) + throws IOException { + footerBuilder.addAllStreams(unencrypted.placeStreams(StreamName.Area.INDEX, sizes)); + final long unencryptedIndexSize = sizes.index; + int v = 0; + for (VariantTracker variant : variants.values()) { + OrcProto.StripeEncryptionVariant.Builder builder = + footerBuilder.getEncryptionBuilder(v++); + builder.addAllStreams(variant.placeStreams(StreamName.Area.INDEX, sizes)); + } + if (sizes.index != unencryptedIndexSize) { + // add a placeholder that covers the hole where the encrypted indexes are + footerBuilder.addStreams( + OrcProto.Stream.newBuilder() + .setKind(OrcProto.Stream.Kind.ENCRYPTED_INDEX) + .setLength(sizes.index - unencryptedIndexSize)); + } + footerBuilder.addAllStreams(unencrypted.placeStreams(StreamName.Area.DATA, sizes)); + final long unencryptedDataSize = sizes.data; + v = 0; + for (VariantTracker variant : variants.values()) { + OrcProto.StripeEncryptionVariant.Builder builder = + footerBuilder.getEncryptionBuilder(v++); + builder.addAllStreams(variant.placeStreams(StreamName.Area.DATA, sizes)); + } + if (sizes.data != unencryptedDataSize) { + // add a placeholder that covers the hole where the encrypted indexes are + footerBuilder.addStreams( + OrcProto.Stream.newBuilder() + .setKind(OrcProto.Stream.Kind.ENCRYPTED_DATA) + .setLength(sizes.data - unencryptedDataSize)); + } + } + + @Override + public void finalizeStripe( + OrcProto.StripeFooter.Builder footerBuilder, + OrcProto.StripeInformation.Builder dirEntry) + throws IOException { + SizeCounters sizes = new SizeCounters(); + buildStreamList(footerBuilder, sizes); + + OrcProto.StripeFooter footer = footerBuilder.build(); + + // Do we need to pad the file so the stripe doesn't straddle a block boundary? + padStripe(sizes.total() + footer.getSerializedSize()); + + // write the unencrypted index streams + unencrypted.writeStreams(StreamName.Area.INDEX, rawWriter); + // write the encrypted index streams + for (VariantTracker variant : variants.values()) { + variant.writeStreams(StreamName.Area.INDEX, rawWriter); + } + + // write the unencrypted data streams + unencrypted.writeStreams(StreamName.Area.DATA, rawWriter); + // write out the unencrypted data streams + for (VariantTracker variant : variants.values()) { + variant.writeStreams(StreamName.Area.DATA, rawWriter); + } + + // Write out the footer. + writeStripeFooter(footer, sizes, dirEntry); + + // fill in the data sizes + dirEntry.setDataLength(sizes.data); + dirEntry.setIndexLength(sizes.index); + + stripeNumber += 1; + } + + @Override + public void writeHeader() throws IOException { + rawWriter.writeBytes(OrcFile.MAGIC); + headerLength = rawWriter.getPos(); + } + + @Override + public BufferedStream createDataStream(StreamName name) { + VariantTracker variant = getVariant(name.getEncryption()); + BufferedStream result = variant.streams.get(name); + if (result == null) { + result = new BufferedStream(); + variant.streams.put(name, result); + } + return result; + } + + private StreamOptions getOptions(OrcProto.Stream.Kind kind) { + return SerializationUtils.getCustomizedCodec(compress, compressionStrategy, kind); + } + + protected OutputStream createIndexStream(StreamName name) { + BufferedStream buffer = createDataStream(name); + VariantTracker tracker = getVariant(name.getEncryption()); + StreamOptions options = + SerializationUtils.getCustomizedCodec( + tracker.options, compressionStrategy, name.getKind()); + if (options.isEncrypted()) { + if (options == tracker.options) { + options = new StreamOptions(options); + } + options.modifyIv(CryptoUtils.modifyIvForStream(name, stripeNumber + 1)); + } + return new OutStream(name.toString(), options, buffer); + } + + @Override + public void writeIndex(StreamName name, OrcProto.RowIndex.Builder index) throws IOException { + OutputStream stream = createIndexStream(name); + index.build().writeTo(stream); + stream.flush(); + } + + @Override + public void writeBloomFilter(StreamName name, OrcProto.BloomFilterIndex.Builder bloom) + throws IOException { + OutputStream stream = createIndexStream(name); + bloom.build().writeTo(stream); + stream.flush(); + } + + @Override + public void writeStatistics(StreamName name, OrcProto.ColumnStatistics.Builder statistics) { + VariantTracker tracker = getVariant(name.getEncryption()); + if (name.getKind() == OrcProto.Stream.Kind.FILE_STATISTICS) { + tracker.fileStats[name.getColumn() - tracker.rootColumn] = statistics.build(); + } else { + tracker.stripeStats[name.getColumn() - tracker.rootColumn].add(statistics.build()); + } + } + + @Override + public String toString() { + if (path != null) { + return path.toString(); + } else { + return ByteString.EMPTY.toString(); + } + } +} diff --git a/paimon-format/src/main/java/org/apache/orc/impl/WriterImpl.java b/paimon-format/src/main/java/org/apache/orc/impl/WriterImpl.java new file mode 100644 index 000000000000..bbbc8f03172d --- /dev/null +++ b/paimon-format/src/main/java/org/apache/orc/impl/WriterImpl.java @@ -0,0 +1,1063 @@ +/* + * 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.orc.impl; + +import com.github.luben.zstd.util.Native; +import com.google.protobuf.ByteString; +import io.airlift.compress.lz4.Lz4Compressor; +import io.airlift.compress.lz4.Lz4Decompressor; +import io.airlift.compress.lzo.LzoCompressor; +import io.airlift.compress.lzo.LzoDecompressor; +import io.airlift.compress.zstd.ZstdCompressor; +import io.airlift.compress.zstd.ZstdDecompressor; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.orc.ColumnStatistics; +import org.apache.orc.CompressionCodec; +import org.apache.orc.CompressionKind; +import org.apache.orc.DataMask; +import org.apache.orc.MemoryManager; +import org.apache.orc.OrcConf; +import org.apache.orc.OrcFile; +import org.apache.orc.OrcProto; +import org.apache.orc.OrcUtils; +import org.apache.orc.PhysicalWriter; +import org.apache.orc.StripeInformation; +import org.apache.orc.StripeStatistics; +import org.apache.orc.TypeDescription; +import org.apache.orc.impl.writer.StreamOptions; +import org.apache.orc.impl.writer.TreeWriter; +import org.apache.orc.impl.writer.WriterContext; +import org.apache.orc.impl.writer.WriterEncryptionKey; +import org.apache.orc.impl.writer.WriterEncryptionVariant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.security.SecureRandom; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TimeZone; +import java.util.TreeMap; + +/** + * An ORC file writer. The file is divided into stripes, which is the natural unit of work when + * reading. Each stripe is buffered in memory until the memory reaches the stripe size and then it + * is written out broken down by columns. Each column is written by a TreeWriter that is specific to + * that type of column. TreeWriters may have children TreeWriters that handle the sub-types. Each of + * the TreeWriters writes the column's data as a set of streams. + * + *

This class is unsynchronized like most Stream objects, so from the creation of an OrcFile and + * all access to a single instance has to be from a single thread. + * + *

There are no known cases where these happen between different threads today. + * + *

Caveat: the MemoryManager is created during WriterOptions create, that has to be confined to a + * single thread as well. + */ +public class WriterImpl implements WriterInternal, MemoryManager.Callback { + + private static final Logger LOG = LoggerFactory.getLogger(WriterImpl.class); + + private static final int MIN_ROW_INDEX_STRIDE = 1000; + + private final Path path; + private final long stripeSize; + private final long stripeRowCount; + private final int rowIndexStride; + private final TypeDescription schema; + private final PhysicalWriter physicalWriter; + private final OrcFile.WriterVersion writerVersion; + private final StreamOptions unencryptedOptions; + + private long rowCount = 0; + private long rowsInStripe = 0; + private long rawDataSize = 0; + private int rowsInIndex = 0; + private long lastFlushOffset = 0; + private int stripesAtLastFlush = -1; + private final List stripes = new ArrayList<>(); + private final Map userMetadata = new TreeMap<>(); + private final TreeWriter treeWriter; + private final boolean buildIndex; + private final MemoryManager memoryManager; + private long previousAllocation = -1; + private long memoryLimit; + private final long rowsPerCheck; + private long rowsSinceCheck = 0; + private final OrcFile.Version version; + private final Configuration conf; + private final OrcFile.WriterCallback callback; + private final OrcFile.WriterContext callbackContext; + private final OrcFile.EncodingStrategy encodingStrategy; + private final OrcFile.CompressionStrategy compressionStrategy; + private final boolean[] bloomFilterColumns; + private final double bloomFilterFpp; + private final OrcFile.BloomFilterVersion bloomFilterVersion; + private final boolean writeTimeZone; + private final boolean useUTCTimeZone; + private final double dictionaryKeySizeThreshold; + private final boolean[] directEncodingColumns; + private final List unencryptedEncodings = new ArrayList<>(); + + // the list of maskDescriptions, keys, and variants + private SortedMap maskDescriptions = new TreeMap<>(); + private SortedMap keys = new TreeMap<>(); + private final WriterEncryptionVariant[] encryption; + // the mapping of columns to maskDescriptions + private final MaskDescriptionImpl[] columnMaskDescriptions; + // the mapping of columns to EncryptionVariants + private final WriterEncryptionVariant[] columnEncryption; + private KeyProvider keyProvider; + // do we need to include the current encryption keys in the next stripe + // information + private boolean needKeyFlush; + private final boolean useProlepticGregorian; + private boolean isClose = false; + + public WriterImpl(FileSystem fs, Path path, OrcFile.WriterOptions opts) throws IOException { + this.path = path; + this.conf = opts.getConfiguration(); + // clone it so that we can annotate it with encryption + this.schema = opts.getSchema().clone(); + int numColumns = schema.getMaximumId() + 1; + if (!opts.isEnforceBufferSize()) { + opts.bufferSize( + getEstimatedBufferSize(opts.getStripeSize(), numColumns, opts.getBufferSize())); + } + + // Annotate the schema with the column encryption + schema.annotateEncryption(opts.getEncryption(), opts.getMasks()); + columnEncryption = new WriterEncryptionVariant[numColumns]; + columnMaskDescriptions = new MaskDescriptionImpl[numColumns]; + encryption = setupEncryption(opts.getKeyProvider(), schema, opts.getKeyOverrides()); + needKeyFlush = encryption.length > 0; + + this.directEncodingColumns = + OrcUtils.includeColumns(opts.getDirectEncodingColumns(), opts.getSchema()); + dictionaryKeySizeThreshold = OrcConf.DICTIONARY_KEY_SIZE_THRESHOLD.getDouble(conf); + + this.callback = opts.getCallback(); + if (callback != null) { + callbackContext = () -> WriterImpl.this; + } else { + callbackContext = null; + } + + this.useProlepticGregorian = opts.getProlepticGregorian(); + this.writeTimeZone = hasTimestamp(schema); + this.useUTCTimeZone = opts.getUseUTCTimestamp(); + + this.encodingStrategy = opts.getEncodingStrategy(); + this.compressionStrategy = opts.getCompressionStrategy(); + + if (opts.getRowIndexStride() >= 0) { + this.rowIndexStride = opts.getRowIndexStride(); + } else { + this.rowIndexStride = 0; + } + + // ORC-1343: We ignore `opts.isBuildIndex` due to the lack of reader support + this.buildIndex = rowIndexStride > 0; + if (buildIndex && rowIndexStride < MIN_ROW_INDEX_STRIDE) { + throw new IllegalArgumentException( + "Row stride must be at least " + MIN_ROW_INDEX_STRIDE); + } + + this.writerVersion = opts.getWriterVersion(); + this.version = opts.getVersion(); + if (version == OrcFile.Version.FUTURE) { + throw new IllegalArgumentException("Can not write in a unknown version."); + } else if (version == OrcFile.Version.UNSTABLE_PRE_2_0) { + LOG.warn( + "ORC files written in " + + version.getName() + + " will not be" + + " readable by other versions of the software. It is only for" + + " developer testing."); + } + + this.bloomFilterVersion = opts.getBloomFilterVersion(); + this.bloomFilterFpp = opts.getBloomFilterFpp(); + /* do not write bloom filters for ORC v11 */ + if (!buildIndex || version == OrcFile.Version.V_0_11) { + this.bloomFilterColumns = new boolean[schema.getMaximumId() + 1]; + } else { + this.bloomFilterColumns = OrcUtils.includeColumns(opts.getBloomFilterColumns(), schema); + } + + // ensure that we are able to handle callbacks before we register ourselves + rowsPerCheck = + Math.min(opts.getStripeRowCountValue(), OrcConf.ROWS_BETWEEN_CHECKS.getLong(conf)); + this.stripeRowCount = opts.getStripeRowCountValue(); + this.stripeSize = opts.getStripeSize(); + memoryLimit = stripeSize; + memoryManager = opts.getMemoryManager(); + memoryManager.addWriter(path, stripeSize, this); + + // Set up the physical writer + this.physicalWriter = + opts.getPhysicalWriter() == null + ? new PhysicalFsWriter(fs, path, opts, encryption) + : opts.getPhysicalWriter(); + physicalWriter.writeHeader(); + unencryptedOptions = physicalWriter.getStreamOptions(); + OutStream.assertBufferSizeValid(unencryptedOptions.getBufferSize()); + + treeWriter = TreeWriter.Factory.create(schema, null, new StreamFactory()); + + LOG.info( + "ORC writer created for path: {} with stripeSize: {} options: {}", + path, + stripeSize, + unencryptedOptions); + } + + // @VisibleForTesting + public static int getEstimatedBufferSize(long stripeSize, int numColumns, int bs) { + // The worst case is that there are 2 big streams per a column and + // we want to guarantee that each stream gets ~10 buffers. + // This keeps buffers small enough that we don't get really small stripe + // sizes. + int estBufferSize = (int) (stripeSize / (20L * numColumns)); + estBufferSize = getClosestBufferSize(estBufferSize); + return Math.min(estBufferSize, bs); + } + + @Override + public void increaseCompressionSize(int newSize) { + if (newSize > unencryptedOptions.getBufferSize()) { + unencryptedOptions.bufferSize(newSize); + } + } + + /** + * Given a buffer size, return the nearest superior power of 2. Min value is 4Kib, Max value is + * 256Kib. + * + * @param size Proposed buffer size + * @return the suggested buffer size + */ + private static int getClosestBufferSize(int size) { + final int kb4 = 4 * 1024; + final int kb256 = 256 * 1024; + final int pow2 = size == 1 ? 1 : Integer.highestOneBit(size - 1) * 2; + return Math.min(kb256, Math.max(kb4, pow2)); + } + + static { + try { + if (!"java".equalsIgnoreCase(System.getProperty("orc.compression.zstd.impl"))) { + Native.load(); + } + } catch (UnsatisfiedLinkError | ExceptionInInitializerError e) { + LOG.warn( + "Unable to load zstd-jni library for your platform. " + + "Using builtin-java classes where applicable"); + } + } + + public static CompressionCodec createCodec(CompressionKind kind) { + switch (kind) { + case NONE: + return null; + case ZLIB: + return new ZlibCodec(); + case SNAPPY: + return new SnappyCodec(); + case LZO: + return new AircompressorCodec(kind, new LzoCompressor(), new LzoDecompressor()); + case LZ4: + return new AircompressorCodec(kind, new Lz4Compressor(), new Lz4Decompressor()); + case ZSTD: + if ("java".equalsIgnoreCase(System.getProperty("orc.compression.zstd.impl"))) { + return new AircompressorCodec( + kind, new ZstdCompressor(), new ZstdDecompressor()); + } + if (Native.isLoaded()) { + return new ZstdCodec(); + } else { + return new AircompressorCodec( + kind, new ZstdCompressor(), new ZstdDecompressor()); + } + default: + throw new IllegalArgumentException("Unknown compression codec: " + kind); + } + } + + @Override + public boolean checkMemory(double newScale) throws IOException { + memoryLimit = Math.round(stripeSize * newScale); + return checkMemory(); + } + + private boolean checkMemory() throws IOException { + if (rowsSinceCheck >= rowsPerCheck) { + rowsSinceCheck = 0; + long size = treeWriter.estimateMemory(); + if (LOG.isDebugEnabled()) { + LOG.debug( + "ORC writer " + + physicalWriter + + " size = " + + size + + " memoryLimit = " + + memoryLimit + + " rowsInStripe = " + + rowsInStripe + + " stripeRowCountLimit = " + + stripeRowCount); + } + if (size > memoryLimit || rowsInStripe >= stripeRowCount) { + flushStripe(); + return true; + } + } + return false; + } + + /** + * Interface from the Writer to the TreeWriters. This limits the visibility that the TreeWriters + * have into the Writer. + */ + private class StreamFactory implements WriterContext { + + /** + * Create a stream to store part of a column. + * + * @param name the name for the stream + * @return The output outStream that the section needs to be written to. + */ + @Override + public OutStream createStream(StreamName name) throws IOException { + StreamOptions options = + SerializationUtils.getCustomizedCodec( + unencryptedOptions, compressionStrategy, name.getKind()); + WriterEncryptionVariant encryption = (WriterEncryptionVariant) name.getEncryption(); + if (encryption != null) { + if (options == unencryptedOptions) { + options = new StreamOptions(options); + } + options.withEncryption( + encryption.getKeyDescription().getAlgorithm(), + encryption.getFileFooterKey()) + .modifyIv(CryptoUtils.modifyIvForStream(name, 1)); + } + return new OutStream(name, options, physicalWriter.createDataStream(name)); + } + + /** Get the stride rate of the row index. */ + @Override + public int getRowIndexStride() { + return rowIndexStride; + } + + /** + * Should be building the row index. + * + * @return true if we are building the index + */ + @Override + public boolean buildIndex() { + return buildIndex; + } + + /** + * Is the ORC file compressed? + * + * @return are the streams compressed + */ + @Override + public boolean isCompressed() { + return unencryptedOptions.getCodec() != null; + } + + /** + * Get the encoding strategy to use. + * + * @return encoding strategy + */ + @Override + public OrcFile.EncodingStrategy getEncodingStrategy() { + return encodingStrategy; + } + + /** + * Get the bloom filter columns. + * + * @return bloom filter columns + */ + @Override + public boolean[] getBloomFilterColumns() { + return bloomFilterColumns; + } + + /** + * Get bloom filter false positive percentage. + * + * @return fpp + */ + @Override + public double getBloomFilterFPP() { + return bloomFilterFpp; + } + + /** + * Get the writer's configuration. + * + * @return configuration + */ + @Override + public Configuration getConfiguration() { + return conf; + } + + /** Get the version of the file to write. */ + @Override + public OrcFile.Version getVersion() { + return version; + } + + /** + * Get the PhysicalWriter. + * + * @return the file's physical writer. + */ + @Override + public PhysicalWriter getPhysicalWriter() { + return physicalWriter; + } + + @Override + public OrcFile.BloomFilterVersion getBloomFilterVersion() { + return bloomFilterVersion; + } + + @Override + public void writeIndex(StreamName name, OrcProto.RowIndex.Builder index) + throws IOException { + physicalWriter.writeIndex(name, index); + } + + @Override + public void writeBloomFilter(StreamName name, OrcProto.BloomFilterIndex.Builder bloom) + throws IOException { + physicalWriter.writeBloomFilter(name, bloom); + } + + @Override + public WriterEncryptionVariant getEncryption(int columnId) { + return columnId < columnEncryption.length ? columnEncryption[columnId] : null; + } + + @Override + public DataMask getUnencryptedMask(int columnId) { + if (columnMaskDescriptions != null) { + MaskDescriptionImpl descr = columnMaskDescriptions[columnId]; + if (descr != null) { + return DataMask.Factory.build( + descr, + schema.findSubtype(columnId), + (type) -> columnMaskDescriptions[type.getId()]); + } + } + return null; + } + + @Override + public void setEncoding( + int column, WriterEncryptionVariant encryption, OrcProto.ColumnEncoding encoding) { + if (encryption == null) { + unencryptedEncodings.add(encoding); + } else { + encryption.addEncoding(encoding); + } + } + + @Override + public void writeStatistics(StreamName name, OrcProto.ColumnStatistics.Builder stats) + throws IOException { + physicalWriter.writeStatistics(name, stats); + } + + @Override + public boolean getUseUTCTimestamp() { + return useUTCTimeZone; + } + + @Override + public double getDictionaryKeySizeThreshold(int columnId) { + return directEncodingColumns[columnId] ? 0.0 : dictionaryKeySizeThreshold; + } + + @Override + public boolean getProlepticGregorian() { + return useProlepticGregorian; + } + } + + private static void writeTypes(OrcProto.Footer.Builder builder, TypeDescription schema) { + builder.addAllTypes(OrcUtils.getOrcTypes(schema)); + } + + private void createRowIndexEntry() throws IOException { + treeWriter.createRowIndexEntry(); + rowsInIndex = 0; + } + + /** + * Write the encrypted keys into the StripeInformation along with the stripe id, so that the + * readers can decrypt the data. + * + * @param dirEntry the entry to modify + */ + private void addEncryptedKeys(OrcProto.StripeInformation.Builder dirEntry) { + for (WriterEncryptionVariant variant : encryption) { + dirEntry.addEncryptedLocalKeys( + ByteString.copyFrom(variant.getMaterial().getEncryptedKey())); + } + dirEntry.setEncryptStripeId(1 + stripes.size()); + } + + private void flushStripe() throws IOException { + if (buildIndex && rowsInIndex != 0) { + createRowIndexEntry(); + } + if (rowsInStripe != 0) { + if (callback != null) { + callback.preStripeWrite(callbackContext); + } + // finalize the data for the stripe + int requiredIndexEntries = + rowIndexStride == 0 + ? 0 + : (int) ((rowsInStripe + rowIndexStride - 1) / rowIndexStride); + OrcProto.StripeFooter.Builder builder = OrcProto.StripeFooter.newBuilder(); + if (writeTimeZone) { + if (useUTCTimeZone) { + builder.setWriterTimezone("UTC"); + } else { + builder.setWriterTimezone(TimeZone.getDefault().getID()); + } + } + treeWriter.flushStreams(); + treeWriter.writeStripe(requiredIndexEntries); + // update the encodings + builder.addAllColumns(unencryptedEncodings); + unencryptedEncodings.clear(); + for (WriterEncryptionVariant writerEncryptionVariant : encryption) { + OrcProto.StripeEncryptionVariant.Builder encrypt = + OrcProto.StripeEncryptionVariant.newBuilder(); + encrypt.addAllEncoding(writerEncryptionVariant.getEncodings()); + writerEncryptionVariant.clearEncodings(); + builder.addEncryption(encrypt); + } + OrcProto.StripeInformation.Builder dirEntry = + OrcProto.StripeInformation.newBuilder().setNumberOfRows(rowsInStripe); + if (encryption.length > 0 && needKeyFlush) { + addEncryptedKeys(dirEntry); + needKeyFlush = false; + } + physicalWriter.finalizeStripe(builder, dirEntry); + + stripes.add(dirEntry.build()); + rowCount += rowsInStripe; + rowsInStripe = 0; + } + } + + private long computeRawDataSize() { + return treeWriter.getRawDataSize(); + } + + private OrcProto.CompressionKind writeCompressionKind(CompressionKind kind) { + switch (kind) { + case NONE: + return OrcProto.CompressionKind.NONE; + case ZLIB: + return OrcProto.CompressionKind.ZLIB; + case SNAPPY: + return OrcProto.CompressionKind.SNAPPY; + case LZO: + return OrcProto.CompressionKind.LZO; + case LZ4: + return OrcProto.CompressionKind.LZ4; + case ZSTD: + return OrcProto.CompressionKind.ZSTD; + default: + throw new IllegalArgumentException("Unknown compression " + kind); + } + } + + private void writeMetadata() throws IOException { + // The physical writer now has the stripe statistics, so we pass a + // new builder in here. + physicalWriter.writeFileMetadata(OrcProto.Metadata.newBuilder()); + } + + private long writePostScript() throws IOException { + OrcProto.PostScript.Builder builder = + OrcProto.PostScript.newBuilder() + .setMagic(OrcFile.MAGIC) + .addVersion(version.getMajor()) + .addVersion(version.getMinor()) + .setWriterVersion(writerVersion.getId()); + CompressionCodec codec = unencryptedOptions.getCodec(); + if (codec == null) { + builder.setCompression(OrcProto.CompressionKind.NONE); + } else { + builder.setCompression(writeCompressionKind(codec.getKind())) + .setCompressionBlockSize(unencryptedOptions.getBufferSize()); + } + return physicalWriter.writePostScript(builder); + } + + private OrcProto.EncryptionKey.Builder writeEncryptionKey(WriterEncryptionKey key) { + OrcProto.EncryptionKey.Builder result = OrcProto.EncryptionKey.newBuilder(); + HadoopShims.KeyMetadata meta = key.getMetadata(); + result.setKeyName(meta.getKeyName()); + result.setKeyVersion(meta.getVersion()); + result.setAlgorithm( + OrcProto.EncryptionAlgorithm.valueOf(meta.getAlgorithm().getSerialization())); + return result; + } + + private OrcProto.EncryptionVariant.Builder writeEncryptionVariant( + WriterEncryptionVariant variant) { + OrcProto.EncryptionVariant.Builder result = OrcProto.EncryptionVariant.newBuilder(); + result.setRoot(variant.getRoot().getId()); + result.setKey(variant.getKeyDescription().getId()); + result.setEncryptedKey(ByteString.copyFrom(variant.getMaterial().getEncryptedKey())); + return result; + } + + private OrcProto.Encryption.Builder writeEncryptionFooter() { + OrcProto.Encryption.Builder encrypt = OrcProto.Encryption.newBuilder(); + for (MaskDescriptionImpl mask : maskDescriptions.values()) { + OrcProto.DataMask.Builder maskBuilder = OrcProto.DataMask.newBuilder(); + maskBuilder.setName(mask.getName()); + for (String param : mask.getParameters()) { + maskBuilder.addMaskParameters(param); + } + for (TypeDescription column : mask.getColumns()) { + maskBuilder.addColumns(column.getId()); + } + encrypt.addMask(maskBuilder); + } + for (WriterEncryptionKey key : keys.values()) { + encrypt.addKey(writeEncryptionKey(key)); + } + for (WriterEncryptionVariant variant : encryption) { + encrypt.addVariants(writeEncryptionVariant(variant)); + } + encrypt.setKeyProvider(OrcProto.KeyProviderKind.valueOf(keyProvider.getKind().getValue())); + return encrypt; + } + + private long writeFooter() throws IOException { + writeMetadata(); + OrcProto.Footer.Builder builder = OrcProto.Footer.newBuilder(); + builder.setNumberOfRows(rowCount); + builder.setRowIndexStride(rowIndexStride); + rawDataSize = computeRawDataSize(); + // serialize the types + writeTypes(builder, schema); + builder.setCalendar( + useProlepticGregorian + ? OrcProto.CalendarKind.PROLEPTIC_GREGORIAN + : OrcProto.CalendarKind.JULIAN_GREGORIAN); + // add the stripe information + for (OrcProto.StripeInformation stripe : stripes) { + builder.addStripes(stripe); + } + // add the column statistics + treeWriter.writeFileStatistics(); + // add all of the user metadata + for (Map.Entry entry : userMetadata.entrySet()) { + builder.addMetadata( + OrcProto.UserMetadataItem.newBuilder() + .setName(entry.getKey()) + .setValue(entry.getValue())); + } + if (encryption.length > 0) { + builder.setEncryption(writeEncryptionFooter()); + } + builder.setWriter(OrcFile.WriterImplementation.ORC_JAVA.getId()); + builder.setSoftwareVersion(OrcUtils.getOrcVersion()); + physicalWriter.writeFileFooter(builder); + return writePostScript(); + } + + @Override + public TypeDescription getSchema() { + return schema; + } + + @Override + public void addUserMetadata(String name, ByteBuffer value) { + userMetadata.put(name, ByteString.copyFrom(value)); + } + + @Override + public void addRowBatch(VectorizedRowBatch batch) throws IOException { + try { + // If this is the first set of rows in this stripe, tell the tree writers + // to prepare the stripe. + if (batch.size != 0 && rowsInStripe == 0) { + treeWriter.prepareStripe(stripes.size() + 1); + } + if (buildIndex) { + // Batch the writes up to the rowIndexStride so that we can get the + // right size indexes. + int posn = 0; + while (posn < batch.size) { + int chunkSize = Math.min(batch.size - posn, rowIndexStride - rowsInIndex); + if (batch.isSelectedInUse()) { + // find the longest chunk that is continuously selected from posn + for (int len = 1; len < chunkSize; ++len) { + if (batch.selected[posn + len] - batch.selected[posn] != len) { + chunkSize = len; + break; + } + } + treeWriter.writeRootBatch(batch, batch.selected[posn], chunkSize); + } else { + treeWriter.writeRootBatch(batch, posn, chunkSize); + } + posn += chunkSize; + rowsInIndex += chunkSize; + rowsInStripe += chunkSize; + if (rowsInIndex >= rowIndexStride) { + createRowIndexEntry(); + } + } + } else { + if (batch.isSelectedInUse()) { + int posn = 0; + while (posn < batch.size) { + int chunkSize = 1; + while (posn + chunkSize < batch.size) { + // find the longest chunk that is continuously selected from posn + if (batch.selected[posn + chunkSize] - batch.selected[posn] + != chunkSize) { + break; + } + ++chunkSize; + } + treeWriter.writeRootBatch(batch, batch.selected[posn], chunkSize); + posn += chunkSize; + } + } else { + treeWriter.writeRootBatch(batch, 0, batch.size); + } + rowsInStripe += batch.size; + } + rowsSinceCheck += batch.size; + previousAllocation = memoryManager.checkMemory(previousAllocation, this); + checkMemory(); + } catch (Throwable t) { + try { + close(); + } catch (Throwable ignore) { + // ignore + } + if (t instanceof IOException) { + throw (IOException) t; + } else { + throw new IOException("Problem adding row to " + path, t); + } + } + } + + @Override + public void close() throws IOException { + if (!isClose) { + try { + if (callback != null) { + callback.preFooterWrite(callbackContext); + } + // remove us from the memory manager so that we don't get any callbacks + memoryManager.removeWriter(path); + // actually close the file + flushStripe(); + lastFlushOffset = writeFooter(); + physicalWriter.close(); + } finally { + isClose = true; + } + } + } + + /** + * Raw data size will be compute when writing the file footer. Hence raw data size value will be + * available only after closing the writer. + */ + @Override + public long getRawDataSize() { + return rawDataSize; + } + + /** + * Row count gets updated when flushing the stripes. To get accurate row count call this method + * after writer is closed. + */ + @Override + public long getNumberOfRows() { + return rowCount; + } + + @Override + public long writeIntermediateFooter() throws IOException { + // flush any buffered rows + flushStripe(); + // write a footer + if (stripesAtLastFlush != stripes.size()) { + if (callback != null) { + callback.preFooterWrite(callbackContext); + } + lastFlushOffset = writeFooter(); + stripesAtLastFlush = stripes.size(); + physicalWriter.flush(); + } + return lastFlushOffset; + } + + private static void checkArgument(boolean expression, String message) { + if (!expression) { + throw new IllegalArgumentException(message); + } + } + + @Override + public void appendStripe( + byte[] stripe, + int offset, + int length, + StripeInformation stripeInfo, + OrcProto.StripeStatistics stripeStatistics) + throws IOException { + appendStripe( + stripe, + offset, + length, + stripeInfo, + new StripeStatistics[] { + new StripeStatisticsImpl( + schema, stripeStatistics.getColStatsList(), false, false) + }); + } + + @Override + public void appendStripe( + byte[] stripe, + int offset, + int length, + StripeInformation stripeInfo, + StripeStatistics[] stripeStatistics) + throws IOException { + checkArgument(stripe != null, "Stripe must not be null"); + checkArgument( + length <= stripe.length, + "Specified length must not be greater specified array length"); + checkArgument(stripeInfo != null, "Stripe information must not be null"); + checkArgument(stripeStatistics != null, "Stripe statistics must not be null"); + + // If we have buffered rows, flush them + if (rowsInStripe > 0) { + flushStripe(); + } + rowsInStripe = stripeInfo.getNumberOfRows(); + // update stripe information + OrcProto.StripeInformation.Builder dirEntry = + OrcProto.StripeInformation.newBuilder() + .setNumberOfRows(rowsInStripe) + .setIndexLength(stripeInfo.getIndexLength()) + .setDataLength(stripeInfo.getDataLength()) + .setFooterLength(stripeInfo.getFooterLength()); + // If this is the first stripe of the original file, we need to copy the + // encryption information. + if (stripeInfo.hasEncryptionStripeId()) { + dirEntry.setEncryptStripeId(stripeInfo.getEncryptionStripeId()); + for (byte[] key : stripeInfo.getEncryptedLocalKeys()) { + dirEntry.addEncryptedLocalKeys(ByteString.copyFrom(key)); + } + } + physicalWriter.appendRawStripe(ByteBuffer.wrap(stripe, offset, length), dirEntry); + + // since we have already written the stripe, just update stripe statistics + treeWriter.addStripeStatistics(stripeStatistics); + + stripes.add(dirEntry.build()); + + // reset it after writing the stripe + rowCount += rowsInStripe; + rowsInStripe = 0; + needKeyFlush = encryption.length > 0; + } + + @Override + public void appendUserMetadata(List userMetadata) { + if (userMetadata != null) { + for (OrcProto.UserMetadataItem item : userMetadata) { + this.userMetadata.put(item.getName(), item.getValue()); + } + } + } + + @Override + public ColumnStatistics[] getStatistics() { + // get the column statistics + final ColumnStatistics[] result = new ColumnStatistics[schema.getMaximumId() + 1]; + // Get the file statistics, preferring the encrypted one. + treeWriter.getCurrentStatistics(result); + return result; + } + + @Override + public List getStripes() throws IOException { + return Collections.unmodifiableList(OrcUtils.convertProtoStripesToStripes(stripes)); + } + + public CompressionCodec getCompressionCodec() { + return unencryptedOptions.getCodec(); + } + + private static boolean hasTimestamp(TypeDescription schema) { + if (schema.getCategory() == TypeDescription.Category.TIMESTAMP) { + return true; + } + List children = schema.getChildren(); + if (children != null) { + for (TypeDescription child : children) { + if (hasTimestamp(child)) { + return true; + } + } + } + return false; + } + + private WriterEncryptionKey getKey(String keyName, KeyProvider provider) throws IOException { + WriterEncryptionKey result = keys.get(keyName); + if (result == null) { + result = new WriterEncryptionKey(provider.getCurrentKeyVersion(keyName)); + keys.put(keyName, result); + } + return result; + } + + private MaskDescriptionImpl getMask(String maskString) { + // if it is already there, get the earlier object + MaskDescriptionImpl result = maskDescriptions.get(maskString); + if (result == null) { + result = ParserUtils.buildMaskDescription(maskString); + maskDescriptions.put(maskString, result); + } + return result; + } + + private int visitTypeTree(TypeDescription schema, boolean encrypted, KeyProvider provider) + throws IOException { + int result = 0; + String keyName = schema.getAttributeValue(TypeDescription.ENCRYPT_ATTRIBUTE); + String maskName = schema.getAttributeValue(TypeDescription.MASK_ATTRIBUTE); + if (keyName != null) { + if (provider == null) { + throw new IllegalArgumentException("Encryption requires a KeyProvider."); + } + if (encrypted) { + throw new IllegalArgumentException("Nested encryption type: " + schema); + } + encrypted = true; + result += 1; + WriterEncryptionKey key = getKey(keyName, provider); + HadoopShims.KeyMetadata metadata = key.getMetadata(); + WriterEncryptionVariant variant = + new WriterEncryptionVariant(key, schema, provider.createLocalKey(metadata)); + key.addRoot(variant); + } + if (encrypted && (keyName != null || maskName != null)) { + MaskDescriptionImpl mask = getMask(maskName == null ? "nullify" : maskName); + mask.addColumn(schema); + } + List children = schema.getChildren(); + if (children != null) { + for (TypeDescription child : children) { + result += visitTypeTree(child, encrypted, provider); + } + } + return result; + } + + /** + * Iterate through the encryption options given by the user and set up our data structures. + * + * @param provider the KeyProvider to use to generate keys + * @param schema the type tree that we search for annotations + * @param keyOverrides user specified key overrides + */ + private WriterEncryptionVariant[] setupEncryption( + KeyProvider provider, + TypeDescription schema, + Map keyOverrides) + throws IOException { + keyProvider = + provider != null ? provider : CryptoUtils.getKeyProvider(conf, new SecureRandom()); + // Load the overrides into the cache so that we use the required key versions. + for (HadoopShims.KeyMetadata key : keyOverrides.values()) { + keys.put(key.getKeyName(), new WriterEncryptionKey(key)); + } + int variantCount = visitTypeTree(schema, false, keyProvider); + + // Now that we have de-duped the keys and maskDescriptions, make the arrays + int nextId = 0; + if (variantCount > 0) { + for (MaskDescriptionImpl mask : maskDescriptions.values()) { + mask.setId(nextId++); + for (TypeDescription column : mask.getColumns()) { + this.columnMaskDescriptions[column.getId()] = mask; + } + } + } + nextId = 0; + int nextVariantId = 0; + WriterEncryptionVariant[] result = new WriterEncryptionVariant[variantCount]; + for (WriterEncryptionKey key : keys.values()) { + key.setId(nextId++); + key.sortRoots(); + for (WriterEncryptionVariant variant : key.getEncryptionRoots()) { + result[nextVariantId] = variant; + columnEncryption[variant.getRoot().getId()] = variant; + variant.setId(nextVariantId++); + } + } + return result; + } + + @Override + public long estimateMemory() { + return this.treeWriter.estimateMemory(); + } +} diff --git a/paimon-format/src/main/java/org/apache/orc/impl/ZstdCodec.java b/paimon-format/src/main/java/org/apache/orc/impl/ZstdCodec.java new file mode 100644 index 000000000000..2eeb723c4ac5 --- /dev/null +++ b/paimon-format/src/main/java/org/apache/orc/impl/ZstdCodec.java @@ -0,0 +1,287 @@ +/* + * 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.orc.impl; + +import com.github.luben.zstd.Zstd; +import com.github.luben.zstd.ZstdCompressCtx; +import org.apache.orc.CompressionCodec; +import org.apache.orc.CompressionKind; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** orc ZstdCodec. */ +public class ZstdCodec implements CompressionCodec, DirectDecompressionCodec { + private ZstdOptions zstdOptions = null; + private ZstdCompressCtx zstdCompressCtx = null; + + public ZstdCodec(int level, int windowLog) { + this.zstdOptions = new ZstdOptions(level, windowLog); + } + + public ZstdCodec() { + this(3, 0); + } + + public ZstdOptions getZstdOptions() { + return zstdOptions; + } + + // Thread local buffer + private static final ThreadLocal threadBuffer = ThreadLocal.withInitial(() -> null); + + protected static byte[] getBuffer(int size) { + byte[] result = threadBuffer.get(); + if (result == null || result.length < size || result.length > size * 2) { + result = new byte[size]; + threadBuffer.set(result); + } + return result; + } + + static class ZstdOptions implements Options { + private int level; + private int windowLog; + + ZstdOptions(int level, int windowLog) { + this.level = level; + this.windowLog = windowLog; + } + + @Override + public ZstdOptions copy() { + return new ZstdOptions(level, windowLog); + } + + @Override + public Options setSpeed(SpeedModifier newValue) { + return this; + } + + /** + * Sets the Zstandard long mode maximum back-reference distance, expressed as a power of 2. + * + *

The value must be between ZSTD_WINDOWLOG_MIN (10) and ZSTD_WINDOWLOG_MAX (30 and 31 on + * 32/64-bit architectures, respectively). + * + *

A value of 0 is a special value indicating to use the default + * ZSTD_WINDOWLOG_LIMIT_DEFAULT of 27, which corresponds to back-reference window size of + * 128MiB. + * + * @param newValue The desired power-of-2 value back-reference distance. + * @return ZstdOptions + */ + public ZstdOptions setWindowLog(int newValue) { + if ((newValue < Zstd.windowLogMin() || newValue > Zstd.windowLogMax()) + && newValue != 0) { + throw new IllegalArgumentException( + String.format( + "Zstd compression window size should be in the range %d to %d," + + " or set to the default value of 0.", + Zstd.windowLogMin(), Zstd.windowLogMax())); + } + windowLog = newValue; + return this; + } + + /** + * Sets the Zstandard compression codec compression level directly using the integer + * setting. This value is typically between 0 and 22, with larger numbers indicating more + * aggressive compression and lower speed. + * + *

This method provides additional granularity beyond the setSpeed method so that users + * can select a specific level. + * + * @param newValue The level value of compression to set. + * @return ZstdOptions + */ + public ZstdOptions setLevel(int newValue) { + if (newValue < Zstd.minCompressionLevel() || newValue > Zstd.maxCompressionLevel()) { + throw new IllegalArgumentException( + String.format( + "Zstd compression level should be in the range %d to %d", + Zstd.minCompressionLevel(), Zstd.maxCompressionLevel())); + } + level = newValue; + return this; + } + + @Override + public ZstdOptions setData(DataKind newValue) { + return this; // We don't support setting DataKind in ZstdCodec. + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + ZstdOptions that = (ZstdOptions) o; + + if (level != that.level) { + return false; + } + return windowLog == that.windowLog; + } + + @Override + public int hashCode() { + int result = level; + result = 31 * result + windowLog; + return result; + } + } + + private static final ZstdOptions DEFAULT_OPTIONS = new ZstdOptions(3, 0); + + @Override + public Options getDefaultOptions() { + return DEFAULT_OPTIONS; + } + + /** + * Compresses an input ByteBuffer into an output ByteBuffer using Zstandard compression. If the + * maximum bound of the number of output bytes exceeds the output ByteBuffer size, the remaining + * bytes are written to the overflow ByteBuffer. + * + * @param in the bytes to compress + * @param out the compressed bytes + * @param overflow put any additional bytes here + * @param options the options to control compression + * @return ZstdOptions + */ + @Override + public boolean compress(ByteBuffer in, ByteBuffer out, ByteBuffer overflow, Options options) + throws IOException { + int inBytes = in.remaining(); + // Skip with minimum ZStandard format size: + // https://datatracker.ietf.org/doc/html/rfc8878#name-zstandard-frames + // Magic Number (4 bytes) + Frame Header (2 bytes) + Data Block Header (3 bytes) + if (inBytes < 10) { + return false; + } + + ZstdOptions zso = (ZstdOptions) options; + + zstdCompressCtx = new ZstdCompressCtx(); + zstdCompressCtx.setLevel(zso.level); + zstdCompressCtx.setLong(zso.windowLog); + zstdCompressCtx.setChecksum(false); + + try { + byte[] compressed = getBuffer((int) Zstd.compressBound(inBytes)); + + int outBytes = + zstdCompressCtx.compressByteArray( + compressed, + 0, + compressed.length, + in.array(), + in.arrayOffset() + in.position(), + inBytes); + if (outBytes < inBytes) { + int remaining = out.remaining(); + if (remaining >= outBytes) { + System.arraycopy( + compressed, + 0, + out.array(), + out.arrayOffset() + out.position(), + outBytes); + out.position(out.position() + outBytes); + } else { + System.arraycopy( + compressed, + 0, + out.array(), + out.arrayOffset() + out.position(), + remaining); + out.position(out.limit()); + System.arraycopy( + compressed, + remaining, + overflow.array(), + overflow.arrayOffset(), + outBytes - remaining); + overflow.position(outBytes - remaining); + } + return true; + } else { + return false; + } + } finally { + zstdCompressCtx.close(); + } + } + + @Override + public void decompress(ByteBuffer in, ByteBuffer out) throws IOException { + if (in.isDirect() && out.isDirect()) { + directDecompress(in, out); + return; + } + + int srcOffset = in.arrayOffset() + in.position(); + int srcSize = in.remaining(); + int dstOffset = out.arrayOffset() + out.position(); + int dstSize = out.remaining() - dstOffset; + + long decompressOut = + Zstd.decompressByteArray( + out.array(), dstOffset, dstSize, in.array(), srcOffset, srcSize); + in.position(in.limit()); + out.position(dstOffset + (int) decompressOut); + out.flip(); + } + + @Override + public boolean isAvailable() { + return true; + } + + @Override + public void directDecompress(ByteBuffer in, ByteBuffer out) throws IOException { + Zstd.decompress(out, in); + out.flip(); + } + + @Override + public void reset() {} + + @Override + public void destroy() { + if (zstdCompressCtx != null) { + zstdCompressCtx.close(); + } + } + + @Override + public CompressionKind getKind() { + return CompressionKind.ZSTD; + } + + @Override + public void close() { + OrcCodecPool.returnCodec(CompressionKind.ZSTD, this); + } +} diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java index ac33991c3062..1a925c7250f3 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java @@ -79,7 +79,7 @@ public OrcFileFormat(FormatContext formatContext) { } @VisibleForTesting - Properties orcProperties() { + public Properties orcProperties() { return orcProperties; } diff --git a/paimon-format/src/main/resources/META-INF/NOTICE b/paimon-format/src/main/resources/META-INF/NOTICE index f6e607ef3502..dae8d5fec19a 100644 --- a/paimon-format/src/main/resources/META-INF/NOTICE +++ b/paimon-format/src/main/resources/META-INF/NOTICE @@ -6,8 +6,8 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- org.apache.orc:orc-core:1.8.3 -- org.apache.orc:orc-shims:1.8.3 +- org.apache.orc:orc-core:1.9.2 +- org.apache.orc:orc-shims:1.9.2 - org.apache.hive:hive-storage-api:2.8.1 - io.airlift:aircompressor:0.21 - commons-lang:commons-lang:2.6 @@ -32,5 +32,5 @@ You find it under licenses/LICENSE.protobuf, licenses/LICENSE.zstd-jni and licenses/LICENSE.threeten-extra - com.google.protobuf:protobuf-java:3.17.3 -- com.github.luben:zstd-jni:1.5.0-1 +- com.github.luben:zstd-jni:1.5.5-11 - org.threeten:threeten-extra:1.7.1 diff --git a/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcZstdTest.java b/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcZstdTest.java new file mode 100644 index 000000000000..d5f8f8bc7cc7 --- /dev/null +++ b/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcZstdTest.java @@ -0,0 +1,166 @@ +/* + * 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.orc.writer; + +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.format.FileFormatFactory; +import org.apache.paimon.format.FormatWriter; +import org.apache.paimon.format.FormatWriterFactory; +import org.apache.paimon.format.orc.OrcFileFormat; +import org.apache.paimon.format.orc.OrcFileFormatFactory; +import org.apache.paimon.format.orc.OrcWriterFactory; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.PositionOutputStream; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.options.Options; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import com.github.luben.zstd.ZstdException; +import org.apache.hadoop.conf.Configuration; +import org.apache.orc.CompressionCodec; +import org.apache.orc.CompressionKind; +import org.apache.orc.OrcFile; +import org.apache.orc.Reader; +import org.apache.orc.impl.ZstdCodec; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Random; +import java.util.UUID; + +import static org.apache.paimon.format.orc.OrcFileFormatFactory.IDENTIFIER; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +class OrcZstdTest { + + @Test + void testWriteOrcWithZstd(@TempDir java.nio.file.Path tempDir) throws IOException { + Options options = new Options(); + options.set("compress", "zstd"); + options.set("stripe.size", "31457280"); + options.set("compression.zstd.level", "1"); + OrcFileFormat orc = + new OrcFileFormatFactory() + .create(new FileFormatFactory.FormatContext(options, 1024)); + Assertions.assertThat(orc).isInstanceOf(OrcFileFormat.class); + + Assertions.assertThat(orc.orcProperties().getProperty(IDENTIFIER + ".compress", "")) + .isEqualTo("zstd"); + Assertions.assertThat( + orc.orcProperties().getProperty(IDENTIFIER + ".compression.zstd.level", "")) + .isEqualTo("1"); + Assertions.assertThat(orc.orcProperties().getProperty(IDENTIFIER + ".stripe.size", "")) + .isEqualTo("31457280"); + + RowType rowType = + RowType.builder() + .field("a", DataTypes.INT()) + .field("b", DataTypes.STRING()) + .field("c", DataTypes.STRING()) + .field("d", DataTypes.STRING()) + .build(); + FormatWriterFactory writerFactory = orc.createWriterFactory(rowType); + Assertions.assertThat(writerFactory).isInstanceOf(OrcWriterFactory.class); + + Path path = new Path(tempDir.toUri().toString(), "1.orc"); + PositionOutputStream out = LocalFileIO.create().newOutputStream(path, true); + FormatWriter formatWriter = writerFactory.create(out, "zstd"); + + Assertions.assertThat(formatWriter).isInstanceOf(OrcBulkWriter.class); + + Options optionsWithLowLevel = new Options(); + optionsWithLowLevel.set("compress", "zstd"); + optionsWithLowLevel.set("stripe.size", "31457280"); + optionsWithLowLevel.set("compression.zstd.level", "1"); + + Random random = new Random(); + for (int i = 0; i < 1000; i++) { + GenericRow element = + GenericRow.of( + random.nextInt(), + BinaryString.fromString( + UUID.randomUUID().toString() + random.nextInt()), + BinaryString.fromString( + UUID.randomUUID().toString() + random.nextInt()), + BinaryString.fromString( + UUID.randomUUID().toString() + random.nextInt())); + formatWriter.addElement(element); + } + formatWriter.finish(); + OrcFile.ReaderOptions readerOptions = OrcFile.readerOptions(new Configuration()); + Reader reader = + OrcFile.createReader(new org.apache.hadoop.fs.Path(path.toString()), readerOptions); + Assertions.assertThat(reader.getNumberOfRows()).isEqualTo(1000); + Assertions.assertThat(reader.getCompressionKind()).isEqualTo(CompressionKind.ZSTD); + Assertions.assertThat(com.github.luben.zstd.util.Native.isLoaded()).isEqualTo(true); + } + + @Test + public void testCorrupt() throws Exception { + ByteBuffer buf = ByteBuffer.allocate(1000); + buf.put(new byte[] {127, 125, 1, 99, 98, 1}); + buf.flip(); + try (CompressionCodec codec = new ZstdCodec()) { + ByteBuffer out = ByteBuffer.allocate(1000); + codec.decompress(buf, out); + fail(); + } catch (ZstdException ioe) { + // EXPECTED + } + } + + @Test + public void testZstdDirectDecompress() { + ByteBuffer in = ByteBuffer.allocate(10000); + ByteBuffer out = ByteBuffer.allocate(10000); + ByteBuffer directOut = ByteBuffer.allocateDirect(10000); + ByteBuffer directResult = ByteBuffer.allocateDirect(10000); + for (int i = 0; i < 10000; i++) { + in.put((byte) i); + } + in.flip(); + try (ZstdCodec zstdCodec = new ZstdCodec()) { + // write bytes to heap buffer. + assertTrue(zstdCodec.compress(in, out, null, zstdCodec.getDefaultOptions())); + int position = out.position(); + out.flip(); + // copy heap buffer to direct buffer. + directOut.put(out.array()); + directOut.flip(); + directOut.limit(position); + + zstdCodec.decompress(directOut, directResult); + + // copy result from direct buffer to heap. + byte[] heapBytes = new byte[in.array().length]; + directResult.get(heapBytes, 0, directResult.limit()); + + assertArrayEquals(in.array(), heapBytes); + } catch (Exception e) { + fail(e); + } + } +} From 218aa187744f81f2e9b831134f2f85db4ab94011 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Wed, 20 Mar 2024 18:01:24 +0800 Subject: [PATCH 37/79] [license] Add license to orc copied files --- LICENSE | 8 ++++++++ .../src/main/java/org/apache/orc/CompressionKind.java | 7 +++++++ .../src/main/java/org/apache/orc/OrcConf.java | 11 ++++++++++- .../src/main/java/org/apache/orc/OrcFile.java | 11 ++++++++++- .../java/org/apache/orc/impl/PhysicalFsWriter.java | 11 ++++++++++- .../src/main/java/org/apache/orc/impl/WriterImpl.java | 7 +++++++ .../src/main/java/org/apache/orc/impl/ZstdCodec.java | 11 ++++++++++- 7 files changed, 62 insertions(+), 4 deletions(-) diff --git a/LICENSE b/LICENSE index eb317b6613a4..813296254542 100644 --- a/LICENSE +++ b/LICENSE @@ -255,6 +255,14 @@ from http://iceberg.apache.org/ version 1.3.0 paimon-hive/paimon-hive-common/src/test/resources/hive-schema-3.1.0.derby.sql from https://hive.apache.org/ version 3.1.0 +paimon-format/src/main/java/org/apache/orc/impl/PhysicalFsWriter.java +paimon-format/src/main/java/org/apache/orc/impl/WriterImpl.java +paimon-format/src/main/java/org/apache/orc/impl/ZstdCodec.java +paimon-format/src/main/java/org/apache/orc/CompressionKind.java +paimon-format/src/main/java/org/apache/orc/OrcConf.java +paimon-format/src/main/java/org/apache/orc/OrcFile.java +from https://orc.apache.org/ version 2.0 + MIT License ----------- diff --git a/paimon-format/src/main/java/org/apache/orc/CompressionKind.java b/paimon-format/src/main/java/org/apache/orc/CompressionKind.java index 9841f5a2919e..84de443a44ee 100644 --- a/paimon-format/src/main/java/org/apache/orc/CompressionKind.java +++ b/paimon-format/src/main/java/org/apache/orc/CompressionKind.java @@ -18,8 +18,15 @@ package org.apache.orc; +/* This file is based on source code from the ORC Project (http://orc.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + /** * An enumeration that lists the generic compression algorithms that can be applied to ORC files. + * + *

NOTE: The file was copied and modified to support zstd-jni. This feature is only supported in + * ORC 2.0, but 2.0 only supports JDK17. We need to support JDK8. */ public enum CompressionKind { NONE, diff --git a/paimon-format/src/main/java/org/apache/orc/OrcConf.java b/paimon-format/src/main/java/org/apache/orc/OrcConf.java index 2e1507eedf82..6aa357868f17 100644 --- a/paimon-format/src/main/java/org/apache/orc/OrcConf.java +++ b/paimon-format/src/main/java/org/apache/orc/OrcConf.java @@ -25,7 +25,16 @@ import java.util.List; import java.util.Properties; -/** Define the configuration properties that Orc understands. */ +/* This file is based on source code from the ORC Project (http://orc.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** + * Define the configuration properties that Orc understands. + * + *

NOTE: The file was copied and modified to support zstd-jni. This feature is only supported in + * ORC 2.0, but 2.0 only supports JDK17. We need to support JDK8. + */ public enum OrcConf { STRIPE_SIZE( "orc.stripe.size", diff --git a/paimon-format/src/main/java/org/apache/orc/OrcFile.java b/paimon-format/src/main/java/org/apache/orc/OrcFile.java index a903ba9e7545..f482174edc1f 100644 --- a/paimon-format/src/main/java/org/apache/orc/OrcFile.java +++ b/paimon-format/src/main/java/org/apache/orc/OrcFile.java @@ -42,7 +42,16 @@ import java.util.Map; import java.util.Properties; -/** Contains factory methods to read or write ORC files. */ +/* This file is based on source code from the ORC Project (http://orc.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** + * Contains factory methods to read or write ORC files. + * + *

NOTE: The file was copied and modified to support zstd-jni. This feature is only supported in + * ORC 2.0, but 2.0 only supports JDK17. We need to support JDK8. + */ public class OrcFile { private static final Logger LOG = LoggerFactory.getLogger(OrcFile.class); public static final String MAGIC = "ORC"; diff --git a/paimon-format/src/main/java/org/apache/orc/impl/PhysicalFsWriter.java b/paimon-format/src/main/java/org/apache/orc/impl/PhysicalFsWriter.java index f887e860a830..aab908fc718c 100644 --- a/paimon-format/src/main/java/org/apache/orc/impl/PhysicalFsWriter.java +++ b/paimon-format/src/main/java/org/apache/orc/impl/PhysicalFsWriter.java @@ -44,7 +44,16 @@ import java.util.Map; import java.util.TreeMap; -/** A orc PhysicalFsWriter. */ +/* This file is based on source code from the ORC Project (http://orc.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** + * An orc PhysicalFsWriter. + * + *

NOTE: The file was copied and modified to support zstd-jni. This feature is only supported in + * ORC 2.0, but 2.0 only supports JDK17. We need to support JDK8. + */ public class PhysicalFsWriter implements PhysicalWriter { private static final Logger LOG = LoggerFactory.getLogger(PhysicalFsWriter.class); diff --git a/paimon-format/src/main/java/org/apache/orc/impl/WriterImpl.java b/paimon-format/src/main/java/org/apache/orc/impl/WriterImpl.java index bbbc8f03172d..4249c8bd3579 100644 --- a/paimon-format/src/main/java/org/apache/orc/impl/WriterImpl.java +++ b/paimon-format/src/main/java/org/apache/orc/impl/WriterImpl.java @@ -62,6 +62,10 @@ import java.util.TimeZone; import java.util.TreeMap; +/* This file is based on source code from the ORC Project (http://orc.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + /** * An ORC file writer. The file is divided into stripes, which is the natural unit of work when * reading. Each stripe is buffered in memory until the memory reaches the stripe size and then it @@ -76,6 +80,9 @@ * *

Caveat: the MemoryManager is created during WriterOptions create, that has to be confined to a * single thread as well. + * + *

NOTE: The file was copied and modified to support zstd-jni. This feature is only supported in + * ORC 2.0, but 2.0 only supports JDK17. We need to support JDK8. */ public class WriterImpl implements WriterInternal, MemoryManager.Callback { diff --git a/paimon-format/src/main/java/org/apache/orc/impl/ZstdCodec.java b/paimon-format/src/main/java/org/apache/orc/impl/ZstdCodec.java index 2eeb723c4ac5..6803053dd236 100644 --- a/paimon-format/src/main/java/org/apache/orc/impl/ZstdCodec.java +++ b/paimon-format/src/main/java/org/apache/orc/impl/ZstdCodec.java @@ -26,7 +26,16 @@ import java.io.IOException; import java.nio.ByteBuffer; -/** orc ZstdCodec. */ +/* This file is based on source code from the ORC Project (http://orc.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** + * orc ZstdCodec. + * + *

NOTE: The file was copied and modified to support zstd-jni. This feature is only supported in + * ORC 2.0, but 2.0 only supports JDK17. We need to support JDK8. + */ public class ZstdCodec implements CompressionCodec, DirectDecompressionCodec { private ZstdOptions zstdOptions = null; private ZstdCompressCtx zstdCompressCtx = null; From b879054979ecad67ea7e8a48cede5f1a7cc50ec2 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Thu, 21 Mar 2024 08:44:34 +0800 Subject: [PATCH 38/79] [doc] Introduce orc options documentation page (#3061) --- docs/content/maintenance/configurations.md | 6 +- .../generated/core_configuration.html | 30 -------- .../generated/orc_configuration.html | 60 ++++++++++++++++ .../spark_connector_configuration.html | 12 ++-- .../java/org/apache/paimon/CoreOptions.java | 40 ----------- .../ConfigOptionsDocGenerator.java | 1 + .../org/apache/paimon/format/OrcOptions.java | 72 +++++++++++++++++++ .../paimon/format/orc/OrcWriterFactory.java | 3 +- .../format/orc/writer/OrcBulkWriterTest.java | 5 +- 9 files changed, 149 insertions(+), 80 deletions(-) create mode 100644 docs/layouts/shortcodes/generated/orc_configuration.html create mode 100644 paimon-format/src/main/java/org/apache/paimon/format/OrcOptions.java diff --git a/docs/content/maintenance/configurations.md b/docs/content/maintenance/configurations.md index 87d59cd34eff..564b7c44c9df 100644 --- a/docs/content/maintenance/configurations.md +++ b/docs/content/maintenance/configurations.md @@ -74,7 +74,11 @@ Spark connector options for paimon. {{< generated/spark_connector_configuration >}} -## RocksDB Options +### ORC Options + +{{< generated/orc_configuration >}} + +### RocksDB Options The following options allow users to finely adjust RocksDB for better performance. You can either specify them in table properties or in dynamic table hints. diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index ecf92c2f1067..3c2dbcaae74c 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -383,36 +383,6 @@ Integer The number of sorted runs that trigger the stopping of writes, the default value is 'num-sorted-run.compaction-trigger' + 1. - -

orc.bloom.filter.columns
- (none) - String - A comma-separated list of columns for which to create a bloom filter when writing. - - -
orc.bloom.filter.fpp
- 0.05 - Double - Define the default false positive probability for bloom filters. - - -
orc.column.encoding.direct
- (none) - Integer - Comma-separated list of fields for which dictionary encoding is to be skipped in orc. - - -
orc.dictionary.key.threshold
- (none) - Integer - If the number of distinct keys in a dictionary is greater than this fraction of the total number of non-null rows, turn off dictionary encoding in orc. Use 1 to always use dictionary encoding. - - -
orc.write.batch-size
- 1024 - Integer - write batch size for orc. -
page-size
64 kb diff --git a/docs/layouts/shortcodes/generated/orc_configuration.html b/docs/layouts/shortcodes/generated/orc_configuration.html new file mode 100644 index 000000000000..92b1a9106754 --- /dev/null +++ b/docs/layouts/shortcodes/generated/orc_configuration.html @@ -0,0 +1,60 @@ +{{/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/}} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
orc.column.encoding.direct
(none)IntegerComma-separated list of fields for which dictionary encoding is to be skipped in orc.
orc.compress
"lz4"StringDefine the compression codec for ORC file, if a higher compression ratio is required, it is recommended to configure it as 'zstd', and you can configure: orc.compression.zstd.level
orc.compression.zstd.level
1IntegerDefine the compression level to use with ZStandard codec while writing data. The valid range is 1~22.
orc.dictionary.key.threshold
0.8DoubleIf the number of distinct keys in a dictionary is greater than this fraction of the total number of non-null rows, turn off dictionary encoding in orc. Use 0 to always disable dictionary encoding. Use 1 to always use dictionary encoding.
orc.write.batch-size
1024Integerwrite batch size for orc.
diff --git a/docs/layouts/shortcodes/generated/spark_connector_configuration.html b/docs/layouts/shortcodes/generated/spark_connector_configuration.html index 09d363aff0c4..9e74cefbcfab 100644 --- a/docs/layouts/shortcodes/generated/spark_connector_configuration.html +++ b/docs/layouts/shortcodes/generated/spark_connector_configuration.html @@ -26,6 +26,12 @@ + +
catalog.create-underlying-session-catalog
+ false + Boolean + If true, create and use an underlying session catalog instead of default session catalog when use SparkGenericCatalog. +
read.changelog
false @@ -74,11 +80,5 @@ Boolean If true, allow to merge data types if the two types meet the rules for explicit casting. - -
catalog.create-underlying-session-catalog
- false - Boolean - If true, create and use an underlying session catalog instead of default session catalog when use SparkGenericCatalog. - 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 9deeb6324f3a..05e931237341 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -108,20 +108,6 @@ public class CoreOptions implements Serializable { .withDescription( "Specify the message format of data files, currently orc, parquet and avro are supported."); - public static final ConfigOption ORC_BLOOM_FILTER_COLUMNS = - key("orc.bloom.filter.columns") - .stringType() - .noDefaultValue() - .withDescription( - "A comma-separated list of columns for which to create a bloom filter when writing."); - - public static final ConfigOption ORC_BLOOM_FILTER_FPP = - key("orc.bloom.filter.fpp") - .doubleType() - .defaultValue(0.05) - .withDescription( - "Define the default false positive probability for bloom filters."); - public static final ConfigOption> FILE_COMPRESSION_PER_LEVEL = key("file.compression.per.level") .mapType() @@ -745,12 +731,6 @@ public class CoreOptions implements Serializable { .defaultValue(1024) .withDescription("Read batch size for orc and parquet."); - public static final ConfigOption ORC_WRITE_BATCH_SIZE = - key("orc.write.batch-size") - .intType() - .defaultValue(1024) - .withDescription("write batch size for orc."); - public static final ConfigOption CONSUMER_ID = key("consumer-id") .stringType() @@ -1002,22 +982,6 @@ public class CoreOptions implements Serializable { .noDefaultValue() .withDescription("Turn off the dictionary encoding for all fields in parquet."); - public static final ConfigOption ORC_COLUMN_ENCODING_DIRECT = - key("orc.column.encoding.direct") - .intType() - .noDefaultValue() - .withDescription( - "Comma-separated list of fields for which dictionary encoding is to be skipped in orc."); - - public static final ConfigOption ORC_DICTIONARY_KEY_THRESHOLD = - key("orc.dictionary.key.threshold") - .intType() - .noDefaultValue() - .withDescription( - "If the number of distinct keys in a dictionary is greater than this " - + "fraction of the total number of non-null rows, turn off " - + "dictionary encoding in orc. Use 1 to always use dictionary encoding."); - public static final ConfigOption SINK_WATERMARK_TIME_ZONE = key("sink.watermark-time-zone") .stringType() @@ -1643,10 +1607,6 @@ private Map callbacks( return result; } - public int orcWriteBatch() { - return options.getInteger(ORC_WRITE_BATCH_SIZE.key(), ORC_WRITE_BATCH_SIZE.defaultValue()); - } - public boolean localMergeEnabled() { return options.get(LOCAL_MERGE_BUFFER_SIZE) != null; } diff --git a/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java b/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java index 37d8661d2ead..6f700724ac01 100644 --- a/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java +++ b/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java @@ -77,6 +77,7 @@ public class ConfigOptionsDocGenerator { new OptionsClassLocation("paimon-core", "org.apache.paimon.lookup"), new OptionsClassLocation("paimon-core", "org.apache.paimon.catalog"), new OptionsClassLocation("paimon-core", "org.apache.paimon.jdbc"), + new OptionsClassLocation("paimon-format", "org.apache.paimon.format"), new OptionsClassLocation( "paimon-flink/paimon-flink-common", "org.apache.paimon.flink"), new OptionsClassLocation( diff --git a/paimon-format/src/main/java/org/apache/paimon/format/OrcOptions.java b/paimon-format/src/main/java/org/apache/paimon/format/OrcOptions.java new file mode 100644 index 000000000000..ef79ecd34296 --- /dev/null +++ b/paimon-format/src/main/java/org/apache/paimon/format/OrcOptions.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.format; + +import org.apache.paimon.options.ConfigOption; + +import org.apache.orc.OrcConf; + +import static org.apache.orc.OrcConf.COMPRESSION_ZSTD_LEVEL; +import static org.apache.orc.OrcConf.DICTIONARY_KEY_SIZE_THRESHOLD; +import static org.apache.orc.OrcConf.DIRECT_ENCODING_COLUMNS; +import static org.apache.paimon.options.ConfigOptions.key; + +/** Options for orc format. */ +public class OrcOptions { + + public static final ConfigOption ORC_WRITE_BATCH_SIZE = + key("orc.write.batch-size") + .intType() + .defaultValue(1024) + .withDescription("write batch size for orc."); + + public static final ConfigOption ORC_COMPRESS = + key(OrcConf.COMPRESS.getAttribute()) + .stringType() + .defaultValue("lz4") + .withDescription( + "Define the compression codec for ORC file, if a higher compression ratio is required, " + + "it is recommended to configure it as 'zstd', and you can configure: " + + COMPRESSION_ZSTD_LEVEL.getAttribute()); + + public static final ConfigOption ORC_COLUMN_ENCODING_DIRECT = + key(DIRECT_ENCODING_COLUMNS.getAttribute()) + .intType() + .noDefaultValue() + .withDescription( + "Comma-separated list of fields for which dictionary encoding is to be skipped in orc."); + + public static final ConfigOption ORC_DICTIONARY_KEY_THRESHOLD = + key(DICTIONARY_KEY_SIZE_THRESHOLD.getAttribute()) + .doubleType() + .defaultValue((Double) DICTIONARY_KEY_SIZE_THRESHOLD.getDefaultValue()) + .withDescription( + "If the number of distinct keys in a dictionary is greater than this " + + "fraction of the total number of non-null rows, turn off " + + "dictionary encoding in orc. Use 0 to always disable dictionary encoding. " + + "Use 1 to always use dictionary encoding."); + + public static final ConfigOption ORC_COMPRESSION_ZSTD_LEVEL = + key(COMPRESSION_ZSTD_LEVEL.getAttribute()) + .intType() + .defaultValue((Integer) COMPRESSION_ZSTD_LEVEL.getDefaultValue()) + .withDescription( + "Define the compression level to use with ZStandard codec while writing data. " + + "The valid range is 1~22."); +} diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcWriterFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcWriterFactory.java index 63f555653cc8..9a703bb18a55 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcWriterFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcWriterFactory.java @@ -42,6 +42,7 @@ import java.util.Properties; import java.util.UUID; +import static org.apache.paimon.format.OrcOptions.ORC_WRITE_BATCH_SIZE; import static org.apache.paimon.utils.Preconditions.checkNotNull; /** @@ -128,7 +129,7 @@ public void close() throws IOException { vectorizer, new WriterImpl(null, unusedPath, opts), out, - coreOptions.orcWriteBatch()); + coreOptions.toConfiguration().get(ORC_WRITE_BATCH_SIZE)); } @VisibleForTesting diff --git a/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcBulkWriterTest.java b/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcBulkWriterTest.java index 227d8a9d06ef..eccac13ea649 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcBulkWriterTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcBulkWriterTest.java @@ -18,7 +18,6 @@ package org.apache.paimon.format.orc.writer; -import org.apache.paimon.CoreOptions; import org.apache.paimon.format.FileFormat; import org.apache.paimon.format.FormatWriter; import org.apache.paimon.format.FormatWriterFactory; @@ -37,12 +36,14 @@ import java.io.IOException; +import static org.apache.paimon.format.OrcOptions.ORC_WRITE_BATCH_SIZE; + class OrcBulkWriterTest { @Test void testRowBatch(@TempDir java.nio.file.Path tempDir) throws IOException { Options options = new Options(); - options.set(CoreOptions.ORC_WRITE_BATCH_SIZE, 1); + options.set(ORC_WRITE_BATCH_SIZE, 1); FileFormat orc = FileFormat.getFileFormat(options, "orc"); Assertions.assertThat(orc).isInstanceOf(OrcFileFormat.class); From 28eeda16cef9d60c595680b1063ec26ec5a17a6c Mon Sep 17 00:00:00 2001 From: Kerwin <37063904+zhuangchong@users.noreply.github.com> Date: Thu, 21 Mar 2024 09:15:26 +0800 Subject: [PATCH 39/79] [doc] Add flink 1.19 to the documentation. (#3064) --- docs/content/engines/flink.md | 30 ++++++++++++++++-------------- docs/content/engines/overview.md | 20 ++++++++++---------- docs/content/project/download.md | 2 ++ 3 files changed, 28 insertions(+), 24 deletions(-) diff --git a/docs/content/engines/flink.md b/docs/content/engines/flink.md index eb5bd68364cd..461e565cad6f 100644 --- a/docs/content/engines/flink.md +++ b/docs/content/engines/flink.md @@ -30,32 +30,34 @@ This documentation is a guide for using Paimon in Flink. ## Preparing Paimon Jar File -Paimon currently supports Flink 1.18, 1.17, 1.16, 1.15. We recommend the latest Flink version for a better experience. +Paimon currently supports Flink 1.19, 1.18, 1.17, 1.16, 1.15. We recommend the latest Flink version for a better experience. Download the jar file with corresponding version. > Currently, paimon provides two types jar: one of which(the bundled jar) is used for read/write data, and the other(action jar) for operations such as manually compaction, {{< stable >}} -| Version | Type | Jar | -|------------|-------|------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| Flink 1.18 | Bundled Jar | [paimon-flink-1.18-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-flink-1.18/{{< version >}}/paimon-flink-1.18-{{< version >}}.jar) | -| Flink 1.17 | Bundled Jar | [paimon-flink-1.17-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-flink-1.17/{{< version >}}/paimon-flink-1.17-{{< version >}}.jar) | -| Flink 1.16 | Bundled Jar | [paimon-flink-1.16-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-flink-1.16/{{< version >}}/paimon-flink-1.16-{{< version >}}.jar) | -| Flink 1.15 | Bundled Jar | [paimon-flink-1.15-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-flink-1.15/{{< version >}}/paimon-flink-1.15-{{< version >}}.jar) | +| Version | Type | Jar | +|--------------|-------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| Flink 1.19 | Bundled Jar | [paimon-flink-1.19-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-flink-1.19/{{< version >}}/paimon-flink-1.19-{{< version >}}.jar) | +| Flink 1.18 | Bundled Jar | [paimon-flink-1.18-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-flink-1.18/{{< version >}}/paimon-flink-1.18-{{< version >}}.jar) | +| Flink 1.17 | Bundled Jar | [paimon-flink-1.17-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-flink-1.17/{{< version >}}/paimon-flink-1.17-{{< version >}}.jar) | +| Flink 1.16 | Bundled Jar | [paimon-flink-1.16-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-flink-1.16/{{< version >}}/paimon-flink-1.16-{{< version >}}.jar) | +| Flink 1.15 | Bundled Jar | [paimon-flink-1.15-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-flink-1.15/{{< version >}}/paimon-flink-1.15-{{< version >}}.jar) | | Flink Action | Action Jar | [paimon-flink-action-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-flink-action/{{< version >}}/paimon-flink-action-{{< version >}}.jar) | {{< /stable >}} {{< unstable >}} -| Version | Type | Jar | -|------------|------|---------------------------------------------------------------------------------------------------------------------------------| -| Flink 1.18 | Bundled Jar | [paimon-flink-1.18-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-flink-1.18/{{< version >}}/) | -| Flink 1.17 | Bundled Jar | [paimon-flink-1.17-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-flink-1.17/{{< version >}}/) | -| Flink 1.16 | Bundled Jar | [paimon-flink-1.16-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-flink-1.16/{{< version >}}/) | -| Flink 1.15 | Bundled Jar | [paimon-flink-1.15-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-flink-1.15/{{< version >}}/) | -| Flink Action | Action Jar | [paimon-flink-action-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-flink-action/{{< version >}}/) | +| Version | Type | Jar | +|--------------|-------------|-------------------------------------------------------------------------------------------------------------------------------------------| +| Flink 1.19 | Bundled Jar | [paimon-flink-1.19-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-flink-1.19/{{< version >}}/) | +| Flink 1.18 | Bundled Jar | [paimon-flink-1.18-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-flink-1.18/{{< version >}}/) | +| Flink 1.17 | Bundled Jar | [paimon-flink-1.17-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-flink-1.17/{{< version >}}/) | +| Flink 1.16 | Bundled Jar | [paimon-flink-1.16-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-flink-1.16/{{< version >}}/) | +| Flink 1.15 | Bundled Jar | [paimon-flink-1.15-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-flink-1.15/{{< version >}}/) | +| Flink Action | Action Jar | [paimon-flink-action-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-flink-action/{{< version >}}/) | {{< /unstable >}} diff --git a/docs/content/engines/overview.md b/docs/content/engines/overview.md index 31c5eb7a784a..96a020df0b8e 100644 --- a/docs/content/engines/overview.md +++ b/docs/content/engines/overview.md @@ -32,17 +32,17 @@ Apache Spark and Apache Hive. ## Compatibility Matrix -| Engine | Version | Batch Read | Batch Write | Create Table | Alter Table | Streaming Write | Streaming Read | Batch Overwrite | -|:------:|:-------------:|:----------:|:-----------:|:------------:|:-----------:|:---------------:|:--------------:|:---------------:| -| Flink | 1.15 - 1.18 | ✅ | ✅ | ✅ | ✅(1.17+) | ✅ | ✅ | ✅ | -| Spark | 3.1 - 3.5 | ✅ | ✅ | ✅ | ✅ | ✅ | ✅(3.3+) | ✅ | -| Hive | 2.1 - 3.1 | ✅ | ✅ | ✅ | ❌ | ❌ | ❌ | ❌ | -| Spark | 2.4 | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | -| Trino | 422 - 426 | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | -| Trino | 427 - 439 | ✅ | ❌ | ✅ | ✅ | ❌ | ❌ | ❌ | -| Presto | 0.236 - 0.280 | ✅ | ❌ | ✅ | ✅ | ❌ | ❌ | ❌ | +| Engine | Version | Batch Read | Batch Write | Create Table | Alter Table | Streaming Write | Streaming Read | Batch Overwrite | +|:-------------------------------------------------------------------------------:|:-------------:|:----------:|:-----------:|:------------:|:-----------:|:---------------:|:--------------:|:---------------:| +| Flink | 1.15 - 1.19 | ✅ | ✅ | ✅ | ✅(1.17+) | ✅ | ✅ | ✅ | +| Spark | 3.1 - 3.5 | ✅ | ✅ | ✅ | ✅ | ✅ | ✅(3.3+) | ✅ | +| Hive | 2.1 - 3.1 | ✅ | ✅ | ✅ | ❌ | ❌ | ❌ | ❌ | +| Spark | 2.4 | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | +| Trino | 422 - 426 | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | +| Trino | 427 - 439 | ✅ | ❌ | ✅ | ✅ | ❌ | ❌ | ❌ | +| Presto | 0.236 - 0.280 | ✅ | ❌ | ✅ | ✅ | ❌ | ❌ | ❌ | | [StarRocks](https://docs.starrocks.io/docs/data_source/catalog/paimon_catalog/) | 3.1+ | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | -| [Doris](https://doris.apache.org/docs/lakehouse/multi-catalog/paimon/) | 2.0+ | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | +| [Doris](https://doris.apache.org/docs/lakehouse/multi-catalog/paimon/) | 2.0+ | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | Recommended versions are Flink 1.17.2, Spark 3.5.0, Hive 2.3.9 diff --git a/docs/content/project/download.md b/docs/content/project/download.md index 9b39cc8eeaee..f24793a7c6f4 100644 --- a/docs/content/project/download.md +++ b/docs/content/project/download.md @@ -34,6 +34,7 @@ This documentation is a guide for downloading Paimon Jars. | Version | Jar | |------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| Flink 1.19 | [paimon-flink-1.19-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-flink-1.19/{{< version >}}/) | | Flink 1.18 | [paimon-flink-1.18-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-flink-1.18/{{< version >}}/) | | Flink 1.17 | [paimon-flink-1.17-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-flink-1.17/{{< version >}}/) | | Flink 1.16 | [paimon-flink-1.16-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-flink-1.16/{{< version >}}/) | @@ -62,6 +63,7 @@ This documentation is a guide for downloading Paimon Jars. | Version | Jar | |------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| Flink 1.19 | [paimon-flink-1.19-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-flink-1.19/{{< version >}}/paimon-flink-1.19-{{< version >}}.jar) | | Flink 1.18 | [paimon-flink-1.18-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-flink-1.18/{{< version >}}/paimon-flink-1.18-{{< version >}}.jar) | | Flink 1.17 | [paimon-flink-1.17-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-flink-1.17/{{< version >}}/paimon-flink-1.17-{{< version >}}.jar) | | Flink 1.16 | [paimon-flink-1.16-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-flink-1.16/{{< version >}}/paimon-flink-1.16-{{< version >}}.jar) | From 0c15e56e2c04ce0d448282ef422ecda2752139f9 Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Thu, 21 Mar 2024 13:54:53 +0800 Subject: [PATCH 40/79] [hotfix] Remove redundant and unused test code (#3066) --- .../paimon/flink/CatalogITCaseBase.java | 11 ---- .../paimon/flink/RemoteLookupJoinITCase.java | 4 +- .../paimon/flink/sink/FlinkSinkTest.java | 62 ------------------- 3 files changed, 2 insertions(+), 75 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java index 20ebdd505b30..79f1079a230d 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java @@ -98,17 +98,6 @@ public void before() throws IOException { prepareEnv(); } - protected Table getPaimonTable(String tableName) { - FlinkCatalog flinkCatalog = (FlinkCatalog) tEnv.getCatalog(tEnv.getCurrentCatalog()).get(); - try { - return flinkCatalog - .catalog() - .getTable(new Identifier(tEnv.getCurrentDatabase(), tableName)); - } catch (org.apache.paimon.catalog.Catalog.TableNotExistException e) { - throw new RuntimeException(e); - } - } - protected Map catalogOptions() { return Collections.emptyMap(); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RemoteLookupJoinITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RemoteLookupJoinITCase.java index 143b4f39652f..9e7aac281f1d 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RemoteLookupJoinITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RemoteLookupJoinITCase.java @@ -67,7 +67,7 @@ public void testQueryServiceLookup() throws Exception { sql( "CREATE TABLE DIM (k INT PRIMARY KEY NOT ENFORCED, v INT) WITH ('bucket' = '2', 'continuous.discovery-interval' = '1ms')"); CloseableIterator service = streamSqlIter("CALL sys.query_service('default.DIM', 2)"); - RemoteTableQuery query = new RemoteTableQuery(getPaimonTable("DIM")); + RemoteTableQuery query = new RemoteTableQuery(paimonTable("DIM")); sql("INSERT INTO DIM VALUES (1, 11), (2, 22), (3, 33), (4, 44)"); Thread.sleep(2000); @@ -132,7 +132,7 @@ public void testLookupRemoteTable() throws Throwable { } private ServiceProxy launchQueryServer(String tableName) throws Throwable { - FileStoreTable table = (FileStoreTable) getPaimonTable(tableName); + FileStoreTable table = (FileStoreTable) paimonTable(tableName); LocalTableQuery query = table.newLocalTableQuery().withIOManager(IOManager.create(path)); KvQueryServer server = new KvQueryServer( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java index 23b983b06238..2a1022dced7d 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java @@ -37,22 +37,17 @@ import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.state.StateInitializationContextImpl; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.operators.collect.utils.MockOperatorStateStore; import org.apache.flink.streaming.api.transformations.OneInputTransformation; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -139,61 +134,4 @@ private FileStoreTable createFileStoreTable() throws Exception { options, new CatalogEnvironment(Lock.emptyFactory(), null, null)); } - - private OneInputStreamOperatorTestHarness createTestHarness( - OneInputStreamOperator operator) throws Exception { - TypeSerializer serializer = - new CommittableTypeInfo().createSerializer(new ExecutionConfig()); - OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(operator); - harness.setup(serializer); - return harness; - } - - private OneInputStreamOperatorTestHarness, Committable> - createDynamicBucketTestHarness( - OneInputStreamOperator, Committable> operator) - throws Exception { - TypeSerializer serializer = - new CommittableTypeInfo().createSerializer(new ExecutionConfig()); - OneInputStreamOperatorTestHarness, Committable> harness = - new OneInputStreamOperatorTestHarness<>(operator); - harness.setup(serializer); - return harness; - } - - protected RowDataStoreWriteOperator createWriteOperator(FileStoreTable table) { - return new RowDataStoreWriteOperator( - table, - null, - (t, commitUser, state, ioManager, memoryPool, metricGroup) -> - new StoreSinkWriteImpl( - t, - commitUser, - state, - ioManager, - false, - false, - true, - memoryPool, - metricGroup), - "test"); - } - - protected DynamicBucketRowWriteOperator createDynamicBucketWriteOperator(FileStoreTable table) { - return new DynamicBucketRowWriteOperator( - table, - (t, commitUser, state, ioManager, memoryPool, metricGroup) -> - new StoreSinkWriteImpl( - t, - commitUser, - state, - ioManager, - false, - false, - true, - memoryPool, - metricGroup), - "test"); - } } From c92b00dd536c7f6a4e14394557fe76557af82916 Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Thu, 21 Mar 2024 16:31:45 +0800 Subject: [PATCH 41/79] [doc] Fix version in trino documentation (#3072) --- docs/content/engines/trino.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content/engines/trino.md b/docs/content/engines/trino.md index 9c2bdfd3f91e..ad4f8fb9b261 100644 --- a/docs/content/engines/trino.md +++ b/docs/content/engines/trino.md @@ -34,7 +34,7 @@ Paimon currently supports Trino 422 and above. ## Filesystem -From version 0.8, paimon share trino filesystem for all actions, which means, iyou should +From version 0.8, paimon share trino filesystem for all actions, which means, you should config trino filesystem before using trino-paimon. You can find information about how to config filesystems for trino on trino official website. @@ -51,7 +51,7 @@ https://paimon.apache.org/docs/master/project/download/ | Version | Package | |---------------|-----------------------------------------------------------------------------------------------------------------------------------------------| -| [422, 427] | [paimon-trino-422-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-422/{{< version >}}/) | +| [422, 426] | [paimon-trino-422-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-422/{{< version >}}/) | | [427, latest] | [paimon-trino-427-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-427/{{< version >}}/) | {{< /unstable >}} From 9a2f0bafb13a1ee4d0cbbd54405857534003496d Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Thu, 21 Mar 2024 17:17:11 +0800 Subject: [PATCH 42/79] [core] Skip rewrite when delete row count is 0 with dv (#3070) --- .../compact/ChangelogMergeTreeRewriter.java | 18 ++++++++++-------- .../FullChangelogMergeTreeCompactRewriter.java | 6 +++--- .../LookupMergeTreeCompactRewriter.java | 15 +++++++-------- .../compact/MergeTreeCompactRewriter.java | 4 ++-- 4 files changed, 22 insertions(+), 21 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/ChangelogMergeTreeRewriter.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/ChangelogMergeTreeRewriter.java index ee83df08a1fe..98553030d00d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/ChangelogMergeTreeRewriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/ChangelogMergeTreeRewriter.java @@ -75,7 +75,7 @@ public ChangelogMergeTreeRewriter( protected abstract boolean rewriteChangelog( int outputLevel, boolean dropDelete, List> sections); - protected abstract UpgradeStrategy upgradeChangelog(int outputLevel, DataFileMeta file); + protected abstract UpgradeStrategy upgradeStrategy(int outputLevel, DataFileMeta file); protected abstract MergeFunctionWrapper createMergeWrapper(int outputLevel); @@ -100,19 +100,19 @@ protected boolean rewriteLookupChangelog(int outputLevel, List> public CompactResult rewrite( int outputLevel, boolean dropDelete, List> sections) throws Exception { if (rewriteChangelog(outputLevel, dropDelete, sections)) { - return rewriteChangelogCompaction(outputLevel, sections, dropDelete, true); + return rewriteOrProduceChangelog(outputLevel, sections, dropDelete, true); } else { return rewriteCompaction(outputLevel, dropDelete, sections); } } /** - * Rewrite and produce changelog at the same time. + * Rewrite or produce changelog at the same time. * * @param dropDelete whether to drop delete when rewrite compact file * @param rewriteCompactFile whether to rewrite compact file */ - private CompactResult rewriteChangelogCompaction( + private CompactResult rewriteOrProduceChangelog( int outputLevel, List> sections, boolean dropDelete, @@ -168,7 +168,9 @@ private CompactResult rewriteChangelogCompaction( .map(x -> x.upgrade(outputLevel)) .collect(Collectors.toList()); - notifyCompactBefore(before); + if (rewriteCompactFile) { + notifyRewriteCompactBefore(before); + } List changelogFiles = changelogFileWriter != null @@ -179,9 +181,9 @@ private CompactResult rewriteChangelogCompaction( @Override public CompactResult upgrade(int outputLevel, DataFileMeta file) throws Exception { - UpgradeStrategy strategy = upgradeChangelog(outputLevel, file); + UpgradeStrategy strategy = upgradeStrategy(outputLevel, file); if (strategy.changelog) { - return rewriteChangelogCompaction( + return rewriteOrProduceChangelog( outputLevel, Collections.singletonList( Collections.singletonList(SortedRun.fromSingle(file))), @@ -194,7 +196,7 @@ public CompactResult upgrade(int outputLevel, DataFileMeta file) throws Exceptio /** Strategy for upgrade. */ protected enum UpgradeStrategy { - NO_CHANGELOG(false, false), + NO_CHANGELOG_NO_REWRITE(false, false), CHANGELOG_NO_REWRITE(true, false), CHANGELOG_WITH_REWRITE(true, true); diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/FullChangelogMergeTreeCompactRewriter.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/FullChangelogMergeTreeCompactRewriter.java index 5415427f4245..7283cbb7bf2f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/FullChangelogMergeTreeCompactRewriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/FullChangelogMergeTreeCompactRewriter.java @@ -37,7 +37,7 @@ import java.util.List; import static org.apache.paimon.mergetree.compact.ChangelogMergeTreeRewriter.UpgradeStrategy.CHANGELOG_NO_REWRITE; -import static org.apache.paimon.mergetree.compact.ChangelogMergeTreeRewriter.UpgradeStrategy.NO_CHANGELOG; +import static org.apache.paimon.mergetree.compact.ChangelogMergeTreeRewriter.UpgradeStrategy.NO_CHANGELOG_NO_REWRITE; /** A {@link MergeTreeCompactRewriter} which produces changelog files for each full compaction. */ public class FullChangelogMergeTreeCompactRewriter extends ChangelogMergeTreeRewriter { @@ -84,8 +84,8 @@ protected boolean rewriteChangelog( } @Override - protected UpgradeStrategy upgradeChangelog(int outputLevel, DataFileMeta file) { - return outputLevel == maxLevel ? CHANGELOG_NO_REWRITE : NO_CHANGELOG; + protected UpgradeStrategy upgradeStrategy(int outputLevel, DataFileMeta file) { + return outputLevel == maxLevel ? CHANGELOG_NO_REWRITE : NO_CHANGELOG_NO_REWRITE; } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/LookupMergeTreeCompactRewriter.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/LookupMergeTreeCompactRewriter.java index 02c3e9c6d40e..dc76b24b38f8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/LookupMergeTreeCompactRewriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/LookupMergeTreeCompactRewriter.java @@ -41,7 +41,7 @@ import static org.apache.paimon.mergetree.compact.ChangelogMergeTreeRewriter.UpgradeStrategy.CHANGELOG_NO_REWRITE; import static org.apache.paimon.mergetree.compact.ChangelogMergeTreeRewriter.UpgradeStrategy.CHANGELOG_WITH_REWRITE; -import static org.apache.paimon.mergetree.compact.ChangelogMergeTreeRewriter.UpgradeStrategy.NO_CHANGELOG; +import static org.apache.paimon.mergetree.compact.ChangelogMergeTreeRewriter.UpgradeStrategy.NO_CHANGELOG_NO_REWRITE; /** * A {@link MergeTreeCompactRewriter} which produces changelog files by lookup for the compaction @@ -83,7 +83,7 @@ public LookupMergeTreeCompactRewriter( } @Override - protected void notifyCompactBefore(List files) { + protected void notifyRewriteCompactBefore(List files) { if (dvMaintainer != null) { files.forEach(file -> dvMaintainer.removeDeletionVectorOf(file.fileName())); } @@ -96,15 +96,14 @@ protected boolean rewriteChangelog( } @Override - protected UpgradeStrategy upgradeChangelog(int outputLevel, DataFileMeta file) { + protected UpgradeStrategy upgradeStrategy(int outputLevel, DataFileMeta file) { if (file.level() != 0) { - return NO_CHANGELOG; + return NO_CHANGELOG_NO_REWRITE; } - // TODO In deletionVector mode, since drop delete is required, rewrite is always required. - // TODO wait https://github.com/apache/incubator-paimon/pull/2962 - // TODO but should be careful to not be deleted by DeletionVectorsMaintainer! - if (dvMaintainer != null) { + // In deletionVector mode, since drop delete is required, when delete row count > 0 rewrite + // is required. + if (dvMaintainer != null && file.deleteRowCount().map(cnt -> cnt > 0).orElse(true)) { return CHANGELOG_WITH_REWRITE; } diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactRewriter.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactRewriter.java index 07b2fe4fcc7f..4cfe00f3f73e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactRewriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/MergeTreeCompactRewriter.java @@ -82,7 +82,7 @@ protected CompactResult rewriteCompaction( writer.write(new RecordReaderIterator<>(reader)); writer.close(); List before = extractFilesFromSections(sections); - notifyCompactBefore(before); + notifyRewriteCompactBefore(before); return new CompactResult(before, writer.result()); } @@ -98,5 +98,5 @@ protected RecordReader readerForMergeTree( mergeSorter); } - protected void notifyCompactBefore(List files) {} + protected void notifyRewriteCompactBefore(List files) {} } From 617bff2eb359dfc2e3746f49e3dfd39f473a1ba2 Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Thu, 21 Mar 2024 17:53:04 +0800 Subject: [PATCH 43/79] [hotfix] Add project root path to files to facilitate compiling modules in child dirs (#3063) --- .mvn/readme.txt | 4 ++++ pom.xml | 7 ++++--- 2 files changed, 8 insertions(+), 3 deletions(-) create mode 100644 .mvn/readme.txt diff --git a/.mvn/readme.txt b/.mvn/readme.txt new file mode 100644 index 000000000000..a1333a41d229 --- /dev/null +++ b/.mvn/readme.txt @@ -0,0 +1,4 @@ +The .mvn directory is needed to be able to use the ${maven.multiModuleProjectDirectory} property, since git cannot +commit an empty directory, add this file. + +Once we do not use ${maven.multiModuleProjectDirectory}, we can remove this file and .mvn directory. \ No newline at end of file diff --git a/pom.xml b/pom.xml index 6fdda9566d7b..eb30d0d74bc5 100644 --- a/pom.xml +++ b/pom.xml @@ -831,18 +831,19 @@ under the License. - copyright.txt + + ${maven.multiModuleProjectDirectory}/copyright.txt ${spotless.delimiter} 3.4.3 - .scalafmt.conf + ${maven.multiModuleProjectDirectory}/.scalafmt.conf - copyright.txt + ${maven.multiModuleProjectDirectory}/copyright.txt ${spotless.delimiter} From 8e57754d347cc15aaa99fa4a8a1fd044350cda4f Mon Sep 17 00:00:00 2001 From: YeJunHao <41894543+leaves12138@users.noreply.github.com> Date: Thu, 21 Mar 2024 17:53:38 +0800 Subject: [PATCH 44/79] [flink] Compact procedure does not process 'ALL' word (#3065) --- .../org/apache/paimon/flink/procedure/CompactProcedure.java | 2 +- .../flink/action/SortCompactActionForDynamicBucketITCase.java | 2 +- .../flink/action/SortCompactActionForUnawareBucketITCase.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java index fe354e54b7f8..ae8dfb07822b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java @@ -112,7 +112,7 @@ public String[] call( "You must specify 'order strategy' and 'order by columns' both."); } - if (!(StringUtils.isBlank(partitions) || "ALL".equals(partitions))) { + if (!(StringUtils.isBlank(partitions))) { action.withPartitions(ParameterUtils.getPartitions(partitions.split(";"))); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForDynamicBucketITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForDynamicBucketITCase.java index a5195c2f3bc9..3101ffb1b182 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForDynamicBucketITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForDynamicBucketITCase.java @@ -211,7 +211,7 @@ private SortCompactAction createAction(String orderStrategy, List column private void callProcedure(String orderStrategy, List orderByColumns) { callProcedure( String.format( - "CALL sys.compact('%s.%s', 'ALL', '%s', '%s')", + "CALL sys.compact('%s.%s', '', '%s', '%s')", database, tableName, orderStrategy, String.join(",", orderByColumns)), false, true); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java index 272b3516a3b8..24c9101a3e84 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java @@ -381,7 +381,7 @@ private void callProcedure( String orderStrategy, String rangeStrategy, List orderByColumns) { callProcedure( String.format( - "CALL sys.compact('%s.%s', 'ALL', '%s', '%s','sort-compaction.range-strategy=%s')", + "CALL sys.compact('%s.%s', '', '%s', '%s','sort-compaction.range-strategy=%s')", database, tableName, orderStrategy, From 6378848d7b9f10ac6122a0b23e11ae52a6d9d108 Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Fri, 22 Mar 2024 10:45:33 +0800 Subject: [PATCH 45/79] [fix] Fix unstable test TagAutoCreationTest#testModifyTagPeriod (#3073) --- .../test/java/org/apache/paimon/tag/TagAutoCreationTest.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/paimon-core/src/test/java/org/apache/paimon/tag/TagAutoCreationTest.java b/paimon-core/src/test/java/org/apache/paimon/tag/TagAutoCreationTest.java index 62f53aadb030..f76a58e6cfd1 100644 --- a/paimon-core/src/test/java/org/apache/paimon/tag/TagAutoCreationTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/tag/TagAutoCreationTest.java @@ -195,7 +195,6 @@ public void testModifyTagPeriod() { Options options = new Options(); options.set(TAG_AUTOMATIC_CREATION, TagCreationMode.WATERMARK); options.set(TAG_CREATION_PERIOD, TagCreationPeriod.HOURLY); - options.set(SINK_WATERMARK_TIME_ZONE, ZoneId.systemDefault().toString()); FileStoreTable table; TableCommitImpl commit; TagManager tagManager; @@ -204,7 +203,7 @@ public void testModifyTagPeriod() { tagManager = table.store().newTagManager(); // test first create - commit.commit(new ManifestCommittable(0, localZoneMills("2023-07-18T12:00:09"))); + commit.commit(new ManifestCommittable(0, utcMills("2023-07-18T12:00:09"))); assertThat(tagManager.allTagNames()).containsOnly("2023-07-18 11"); options.set(TAG_CREATION_PERIOD, TagCreationPeriod.DAILY); From d4beafac7c56770f473c0ab13f1550fc9adb0bbd Mon Sep 17 00:00:00 2001 From: Yann Byron Date: Fri, 22 Mar 2024 14:31:37 +0800 Subject: [PATCH 46/79] [spark] Minor refactor SparkWriter to separate commit (#3067) --- .../apache/paimon/table/FileStoreTable.java | 3 + .../spark/commands/BucketProcessor.scala | 142 ++++++++++ .../paimon/spark/commands/PaimonCommand.scala | 26 +- .../spark/commands/PaimonSparkWriter.scala | 214 +++++++++++++++ .../paimon/spark/commands/SchemaHelper.scala | 3 - .../spark/commands/WriteIntoPaimonTable.scala | 252 +----------------- .../spark/sql/DynamicBucketTableTest.scala | 2 +- 7 files changed, 367 insertions(+), 275 deletions(-) create mode 100644 paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/BucketProcessor.scala create mode 100644 paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonSparkWriter.scala diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java index 2134d97cac98..a96a6138d7f6 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java @@ -25,6 +25,7 @@ import org.apache.paimon.schema.TableSchema; import org.apache.paimon.stats.BinaryTableStats; import org.apache.paimon.table.query.LocalTableQuery; +import org.apache.paimon.table.sink.RowKeyExtractor; import org.apache.paimon.table.sink.TableCommitImpl; import org.apache.paimon.table.sink.TableWriteImpl; import org.apache.paimon.types.RowType; @@ -108,4 +109,6 @@ default BinaryTableStats getSchemaFieldStats(DataFileMeta dataFileMeta) { } boolean supportStreamingReadOverwrite(); + + RowKeyExtractor createRowKeyExtractor(); } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/BucketProcessor.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/BucketProcessor.scala new file mode 100644 index 000000000000..4af2b10b1967 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/BucketProcessor.scala @@ -0,0 +1,142 @@ +/* + * 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.commands + +import org.apache.paimon.data.{InternalRow => PaimonInternalRow} +import org.apache.paimon.index.HashBucketAssigner +import org.apache.paimon.spark.SparkRow +import org.apache.paimon.spark.util.EncoderUtils +import org.apache.paimon.table.FileStoreTable +import org.apache.paimon.table.sink.RowPartitionKeyExtractor + +import org.apache.spark.TaskContext +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.{InternalRow => SparkInternalRow} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.{Deserializer, Serializer} +import org.apache.spark.sql.types.StructType + +import java.util.UUID + +case class EncoderSerDeGroup(schema: StructType) { + + val encoder: ExpressionEncoder[Row] = EncoderUtils.encode(schema).resolveAndBind() + + private val serializer: Serializer[Row] = encoder.createSerializer() + + private val deserializer: Deserializer[Row] = encoder.createDeserializer() + + def rowToInternal(row: Row): SparkInternalRow = { + serializer(row) + } + + def internalToRow(internalRow: SparkInternalRow): Row = { + deserializer(internalRow) + } +} + +sealed trait BucketProcessor { + def processPartition(rowIterator: Iterator[Row]): Iterator[Row] +} + +case class CommonBucketProcessor( + table: FileStoreTable, + bucketColIndex: Int, + encoderGroup: EncoderSerDeGroup) + extends BucketProcessor { + + def processPartition(rowIterator: Iterator[Row]): Iterator[Row] = { + val rowType = table.rowType() + val rowKeyExtractor = table.createRowKeyExtractor() + + def getBucketId(row: PaimonInternalRow): Int = { + rowKeyExtractor.setRecord(row) + rowKeyExtractor.bucket() + } + + new Iterator[Row] { + override def hasNext: Boolean = rowIterator.hasNext + + override def next(): Row = { + val row = rowIterator.next + val sparkInternalRow = encoderGroup.rowToInternal(row) + sparkInternalRow.setInt(bucketColIndex, getBucketId((new SparkRow(rowType, row)))) + encoderGroup.internalToRow(sparkInternalRow) + } + } + } +} + +case class DynamicBucketProcessor( + fileStoreTable: FileStoreTable, + bucketColIndex: Int, + numSparkPartitions: Int, + numAssigners: Int, + encoderGroup: EncoderSerDeGroup +) extends BucketProcessor { + + private val targetBucketRowNumber = fileStoreTable.coreOptions.dynamicBucketTargetRowNum + private val rowType = fileStoreTable.rowType + private val commitUser = UUID.randomUUID.toString + + def processPartition(rowIterator: Iterator[Row]): Iterator[Row] = { + val rowPartitionKeyExtractor = new RowPartitionKeyExtractor(fileStoreTable.schema) + val assigner = new HashBucketAssigner( + fileStoreTable.snapshotManager(), + commitUser, + fileStoreTable.store.newIndexFileHandler, + numSparkPartitions, + numAssigners, + TaskContext.getPartitionId(), + targetBucketRowNumber + ) + + new Iterator[Row]() { + override def hasNext: Boolean = rowIterator.hasNext + + override def next(): Row = { + val row = rowIterator.next + val sparkRow = new SparkRow(rowType, row) + val hash = rowPartitionKeyExtractor.trimmedPrimaryKey(sparkRow).hashCode + val partition = rowPartitionKeyExtractor.partition(sparkRow) + val bucket = assigner.assign(partition, hash) + val sparkInternalRow = encoderGroup.rowToInternal(row) + sparkInternalRow.setInt(bucketColIndex, bucket) + encoderGroup.internalToRow(sparkInternalRow) + } + } + } +} + +case class UnawareBucketProcessor(bucketColIndex: Int, encoderGroup: EncoderSerDeGroup) + extends BucketProcessor { + + def processPartition(rowIterator: Iterator[Row]): Iterator[Row] = { + new Iterator[Row] { + override def hasNext: Boolean = rowIterator.hasNext + + override def next(): Row = { + val row = rowIterator.next + val sparkInternalRow = encoderGroup.rowToInternal(row) + sparkInternalRow.setInt(bucketColIndex, 0) + encoderGroup.internalToRow(sparkInternalRow) + } + } + } +} 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 d3d8972e32ba..383e5c2cb1bc 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 @@ -20,14 +20,10 @@ package org.apache.paimon.spark.commands import org.apache.paimon.predicate.{Predicate, PredicateBuilder} import org.apache.paimon.spark.SparkFilterConverter -import org.apache.paimon.table.{BucketMode, FileStoreTable} -import org.apache.paimon.table.sink.{CommitMessage, CommitMessageSerializer} import org.apache.paimon.types.RowType -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.Utils.{normalizeExprs, translateFilter} import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, PredicateHelper} -import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.sources.{AlwaysTrue, And, EqualNullSafe, Filter} import java.io.IOException @@ -35,24 +31,6 @@ import java.io.IOException /** Helper trait for all paimon commands. */ trait PaimonCommand extends WithFileStoreTable with PredicateHelper { - lazy val bucketMode: BucketMode = table match { - case fileStoreTable: FileStoreTable => - fileStoreTable.bucketMode - case _ => - BucketMode.FIXED - } - - def deserializeCommitMessage( - serializer: CommitMessageSerializer, - bytes: Array[Byte]): CommitMessage = { - try { - serializer.deserialize(serializer.getVersion, bytes) - } catch { - case e: IOException => - throw new RuntimeException("Failed to deserialize CommitMessage's object", e) - } - } - protected def convertConditionToPaimonPredicate( condition: Expression, output: Seq[Attribute]): Predicate = { @@ -98,7 +76,7 @@ trait PaimonCommand extends WithFileStoreTable with PredicateHelper { }.toMap } - def splitConjunctiveFilters(filter: Filter): Seq[Filter] = { + private def splitConjunctiveFilters(filter: Filter): Seq[Filter] = { filter match { case And(filter1, filter2) => splitConjunctiveFilters(filter1) ++ splitConjunctiveFilters(filter2) @@ -106,7 +84,7 @@ trait PaimonCommand extends WithFileStoreTable with PredicateHelper { } } - def isNestedFilterInValue(value: Any): Boolean = { + private def isNestedFilterInValue(value: Any): Boolean = { value.isInstanceOf[Filter] } 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 new file mode 100644 index 000000000000..edccb4989a8c --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonSparkWriter.scala @@ -0,0 +1,214 @@ +/* + * 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.commands + +import org.apache.paimon.index.BucketAssigner +import org.apache.paimon.spark.SparkRow +import org.apache.paimon.spark.SparkUtils.createIOManager +import org.apache.paimon.spark.schema.SparkSystemColumns +import org.apache.paimon.spark.schema.SparkSystemColumns.{BUCKET_COL, ROW_KIND_COL} +import org.apache.paimon.spark.util.SparkRowUtils +import org.apache.paimon.table.{BucketMode, FileStoreTable} +import org.apache.paimon.table.sink.{BatchWriteBuilder, CommitMessage, CommitMessageSerializer, RowPartitionKeyExtractor} + +import org.apache.spark.Partitioner +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.functions._ + +import java.io.IOException + +import scala.collection.JavaConverters._ + +trait PaimonSparkWriter extends WithFileStoreTable { + + private lazy val tableSchema = table.schema + + private lazy val rowType = table.rowType() + + private lazy val bucketMode = table match { + case fileStoreTable: FileStoreTable => + fileStoreTable.bucketMode + case _ => + BucketMode.FIXED + } + + private lazy val primaryKeyCols = tableSchema.trimmedPrimaryKeys().asScala + + private lazy val serializer = new CommitMessageSerializer + + def write(data: Dataset[_], writeBuilder: BatchWriteBuilder): Seq[CommitMessage] = { + val sparkSession = data.sparkSession + import sparkSession.implicits._ + + val dataSchema = SparkSystemColumns.filterSparkSystemColumns(data.schema) + val rowkindColIdx = SparkRowUtils.getFieldIndex(data.schema, ROW_KIND_COL) + + // append _bucket_ column as placeholder + val withInitBucketCol = data.withColumn(BUCKET_COL, lit(-1)) + val bucketColIdx = withInitBucketCol.schema.size - 1 + + val originEncoderGroup = EncoderSerDeGroup(dataSchema) + val encoderGroupWithBucketCol = EncoderSerDeGroup(withInitBucketCol.schema) + + val withBucketCol = + assignBucketId(sparkSession, withInitBucketCol, bucketColIdx, encoderGroupWithBucketCol) + + val commitMessages = withBucketCol + .mapPartitions { + iter => + val ioManager = createIOManager + val write = writeBuilder.newWrite() + write.withIOManager(ioManager) + try { + iter.foreach { + row => + val bucket = row.getInt(bucketColIdx) + val bucketColDropped = + originEncoderGroup.internalToRow(encoderGroupWithBucketCol.rowToInternal(row)) + val sparkRow = new SparkRow( + rowType, + bucketColDropped, + SparkRowUtils.getRowKind(row, rowkindColIdx)) + write.write(sparkRow, bucket) + } + val serializer = new CommitMessageSerializer + write.prepareCommit().asScala.map(serializer.serialize).toIterator + + } finally { + write.close() + ioManager.close() + } + } + .collect() + .map(deserializeCommitMessage(serializer, _)) + + commitMessages.toSeq + } + + /** assign a valid bucket id for each of record. */ + private def assignBucketId( + sparkSession: SparkSession, + withInitBucketCol: DataFrame, + bucketColIdx: Int, + encoderGroupWithBucketCol: EncoderSerDeGroup): Dataset[Row] = { + + val encoderWithBucketCOl = encoderGroupWithBucketCol.encoder + + bucketMode match { + case BucketMode.DYNAMIC => + assert(primaryKeyCols.nonEmpty, "Only primary-key table can support dynamic bucket.") + + // Topology: input -> shuffle by special key & partition hash -> bucket-assigner -> shuffle by partition & bucket + val numParallelism = Option(table.coreOptions.dynamicBucketAssignerParallelism) + .map(_.toInt) + .getOrElse(sparkSession.sparkContext.defaultParallelism) + val numAssigners = Option(table.coreOptions.dynamicBucketInitialBuckets) + .map(initialBuckets => Math.min(initialBuckets.toInt, numParallelism)) + .getOrElse(numParallelism) + + val partitioned = + repartitionByKeyPartitionHash( + sparkSession, + withInitBucketCol, + numParallelism, + numAssigners) + val dynamicBucketProcessor = + DynamicBucketProcessor( + table, + bucketColIdx, + numParallelism, + numAssigners, + encoderGroupWithBucketCol) + repartitionByPartitionsAndBucket( + partitioned.mapPartitions(dynamicBucketProcessor.processPartition)(encoderWithBucketCOl)) + + case BucketMode.UNAWARE => + assert(primaryKeyCols.isEmpty, "Only append table can support unaware bucket.") + + // Topology: input -> bucket-assigner + val unawareBucketProcessor = UnawareBucketProcessor(bucketColIdx, encoderGroupWithBucketCol) + withInitBucketCol + .mapPartitions(unawareBucketProcessor.processPartition)(encoderWithBucketCOl) + .toDF() + + case BucketMode.FIXED => + // Topology: input -> bucket-assigner -> shuffle by partition & bucket + val commonBucketProcessor = + CommonBucketProcessor(table, bucketColIdx, encoderGroupWithBucketCol) + repartitionByPartitionsAndBucket( + withInitBucketCol.mapPartitions(commonBucketProcessor.processPartition)( + encoderWithBucketCOl)) + + case _ => + throw new UnsupportedOperationException(s"Spark doesn't support $bucketMode mode.") + } + } + + /** Compute bucket id in dynamic bucket mode. */ + private def repartitionByKeyPartitionHash( + sparkSession: SparkSession, + data: DataFrame, + numParallelism: Int, + numAssigners: Int): DataFrame = { + + sparkSession.createDataFrame( + data.rdd + .mapPartitions( + iterator => { + val rowPartitionKeyExtractor = new RowPartitionKeyExtractor(tableSchema) + iterator.map( + row => { + val sparkRow = new SparkRow(rowType, row) + val partitionHash = rowPartitionKeyExtractor.partition(sparkRow).hashCode + val keyHash = rowPartitionKeyExtractor.trimmedPrimaryKey(sparkRow).hashCode + ( + BucketAssigner + .computeHashKey(partitionHash, keyHash, numParallelism, numAssigners), + row) + }) + }, + preservesPartitioning = true + ) + .partitionBy(ModPartitioner(numParallelism)) + .map(_._2), + data.schema + ) + } + + private def repartitionByPartitionsAndBucket(ds: Dataset[Row]): Dataset[Row] = { + val partitionCols = tableSchema.partitionKeys().asScala.map(col) + ds.toDF().repartition(partitionCols ++ Seq(col(BUCKET_COL)): _*) + } + + private def deserializeCommitMessage( + serializer: CommitMessageSerializer, + bytes: Array[Byte]): CommitMessage = { + try { + serializer.deserialize(serializer.getVersion, bytes) + } catch { + case e: IOException => + throw new RuntimeException("Failed to deserialize CommitMessage's object", e) + } + } + + case class ModPartitioner(partitions: Int) extends Partitioner { + override def numPartitions: Int = partitions + override def getPartition(key: Any): Int = key.asInstanceOf[Int] % numPartitions + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SchemaHelper.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SchemaHelper.scala index 5c1064498188..fdabe6c4366d 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SchemaHelper.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SchemaHelper.scala @@ -18,7 +18,6 @@ package org.apache.paimon.spark.commands -import org.apache.paimon.schema.{SchemaMergingUtils, TableSchema} import org.apache.paimon.spark.SparkTypeUtils import org.apache.paimon.table.FileStoreTable import org.apache.paimon.types.RowType @@ -35,8 +34,6 @@ private[spark] trait SchemaHelper extends WithFileStoreTable { override def table: FileStoreTable = newTable.getOrElse(originTable) - def tableSchema: TableSchema = table.schema - def mergeAndCommitSchema(dataSchema: StructType, allowExplicitCast: Boolean): Unit = { val dataRowType = SparkTypeUtils.toPaimonType(dataSchema).asInstanceOf[RowType] if (table.store().mergeSchema(dataRowType, allowExplicitCast)) { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/WriteIntoPaimonTable.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/WriteIntoPaimonTable.scala index b9c4b3c903c7..b3077eb8a6a0 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/WriteIntoPaimonTable.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/WriteIntoPaimonTable.scala @@ -19,26 +19,15 @@ package org.apache.paimon.spark.commands import org.apache.paimon.CoreOptions.DYNAMIC_PARTITION_OVERWRITE -import org.apache.paimon.index.{BucketAssigner, HashBucketAssigner} import org.apache.paimon.options.Options import org.apache.paimon.spark._ -import org.apache.paimon.spark.SparkUtils.createIOManager import org.apache.paimon.spark.schema.SparkSystemColumns -import org.apache.paimon.spark.schema.SparkSystemColumns.{BUCKET_COL, ROW_KIND_COL} -import org.apache.paimon.spark.util.{EncoderUtils, SparkRowUtils} -import org.apache.paimon.table.{BucketMode, FileStoreTable} -import org.apache.paimon.table.sink.{BatchWriteBuilder, CommitMessageSerializer, RowPartitionKeyExtractor} -import org.apache.paimon.types.RowType +import org.apache.paimon.table.FileStoreTable -import org.apache.spark.{Partitioner, TaskContext} import org.apache.spark.internal.Logging -import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.functions._ - -import java.util.UUID import scala.collection.JavaConverters._ @@ -50,21 +39,15 @@ case class WriteIntoPaimonTable( options: Options) extends RunnableCommand with PaimonCommand + with PaimonSparkWriter with SchemaHelper with Logging { - import WriteIntoPaimonTable._ - - private lazy val serializer = new CommitMessageSerializer - private lazy val mergeSchema = options.get(SparkConnectorOptions.MERGE_SCHEMA) override def run(sparkSession: SparkSession): Seq[Row] = { - import sparkSession.implicits._ - - val dataSchema = SparkSystemColumns.filterSparkSystemColumns(data.schema) - if (mergeSchema) { + val dataSchema = SparkSystemColumns.filterSparkSystemColumns(data.schema) val allowExplicitCast = options.get(SparkConnectorOptions.EXPLICIT_CAST) mergeAndCommitSchema(dataSchema, allowExplicitCast) } @@ -74,136 +57,12 @@ case class WriteIntoPaimonTable( updateTableWithOptions( Map(DYNAMIC_PARTITION_OVERWRITE.key -> dynamicPartitionOverwriteMode.toString)) - val primaryKeyCols = tableSchema.trimmedPrimaryKeys().asScala.map(col) - val partitionCols = tableSchema.partitionKeys().asScala.map(col) - - val dataEncoder = EncoderUtils.encode(dataSchema).resolveAndBind() - val originFromRow = dataEncoder.createDeserializer() - - val rowkindColIdx = SparkRowUtils.getFieldIndex(data.schema, ROW_KIND_COL) - - // append _bucket_ column as placeholder - val withBucketCol = data.withColumn(BUCKET_COL, lit(-1)) - val bucketColIdx = withBucketCol.schema.size - 1 - val withBucketDataEncoder = EncoderUtils.encode(withBucketCol.schema).resolveAndBind() - val toRow = withBucketDataEncoder.createSerializer() - val fromRow = withBucketDataEncoder.createDeserializer() - - def repartitionByKeyPartitionHash( - input: DataFrame, - sparkRowType: RowType, - numParallelism: Int, - numAssigners: Int) = { - sparkSession.createDataFrame( - input.rdd - .mapPartitions( - iterator => { - val rowPartitionKeyExtractor = new RowPartitionKeyExtractor(table.schema) - iterator.map( - row => { - val sparkRow = new SparkRow(sparkRowType, row) - val partitionHash = rowPartitionKeyExtractor.partition(sparkRow).hashCode - val keyHash = rowPartitionKeyExtractor.trimmedPrimaryKey(sparkRow).hashCode - ( - BucketAssigner - .computeHashKey(partitionHash, keyHash, numParallelism, numAssigners), - row) - }) - }, - preservesPartitioning = true - ) - .partitionBy(ModPartitioner(numParallelism)) - .map(_._2), - withBucketCol.schema - ) - } - - def repartitionByBucket(ds: Dataset[Row]) = { - ds.toDF().repartition(partitionCols ++ Seq(col(BUCKET_COL)): _*) - } - - val rowType = table.rowType() val writeBuilder = table.newBatchWriteBuilder() - if (overwritePartition != null) { writeBuilder.withOverwrite(overwritePartition.asJava) } - val df = - bucketMode match { - case BucketMode.DYNAMIC => - // Topology: input -> shuffle by special key & partition hash -> bucket-assigner -> shuffle by partition & bucket - val numParallelism = - if (table.coreOptions.dynamicBucketAssignerParallelism.!=(null)) - table.coreOptions.dynamicBucketAssignerParallelism.toInt - else sparkSession.sparkContext.defaultParallelism - val numAssigners = - if (table.coreOptions().dynamicBucketInitialBuckets.!=(null)) - Math.min(table.coreOptions().dynamicBucketInitialBuckets.toInt, numParallelism) - else numParallelism - - val partitioned = if (primaryKeyCols.nonEmpty) { - repartitionByKeyPartitionHash(withBucketCol, rowType, numParallelism, numAssigners) - } else { - withBucketCol - } - val dynamicBucketProcessor = - DynamicBucketProcessor( - table, - rowType, - bucketColIdx, - numParallelism, - numAssigners, - toRow, - fromRow) - repartitionByBucket( - partitioned.mapPartitions(dynamicBucketProcessor.processPartition)( - withBucketDataEncoder)) - case BucketMode.UNAWARE => - // Topology: input -> bucket-assigner - val unawareBucketProcessor = UnawareBucketProcessor(bucketColIdx, toRow, fromRow) - withBucketCol - .mapPartitions(unawareBucketProcessor.processPartition)(withBucketDataEncoder) - .toDF() - case BucketMode.FIXED => - // Topology: input -> bucket-assigner -> shuffle by partition & bucket - val commonBucketProcessor = - CommonBucketProcessor(writeBuilder, bucketColIdx, toRow, fromRow) - repartitionByBucket( - withBucketCol.mapPartitions(commonBucketProcessor.processPartition)( - withBucketDataEncoder)) - case _ => - throw new UnsupportedOperationException( - s"Write with bucket mode $bucketMode is not supported") - } - - val commitMessages = df - .mapPartitions { - iter => - val ioManager = createIOManager - val write = writeBuilder.newWrite() - write.withIOManager(ioManager) - try { - iter.foreach { - row => - val bucket = row.getInt(bucketColIdx) - val bucketColDropped = originFromRow(toRow(row)) - val sparkRow = new SparkRow( - rowType, - bucketColDropped, - SparkRowUtils.getRowKind(row, rowkindColIdx)) - write.write(sparkRow, bucket) - } - val serializer = new CommitMessageSerializer - write.prepareCommit().asScala.map(serializer.serialize).toIterator - } finally { - write.close() - ioManager.close() - } - } - .collect() - .map(deserializeCommitMessage(serializer, _)) - + val commitMessages = write(data, writeBuilder) val tableCommit = writeBuilder.newCommit() try { tableCommit.commit(commitMessages.toList.asJava) @@ -241,104 +100,3 @@ case class WriteIntoPaimonTable( this.asInstanceOf[WriteIntoPaimonTable] } - -object WriteIntoPaimonTable { - - sealed private trait BucketProcessor { - def processPartition(rowIterator: Iterator[Row]): Iterator[Row] - } - - private case class CommonBucketProcessor( - writeBuilder: BatchWriteBuilder, - bucketColIndex: Int, - toRow: ExpressionEncoder.Serializer[Row], - fromRow: ExpressionEncoder.Deserializer[Row]) - extends BucketProcessor { - - private val rowType = writeBuilder.rowType - - def processPartition(rowIterator: Iterator[Row]): Iterator[Row] = { - val batchTableWrite = writeBuilder.newWrite() - new Iterator[Row] { - override def hasNext: Boolean = rowIterator.hasNext - - override def next(): Row = { - val row = rowIterator.next - val sparkInternalRow = toRow(row) - sparkInternalRow.setInt( - bucketColIndex, - batchTableWrite.getBucket(new SparkRow(rowType, row))) - fromRow(sparkInternalRow) - } - } - } - } - - private case class DynamicBucketProcessor( - fileStoreTable: FileStoreTable, - rowType: RowType, - bucketColIndex: Int, - numSparkPartitions: Int, - numAssigners: Int, - toRow: ExpressionEncoder.Serializer[Row], - fromRow: ExpressionEncoder.Deserializer[Row] - ) extends BucketProcessor { - - private val targetBucketRowNumber = fileStoreTable.coreOptions.dynamicBucketTargetRowNum - private val commitUser = UUID.randomUUID.toString - - def processPartition(rowIterator: Iterator[Row]): Iterator[Row] = { - val rowPartitionKeyExtractor = new RowPartitionKeyExtractor(fileStoreTable.schema) - val assigner = new HashBucketAssigner( - fileStoreTable.snapshotManager(), - commitUser, - fileStoreTable.store.newIndexFileHandler, - numSparkPartitions, - numAssigners, - TaskContext.getPartitionId(), - targetBucketRowNumber - ) - - new Iterator[Row]() { - override def hasNext: Boolean = rowIterator.hasNext - - override def next(): Row = { - val row = rowIterator.next - val sparkRow = new SparkRow(rowType, row) - val hash = rowPartitionKeyExtractor.trimmedPrimaryKey(sparkRow).hashCode - val partition = rowPartitionKeyExtractor.partition(sparkRow) - val bucket = assigner.assign(partition, hash) - val sparkInternalRow = toRow(row) - sparkInternalRow.setInt(bucketColIndex, bucket) - fromRow(sparkInternalRow) - } - } - } - } - - private case class UnawareBucketProcessor( - bucketColIndex: Int, - toRow: ExpressionEncoder.Serializer[Row], - fromRow: ExpressionEncoder.Deserializer[Row]) - extends BucketProcessor { - - def processPartition(rowIterator: Iterator[Row]): Iterator[Row] = { - new Iterator[Row] { - override def hasNext: Boolean = rowIterator.hasNext - - override def next(): Row = { - val row = rowIterator.next - val sparkInternalRow = toRow(row) - sparkInternalRow.setInt(bucketColIndex, 0) - fromRow(sparkInternalRow) - } - } - } - } - - private case class ModPartitioner(partitions: Int) extends Partitioner { - override def numPartitions: Int = partitions - - override def getPartition(key: Any): Int = key.asInstanceOf[Int] % numPartitions - } -} diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala index a8514434c614..d0438f04b48e 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala @@ -102,6 +102,6 @@ class DynamicBucketTableTest extends PaimonSparkTestBase { val error = intercept[UnsupportedOperationException] { spark.sql("INSERT INTO T VALUES ('1', 'a', 'p')") }.getMessage - assert(error.contains("Write with bucket mode GLOBAL_DYNAMIC is not supported")) + assert(error.contains("Spark doesn't support GLOBAL_DYNAMIC mode")) } } From 4a7c5d75fc33888a9bc8aebfc889103dd8be6352 Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Fri, 22 Mar 2024 18:52:52 +0800 Subject: [PATCH 47/79] [flink] Fix that compact actions haven't handled scan parallelism (#3077) --- .../flink/source/CompactorSourceBuilder.java | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) 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 70b8a772a608..e963c92fc2c4 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 @@ -118,11 +118,18 @@ public DataStreamSource build() { BucketsTable bucketsTable = new BucketsTable(table, isContinuous); RowType produceType = bucketsTable.rowType(); - return env.fromSource( - buildSource(bucketsTable), - WatermarkStrategy.noWatermarks(), - tableIdentifier + "-compact-source", - InternalTypeInfo.of(LogicalTypeConversion.toLogicalType(produceType))); + DataStreamSource dataStream = + env.fromSource( + buildSource(bucketsTable), + WatermarkStrategy.noWatermarks(), + tableIdentifier + "-compact-source", + InternalTypeInfo.of(LogicalTypeConversion.toLogicalType(produceType))); + Integer parallelism = + Options.fromMap(table.options()).get(FlinkConnectorOptions.SCAN_PARALLELISM); + if (parallelism != null) { + dataStream.setParallelism(parallelism); + } + return dataStream; } private Map streamingCompactOptions() { From 92cb24b7f8987b44f90b8e45ebdce1c1b2d1e1e7 Mon Sep 17 00:00:00 2001 From: Xiaojian Sun Date: Fri, 22 Mar 2024 19:04:33 +0800 Subject: [PATCH 48/79] [core] JdbcCatalog supports custom configuration for the length of lock table (#3069) --- docs/content/how-to/creating-catalogs.md | 3 +++ .../generated/jdbc_catalog_configuration.html | 6 +++++ .../jdbc/AbstractDistributedLockDialect.java | 10 ++++++-- .../org/apache/paimon/jdbc/JdbcCatalog.java | 23 +++++++++---------- .../paimon/jdbc/JdbcCatalogFactory.java | 2 +- .../apache/paimon/jdbc/JdbcCatalogLock.java | 10 ++++---- .../paimon/jdbc/JdbcCatalogOptions.java | 12 ++++++++++ .../jdbc/JdbcDistributedLockDialect.java | 5 +++- .../org/apache/paimon/jdbc/JdbcUtils.java | 6 +++-- .../jdbc/MysqlDistributedLockDialect.java | 2 +- .../jdbc/SqlLiteDistributedLockDialect.java | 3 +-- .../apache/paimon/jdbc/JdbcCatalogTest.java | 5 +++- 12 files changed, 60 insertions(+), 27 deletions(-) diff --git a/docs/content/how-to/creating-catalogs.md b/docs/content/how-to/creating-catalogs.md index 43d23dc1f51a..536a6b165ff7 100644 --- a/docs/content/how-to/creating-catalogs.md +++ b/docs/content/how-to/creating-catalogs.md @@ -210,8 +210,11 @@ You can configure any connection parameters that have been declared by JDBC thro You can also perform logical isolation for databases under multiple catalogs by specifying "catalog-key". +Additionally, when creating a JdbcCatalog, you can specify the maximum length for the lock key by configuring "lock-key-max-length," which defaults to 255. Since this value is a combination of {catalog-key}.{database-name}.{table-name}, please adjust accordingly. + You can define any default table options with the prefix `table-default.` for tables created in the catalog. + {{< /tab >}} {{< /tabs >}} diff --git a/docs/layouts/shortcodes/generated/jdbc_catalog_configuration.html b/docs/layouts/shortcodes/generated/jdbc_catalog_configuration.html index 87939512b5f8..c2d6e287f31e 100644 --- a/docs/layouts/shortcodes/generated/jdbc_catalog_configuration.html +++ b/docs/layouts/shortcodes/generated/jdbc_catalog_configuration.html @@ -32,5 +32,11 @@ String Custom jdbc catalog store key. + +
lock-key-max-length
+ 255 + Integer + Set the maximum length of the lock key. The 'lock-key' is composed of concatenating three fields : 'catalog-key', 'database', and 'table'. + diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/AbstractDistributedLockDialect.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/AbstractDistributedLockDialect.java index f3469d0b5ba5..b8bf3e62d853 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/AbstractDistributedLockDialect.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/AbstractDistributedLockDialect.java @@ -18,6 +18,8 @@ package org.apache.paimon.jdbc; +import org.apache.paimon.options.Options; + import java.sql.DatabaseMetaData; import java.sql.PreparedStatement; import java.sql.ResultSet; @@ -27,7 +29,9 @@ public abstract class AbstractDistributedLockDialect implements JdbcDistributedLockDialect { @Override - public void createTable(JdbcClientPool connections) throws SQLException, InterruptedException { + public void createTable(JdbcClientPool connections, Options options) + throws SQLException, InterruptedException { + Integer lockKeyMaxLength = JdbcCatalogOptions.lockKeyMaxLength(options); connections.run( conn -> { DatabaseMetaData dbMeta = conn.getMetaData(); @@ -37,7 +41,9 @@ public void createTable(JdbcClientPool connections) throws SQLException, Interru if (tableExists.next()) { return true; } - return conn.prepareStatement(getCreateTableSql()).execute(); + String createDistributedLockTableSql = + String.format(getCreateTableSql(), lockKeyMaxLength); + return conn.prepareStatement(createDistributedLockTableSql).execute(); }); } diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java index 689a93ee91f7..91c11ac24828 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java @@ -71,24 +71,20 @@ public class JdbcCatalog extends AbstractCatalog { private final JdbcClientPool connections; private final String catalogKey; - private final Map options; + private final Options options; private final String warehouse; - protected JdbcCatalog( - FileIO fileIO, String catalogKey, Map config, String warehouse) { - super(fileIO, Options.fromMap(config)); + protected JdbcCatalog(FileIO fileIO, String catalogKey, Options options, String warehouse) { + super(fileIO, options); this.catalogKey = catalogKey; - this.options = config; + this.options = options; this.warehouse = warehouse; Preconditions.checkNotNull(options, "Invalid catalog properties: null"); this.connections = new JdbcClientPool( - Integer.parseInt( - config.getOrDefault( - CatalogOptions.CLIENT_POOL_SIZE.key(), - CatalogOptions.CLIENT_POOL_SIZE.defaultValue().toString())), + options.get(CatalogOptions.CLIENT_POOL_SIZE), options.get(CatalogOptions.URI.key()), - options); + options.toMap()); try { initializeCatalogTablesIfNeed(); } catch (SQLException e) { @@ -135,7 +131,7 @@ private void initializeCatalogTablesIfNeed() throws SQLException, InterruptedExc // if lock enabled, Check and create distributed lock table. if (lockEnabled()) { - JdbcUtils.createDistributedLockTable(connections); + JdbcUtils.createDistributedLockTable(connections, options); } } @@ -357,7 +353,10 @@ private Lock lock(Identifier identifier) { } JdbcCatalogLock lock = new JdbcCatalogLock( - connections, catalogKey, checkMaxSleep(options), acquireTimeout(options)); + connections, + catalogKey, + checkMaxSleep(options.toMap()), + acquireTimeout(options.toMap())); return Lock.fromCatalog(lock, identifier); } diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogFactory.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogFactory.java index 5e605923206b..adaaf3f43632 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogFactory.java @@ -47,6 +47,6 @@ public Catalog create(FileIO fileIO, Path warehouse, CatalogContext context) { options.set(LOCK_TYPE, JdbcCatalogLock.JdbcCatalogLockFactory.IDENTIFIER); } } - return new JdbcCatalog(fileIO, catalogKey, context.options().toMap(), warehouse.toString()); + return new JdbcCatalog(fileIO, catalogKey, context.options(), warehouse.toString()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java index d135a6caf22b..d713feb7e49f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java @@ -19,6 +19,7 @@ package org.apache.paimon.jdbc; import org.apache.paimon.catalog.CatalogLock; +import org.apache.paimon.options.Options; import org.apache.paimon.utils.TimeUtils; import java.io.IOException; @@ -103,18 +104,17 @@ public CatalogLock create(LockContext context) { return new JdbcCatalogLock( lockContext.connections, lockContext.catalogKey, - checkMaxSleep(lockContext.conf), - acquireTimeout(lockContext.conf)); + checkMaxSleep(lockContext.conf.toMap()), + acquireTimeout(lockContext.conf.toMap())); } } static class JdbcLockContext implements LockContext { private final JdbcClientPool connections; private final String catalogKey; - private final Map conf; + private final Options conf; - public JdbcLockContext( - JdbcClientPool connections, String catalogKey, Map conf) { + public JdbcLockContext(JdbcClientPool connections, String catalogKey, Options conf) { this.connections = connections; this.catalogKey = catalogKey; this.conf = conf; diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogOptions.java index 97828f7c8d7d..407dbbc3bf5b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogOptions.java @@ -20,6 +20,7 @@ import org.apache.paimon.options.ConfigOption; import org.apache.paimon.options.ConfigOptions; +import org.apache.paimon.options.Options; /** Options for jdbc catalog. */ public final class JdbcCatalogOptions { @@ -30,5 +31,16 @@ public final class JdbcCatalogOptions { .defaultValue("jdbc") .withDescription("Custom jdbc catalog store key."); + public static final ConfigOption LOCK_KEY_MAX_LENGTH = + ConfigOptions.key("lock-key-max-length") + .intType() + .defaultValue(255) + .withDescription( + "Set the maximum length of the lock key. The 'lock-key' is composed of concatenating three fields : 'catalog-key', 'database', and 'table'."); + private JdbcCatalogOptions() {} + + static Integer lockKeyMaxLength(Options options) { + return options.get(LOCK_KEY_MAX_LENGTH); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcDistributedLockDialect.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcDistributedLockDialect.java index a691aac2295f..1252dc905d6b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcDistributedLockDialect.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcDistributedLockDialect.java @@ -18,11 +18,14 @@ package org.apache.paimon.jdbc; +import org.apache.paimon.options.Options; + import java.sql.SQLException; /** Jdbc distributed lock interface. */ public interface JdbcDistributedLockDialect { - void createTable(JdbcClientPool connections) throws SQLException, InterruptedException; + void createTable(JdbcClientPool connections, Options options) + throws SQLException, InterruptedException; boolean lockAcquire(JdbcClientPool connections, String lockId, long timeoutMillSeconds) throws SQLException, InterruptedException; diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java index 7b9b93a5a4e2..4acb0f25aa91 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcUtils.java @@ -19,6 +19,7 @@ package org.apache.paimon.jdbc; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.options.Options; import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; @@ -402,9 +403,10 @@ private static String insertPropertiesStatement(int size) { return sqlStatement.toString(); } - public static void createDistributedLockTable(JdbcClientPool connections) + public static void createDistributedLockTable(JdbcClientPool connections, Options options) throws SQLException, InterruptedException { - DistributedLockDialectFactory.create(connections.getProtocol()).createTable(connections); + DistributedLockDialectFactory.create(connections.getProtocol()) + .createTable(connections, options); } public static boolean acquire( diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/MysqlDistributedLockDialect.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/MysqlDistributedLockDialect.java index 206aa8cd77ad..bca8aac20b86 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/MysqlDistributedLockDialect.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/MysqlDistributedLockDialect.java @@ -27,7 +27,7 @@ public String getCreateTableSql() { + JdbcUtils.DISTRIBUTED_LOCKS_TABLE_NAME + "(" + JdbcUtils.LOCK_ID - + " VARCHAR(1000) NOT NULL," + + " VARCHAR(%s) NOT NULL," + JdbcUtils.ACQUIRED_AT + " TIMESTAMP DEFAULT CURRENT_TIMESTAMP NOT NULL," + JdbcUtils.EXPIRE_TIME diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/SqlLiteDistributedLockDialect.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/SqlLiteDistributedLockDialect.java index 602fdd1d625e..1ffdd61bc2f2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/SqlLiteDistributedLockDialect.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/SqlLiteDistributedLockDialect.java @@ -20,14 +20,13 @@ /** Distributed lock implementation based on sqlite table. */ public class SqlLiteDistributedLockDialect extends AbstractDistributedLockDialect { - @Override public String getCreateTableSql() { return "CREATE TABLE " + JdbcUtils.DISTRIBUTED_LOCKS_TABLE_NAME + "(" + JdbcUtils.LOCK_ID - + " VARCHAR(1000) NOT NULL," + + " VARCHAR(%s) NOT NULL," + JdbcUtils.ACQUIRED_AT + " TIMESTAMP DEFAULT CURRENT_TIMESTAMP NOT NULL," + JdbcUtils.EXPIRE_TIME diff --git a/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java index d03c64bd825e..cc1febeab023 100644 --- a/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/jdbc/JdbcCatalogTest.java @@ -21,6 +21,7 @@ import org.apache.paimon.catalog.CatalogTestBase; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.options.CatalogOptions; +import org.apache.paimon.options.Options; import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; @@ -57,7 +58,9 @@ private JdbcCatalog initCatalog(Map props) { properties.put(CatalogOptions.LOCK_ENABLED.key(), "true"); properties.put(CatalogOptions.LOCK_TYPE.key(), "jdbc"); properties.putAll(props); - JdbcCatalog catalog = new JdbcCatalog(fileIO, "test-jdbc-catalog", properties, warehouse); + JdbcCatalog catalog = + new JdbcCatalog( + fileIO, "test-jdbc-catalog", Options.fromMap(properties), warehouse); assertThat(catalog.warehouse()).isEqualTo(warehouse); return catalog; } From 48aa793a12ca2eaa86d61b044731a35606a2d707 Mon Sep 17 00:00:00 2001 From: wgcn <1026688210@qq.com> Date: Mon, 25 Mar 2024 09:48:23 +0800 Subject: [PATCH 49/79] [doc] Introduce the minimum version of Maven. (#3082) --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 50c487fbcb0f..d068ad217eec 100644 --- a/README.md +++ b/README.md @@ -68,7 +68,7 @@ You can join the Paimon community on Slack. Paimon channel is in ASF Slack works ## Building -JDK 8/11 is required for building the project. +JDK 8/11 is required for building the project. Maven version >=3.3.1. - Run the `mvn clean install -DskipTests` command to build the project. - Run the `mvn spotless:apply` to format the project (both Java and Scala). From f5e995ab4ee798d77eaa713f98d1799e58685a2a Mon Sep 17 00:00:00 2001 From: YeJunHao <41894543+leaves12138@users.noreply.github.com> Date: Mon, 25 Mar 2024 12:11:12 +0800 Subject: [PATCH 50/79] [doc] Apache Paimon became top level project. (#3085) --- .asf.yaml | 2 +- .github/ISSUE_TEMPLATE/bug-report.yml | 4 ++-- .github/ISSUE_TEMPLATE/feature.yml | 4 ++-- .github/workflows/publish_snapshot.yml | 2 +- DISCLAIMER | 2 +- NOTICE | 2 +- README.md | 2 +- docs/config.toml | 8 ++++---- docs/content/_index.md | 2 +- docs/content/concepts/overview.md | 2 +- docs/content/project/contributing.md | 12 ++++++------ docs/layouts/partials/docs/inject/footer.html | 16 +--------------- pom.xml | 8 ++++---- tools/ci/sonar_check.sh | 2 +- 14 files changed, 27 insertions(+), 41 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index 30bcffa6cfe1..b7c6ae27d579 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -18,7 +18,7 @@ # See: https://cwiki.apache.org/confluence/display/INFRA/git+-+.asf.yaml+features github: - description: "Apache Paimon(incubating) is a lake format that enables building a Realtime Lakehouse Architecture with Flink and Spark for both streaming and batch operations." + description: "Apache Paimon is a lake format that enables building a Realtime Lakehouse Architecture with Flink and Spark for both streaming and batch operations." homepage: https://paimon.apache.org/ labels: - paimon diff --git a/.github/ISSUE_TEMPLATE/bug-report.yml b/.github/ISSUE_TEMPLATE/bug-report.yml index f93892868abe..a187fb28c6b2 100644 --- a/.github/ISSUE_TEMPLATE/bug-report.yml +++ b/.github/ISSUE_TEMPLATE/bug-report.yml @@ -29,10 +29,10 @@ body: attributes: label: Search before asking description: > - Please search [issues](https://github.com/apache/incubator-paimon/issues) to check if your issue has already been reported. + Please search [issues](https://github.com/apache/paimon/issues) to check if your issue has already been reported. options: - label: > - I searched in the [issues](https://github.com/apache/incubator-paimon/issues) and found nothing similar. + I searched in the [issues](https://github.com/apache/paimon/issues) and found nothing similar. required: true - type: textarea attributes: diff --git a/.github/ISSUE_TEMPLATE/feature.yml b/.github/ISSUE_TEMPLATE/feature.yml index 3b0f34a04c10..43f40ff6fe01 100644 --- a/.github/ISSUE_TEMPLATE/feature.yml +++ b/.github/ISSUE_TEMPLATE/feature.yml @@ -29,10 +29,10 @@ body: attributes: label: Search before asking description: > - Please search [issues](https://github.com/apache/incubator-paimon/issues) to check if your issue has already been reported. + Please search [issues](https://github.com/apache/paimon/issues) to check if your issue has already been reported. options: - label: > - I searched in the [issues](https://github.com/apache/incubator-paimon/issues) and found nothing similar. + I searched in the [issues](https://github.com/apache/paimon/issues) and found nothing similar. required: true - type: textarea attributes: diff --git a/.github/workflows/publish_snapshot.yml b/.github/workflows/publish_snapshot.yml index 7fdfd12b2aa7..7d370814da94 100644 --- a/.github/workflows/publish_snapshot.yml +++ b/.github/workflows/publish_snapshot.yml @@ -33,7 +33,7 @@ concurrency: jobs: publish-snapshot: - if: github.repository == 'apache/incubator-paimon' + if: github.repository == 'apache/paimon' runs-on: ubuntu-latest steps: - name: Checkout code diff --git a/DISCLAIMER b/DISCLAIMER index 6681d18687f8..614607a64638 100644 --- a/DISCLAIMER +++ b/DISCLAIMER @@ -1,4 +1,4 @@ -Apache Paimon (incubating) is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Apache Incubator PMC. +Apache Paimon is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Apache PMC. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. diff --git a/NOTICE b/NOTICE index a06c276bc557..30bd8fc78e26 100644 --- a/NOTICE +++ b/NOTICE @@ -1,4 +1,4 @@ -Apache Paimon (incubating) +Apache Paimon Copyright 2023-2024 The Apache Software Foundation This product includes software developed at diff --git a/README.md b/README.md index d068ad217eec..5f95e40b73fe 100644 --- a/README.md +++ b/README.md @@ -3,7 +3,7 @@ [![License](https://img.shields.io/badge/license-Apache%202-4EB1BA.svg)](https://www.apache.org/licenses/LICENSE-2.0.html) [![Get on Slack](https://img.shields.io/badge/slack-join-orange.svg)](https://the-asf.slack.com/archives/C053Q2NCW8G) -Apache Paimon(incubating) is a lake format that enables building a Realtime Lakehouse Architecture with Flink and Spark +Apache Paimon is a lake format that enables building a Realtime Lakehouse Architecture with Flink and Spark for both streaming and batch operations. Paimon innovatively combines lake format and LSM structure, bringing realtime streaming updates into the lake architecture. diff --git a/docs/config.toml b/docs/config.toml index e259e412f580..83842f52b4de 100644 --- a/docs/config.toml +++ b/docs/config.toml @@ -44,13 +44,13 @@ pygmentsUseClasses = true Branch = "master" # The github repository for Apache Paimon - Repo = "//github.com/apache/incubator-paimon" + Repo = "//github.com/apache/paimon" - GithubRepo = "https://github.com/apache/incubator-paimon.git" + GithubRepo = "https://github.com/apache/paimon.git" - TrinoGithubRepo = "https://github.com/apache/incubator-paimon-trino.git" + TrinoGithubRepo = "https://github.com/apache/paimon-trino.git" - PrestoGithubRepo = "https://github.com/apache/incubator-paimon-presto.git" + PrestoGithubRepo = "https://github.com/apache/paimon-presto.git" # This suffix is appended to the Scala-dependent Maven artifact names ScalaVersion = "_2.12" diff --git a/docs/content/_index.md b/docs/content/_index.md index 160dc4f8a11f..e748a8f1bd4e 100644 --- a/docs/content/_index.md +++ b/docs/content/_index.md @@ -24,7 +24,7 @@ under the License. # Apache Paimon -Apache Paimon(incubating) is a lake format that enables building a Realtime Lakehouse Architecture with Flink and Spark +Apache Paimon is a lake format that enables building a Realtime Lakehouse Architecture with Flink and Spark for both streaming and batch operations. Paimon innovatively combines lake format and LSM (Log-structured merge-tree) structure, bringing realtime streaming updates into the lake architecture. diff --git a/docs/content/concepts/overview.md b/docs/content/concepts/overview.md index 269107861ae2..a55151ead55d 100644 --- a/docs/content/concepts/overview.md +++ b/docs/content/concepts/overview.md @@ -26,7 +26,7 @@ under the License. # Overview -Apache Paimon(incubating)'s Architecture: +Apache Paimon's Architecture: {{< img src="/img/architecture.png">}} diff --git a/docs/content/project/contributing.md b/docs/content/project/contributing.md index f3f17d347285..bfb750a700d2 100644 --- a/docs/content/project/contributing.md +++ b/docs/content/project/contributing.md @@ -26,7 +26,7 @@ under the License. # Contributing -Apache Paimon (incubating) is developed by an open and friendly community. Everybody is cordially welcome to join +Apache Paimon is developed by an open and friendly community. Everybody is cordially welcome to join the community and contribute to Apache Paimon. There are several ways to interact with the community and contribute to Paimon including asking questions, filing bug reports, proposing new features, joining discussions on the mailing lists, contributing code or documentation, improving website, testing release candidates and writing corresponding blog etc. @@ -44,7 +44,7 @@ lists, contributing code or documentation, improving website, testing release ca Report Bug - To report a problem with Paimon, open Paimon’s issues.
+ To report a problem with Paimon, open Paimon’s issues.
Please give detailed information about the problem you encountered and, if possible, add a description that helps to reproduce the problem. @@ -61,20 +61,20 @@ lists, contributing code or documentation, improving website, testing release ca Support Users - Reply to questions on the user mailing list, - check the latest issues in Issues for tickets which are actually user questions. + Reply to questions on the user mailing list, + check the latest issues in Issues for tickets which are actually user questions. Spread the Word About Paimon Organize or attend a Paimon Meetup, contribute to the Paimon blog, share your conference, meetup or blog - post on the dev@paimon.apache.org mailing list. + post on the dev@paimon.apache.org mailing list. Any other question? Reach out to the - dev@paimon.apache.org mailing list to get help! + dev@paimon.apache.org mailing list to get help! diff --git a/docs/layouts/partials/docs/inject/footer.html b/docs/layouts/partials/docs/inject/footer.html index e50ef2a5754e..963feefd1593 100644 --- a/docs/layouts/partials/docs/inject/footer.html +++ b/docs/layouts/partials/docs/inject/footer.html @@ -24,23 +24,9 @@ - - - - diff --git a/pom.xml b/pom.xml index eb30d0d74bc5..7f6dfe92ebb1 100644 --- a/pom.xml +++ b/pom.xml @@ -46,9 +46,9 @@ under the License. - https://github.com/apache/incubator-paimon - git@github.com:apache/incubator-paimon.git - scm:git:https://gitbox.apache.org/repos/asf/incubator-paimon.git + https://github.com/apache/paimon + git@github.com:apache/paimon.git + scm:git:https://gitbox.apache.org/repos/asf/paimon.git @@ -711,7 +711,7 @@ under the License. - Apache Paimon (incubating) + Apache Paimon ${project.inceptionYear} UTF-8 diff --git a/tools/ci/sonar_check.sh b/tools/ci/sonar_check.sh index 95673c3896d6..a3b774ef5440 100644 --- a/tools/ci/sonar_check.sh +++ b/tools/ci/sonar_check.sh @@ -21,7 +21,7 @@ fi mvn --batch-mode verify sonar:sonar \ -Dmaven.test.skip=true -Dsonar.host.url=https://sonarcloud.io \ -Dsonar.organization=apache \ --Dsonar.projectKey=apache_incubator-paimon \ +-Dsonar.projectKey=apache-paimon \ -Dsonar.core.codeCoveragePlugin=jacoco \ -Dsonar.coverage.jacoco.xmlReportPaths=target/site/jacoco/jacoco.xml \ -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false \ From 419d354e9e0563a83b1a77936c34fe69735a2219 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Mon, 25 Mar 2024 13:22:29 +0800 Subject: [PATCH 51/79] [incubating] Remove incubator disclaimer --- DISCLAIMER | 7 ------- pom.xml | 18 ------------------ 2 files changed, 25 deletions(-) delete mode 100644 DISCLAIMER diff --git a/DISCLAIMER b/DISCLAIMER deleted file mode 100644 index 614607a64638..000000000000 --- a/DISCLAIMER +++ /dev/null @@ -1,7 +0,0 @@ -Apache Paimon is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Apache PMC. - -Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, -and decision making process have stabilized in a manner consistent with other successful ASF projects. - -While incubation status is not necessarily a reflection of the completeness or stability of the code, -it does indicate that the project has yet to be fully endorsed by the ASF. diff --git a/pom.xml b/pom.xml index 7f6dfe92ebb1..fe09e347574b 100644 --- a/pom.xml +++ b/pom.xml @@ -951,24 +951,6 @@ under the License. - - - org.apache.maven.plugins - maven-remote-resources-plugin - 1.7.0 - - - - process - - - - org.apache.apache.resources:apache-incubator-disclaimer-resource-bundle:1.5 - - - - - From d6d2929704eb303bc7fa0d727d36cf578fbaa52b Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Mon, 25 Mar 2024 14:17:28 +0800 Subject: [PATCH 52/79] [flink] Compact procedure supports named arguments (#3078) --- docs/content/engines/flink.md | 33 +-- .../paimon/tests/FlinkProceduresE2eTest.java | 11 +- .../flink/procedure/CompactProcedure.java | 126 ++++++++++++ .../ProcedurePositionalArgumentsITCase.java | 51 +++++ .../flink/procedure/CompactProcedure.java | 72 +++---- .../paimon/flink/CatalogITCaseBase.java | 7 +- .../flink/action/CompactActionITCase.java | 34 +-- ...rtCompactActionForDynamicBucketITCase.java | 27 +-- ...rtCompactActionForUnawareBucketITCase.java | 32 +-- .../procedure/CompactProcedureITCase.java | 194 ++++++++++++++++++ 10 files changed, 440 insertions(+), 147 deletions(-) create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java create mode 100644 paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/procedure/ProcedurePositionalArgumentsITCase.java create mode 100644 paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java diff --git a/docs/content/engines/flink.md b/docs/content/engines/flink.md index 461e565cad6f..3a3abaf06cfa 100644 --- a/docs/content/engines/flink.md +++ b/docs/content/engines/flink.md @@ -324,17 +324,23 @@ SELECT * FROM T; Flink 1.18 and later versions support [Call Statements](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/sql/call/), which make it easier to manipulate data and metadata of Paimon table by writing SQLs instead of submitting Flink jobs. -All available procedures are listed below. Note that when you call a procedure, you must pass all parameters in order, -and if you don't want to pass some parameters, you must use `''` as placeholder. For example, if you want to compact -table `default.t` with parallelism 4, but you don't want to specify partitions and sort strategy, the call statement -should be \ + +In 1.18, the procedure only supports passing arguments by position. You must pass all arguments in order, and if you +don't want to pass some arguments, you must use `''` as placeholder. For example, if you want to compact table `default.t` +with parallelism 4, but you don't want to specify partitions and sort strategy, the call statement should be \ `CALL sys.compact('default.t', '', '', '', 'sink.parallelism=4')`. +In higher versions, the procedure supports passing arguments by name. You can pass arguments in any order and any optional +argument can be omitted. For the above example, the call statement is \ +``CALL sys.compact(`table` => 'default.t', options => 'sink.parallelism=4')``. + Specify partitions: we use string to represent partition filter. "," means "AND" and ";" means "OR". For example, if you want to specify two partitions date=01 and date=02, you need to write 'date=01;date=02'; If you want to specify one partition with date=01 and day=01, you need to write 'date=01,day=01'. -table options syntax: we use string to represent table options. The format is 'key1=value1,key2=value2...'. +Table options syntax: we use string to represent table options. The format is 'key1=value1,key2=value2...'. + +All available procedures are listed below.
- - - Apache Paimon is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the - Apache Incubator. - Incubation is required of all newly accepted projects until a further review indicates that the - infrastructure, communications, - and decision making process have stabilized in a manner consistent with other successful ASF projects. - While incubation status is not necessarily a reflection of the completeness or stability of the code, - it does indicate that the project has yet to be fully endorsed by the ASF. -
- Copyright © 2023 The Apache Software Foundation. Apache Paimon, Paimon, and its feather logo are + Copyright © 2024 The Apache Software Foundation. Apache Paimon, Paimon, and its feather logo are trademarks of The Apache Software Foundation.
@@ -349,20 +355,17 @@ table options syntax: we use string to represent table options. The format is 'k diff --git a/paimon-e2e-tests/src/test/java/org/apache/paimon/tests/FlinkProceduresE2eTest.java b/paimon-e2e-tests/src/test/java/org/apache/paimon/tests/FlinkProceduresE2eTest.java index 74d1a588f39c..c7759deaaaa8 100644 --- a/paimon-e2e-tests/src/test/java/org/apache/paimon/tests/FlinkProceduresE2eTest.java +++ b/paimon-e2e-tests/src/test/java/org/apache/paimon/tests/FlinkProceduresE2eTest.java @@ -93,9 +93,16 @@ public void testCompact() throws Exception { testDataSourceDdl); // execute compact procedure + String callStatement; + if (System.getProperty("test.flink.main.version").compareTo("1.18") == 0) { + callStatement = "CALL sys.compact('default.ts_table', 'dt=20221205;dt=20221206');"; + } else { + callStatement = + "CALL sys.compact(\\`table\\` => 'default.ts_table', partitions => 'dt=20221205;dt=20221206');"; + } + runSql( - "SET 'execution.checkpointing.interval' = '1s';\n" - + "CALL sys.compact('default.ts_table', 'dt=20221205;dt=20221206');", + "SET 'execution.checkpointing.interval' = '1s';\n" + callStatement, catalogDdl, useCatalogCmd); diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java new file mode 100644 index 000000000000..adf4adf2e09d --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.procedure; + +import org.apache.paimon.catalog.AbstractCatalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.flink.action.CompactAction; +import org.apache.paimon.flink.action.SortCompactAction; +import org.apache.paimon.utils.ParameterUtils; +import org.apache.paimon.utils.StringUtils; + +import org.apache.flink.table.procedure.ProcedureContext; + +import java.util.Collections; +import java.util.Map; + +/** + * Stay compatible with 1.18 procedure which doesn't support named argument. Usage: + * + *

+ *  -- NOTE: use '' as placeholder for optional arguments
+ *
+ *  -- compact a table (tableId should be 'database_name.table_name')
+ *  CALL sys.compact('tableId')
+ *
+ *  -- compact specific partitions ('pt1=A,pt2=a;pt1=B,pt2=b', ...)
+ *  CALL sys.compact('tableId', 'pt1=A,pt2=a;pt1=B,pt2=b')
+ *
+ *  -- compact a table with sorting
+ *  CALL sys.compact('tableId', 'partitions', 'ORDER/ZORDER', 'col1,col2', 'sink.parallelism=6')
+ *
+ * 
+ */ +public class CompactProcedure extends ProcedureBase { + + public static final String IDENTIFIER = "compact"; + + public String[] call(ProcedureContext procedureContext, String tableId) throws Exception { + return call(procedureContext, tableId, ""); + } + + public String[] call(ProcedureContext procedureContext, String tableId, String partitions) + throws Exception { + return call(procedureContext, tableId, partitions, "", "", ""); + } + + public String[] call( + ProcedureContext procedureContext, + String tableId, + String partitions, + String orderStrategy, + String orderByColumns) + throws Exception { + return call(procedureContext, tableId, partitions, orderStrategy, orderByColumns, ""); + } + + public String[] call( + ProcedureContext procedureContext, + String tableId, + String partitions, + String orderStrategy, + String orderByColumns, + String tableOptions) + throws Exception { + String warehouse = ((AbstractCatalog) catalog).warehouse(); + Map catalogOptions = ((AbstractCatalog) catalog).options(); + Map tableConf = + StringUtils.isBlank(tableOptions) + ? Collections.emptyMap() + : ParameterUtils.parseCommaSeparatedKeyValues(tableOptions); + Identifier identifier = Identifier.fromString(tableId); + CompactAction action; + String jobName; + if (orderStrategy.isEmpty() && orderByColumns.isEmpty()) { + action = + new CompactAction( + warehouse, + identifier.getDatabaseName(), + identifier.getObjectName(), + catalogOptions, + tableConf); + jobName = "Compact Job"; + } else if (!orderStrategy.isEmpty() && !orderByColumns.isEmpty()) { + action = + new SortCompactAction( + warehouse, + identifier.getDatabaseName(), + identifier.getObjectName(), + catalogOptions, + tableConf) + .withOrderStrategy(orderStrategy) + .withOrderColumns(orderByColumns.split(",")); + jobName = "Sort Compact Job"; + } else { + throw new IllegalArgumentException( + "You must specify 'order strategy' and 'order by columns' both."); + } + + if (!(StringUtils.isBlank(partitions))) { + action.withPartitions(ParameterUtils.getPartitions(partitions.split(";"))); + } + + return execute(procedureContext, action, jobName); + } + + @Override + public String identifier() { + return IDENTIFIER; + } +} 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 new file mode 100644 index 000000000000..d47947caa248 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/procedure/ProcedurePositionalArgumentsITCase.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.procedure; + +import org.apache.paimon.flink.CatalogITCaseBase; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThatCode; + +/** Ensure that the legacy multiply overloaded CALL with positional arguments can be invoked. */ +public class ProcedurePositionalArgumentsITCase extends CatalogITCaseBase { + + @Test + public void testCallCompact() { + sql( + "CREATE TABLE T (" + + " k INT," + + " v INT," + + " pt INT," + + " PRIMARY KEY (k, pt) NOT ENFORCED" + + ") PARTITIONED BY (pt) WITH (" + + " 'write-only' = 'true'," + + " 'bucket' = '1'" + + ")"); + + assertThatCode(() -> sql("CALL sys.compact('default.T')")).doesNotThrowAnyException(); + assertThatCode(() -> sql("CALL sys.compact('default.T', 'pt=1')")) + .doesNotThrowAnyException(); + assertThatCode(() -> sql("CALL sys.compact('default.T', 'pt=1', '', '')")) + .doesNotThrowAnyException(); + assertThatCode(() -> sql("CALL sys.compact('default.T', '', '', '', 'sink.parallelism=1')")) + .doesNotThrowAnyException(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java index ae8dfb07822b..8a78ad3b8f43 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java @@ -22,54 +22,38 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.action.CompactAction; import org.apache.paimon.flink.action.SortCompactAction; -import org.apache.paimon.utils.ParameterUtils; -import org.apache.paimon.utils.StringUtils; +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.util.Collections; import java.util.Map; -/** - * Compact procedure. Usage: - * - *

- *  -- NOTE: use '' as placeholder for optional arguments
- *
- *  -- compact a table (tableId should be 'database_name.table_name')
- *  CALL sys.compact('tableId')
- *
- *  -- compact specific partitions ('pt1=A,pt2=a;pt1=B,pt2=b', ...)
- *  CALL sys.compact('tableId', 'pt1=A,pt2=a;pt1=B,pt2=b')
- *
- *  -- compact a table with sorting
- *  CALL sys.compact('tableId', 'partitions', 'ORDER/ZORDER', 'col1,col2', 'sink.parallelism=6')
- *
- * 
- */ +import static org.apache.paimon.utils.ParameterUtils.getPartitions; +import static org.apache.paimon.utils.ParameterUtils.parseCommaSeparatedKeyValues; +import static org.apache.paimon.utils.StringUtils.isBlank; + +/** Compact procedure. */ public class CompactProcedure extends ProcedureBase { public static final String IDENTIFIER = "compact"; - public String[] call(ProcedureContext procedureContext, String tableId) throws Exception { - return call(procedureContext, tableId, ""); - } - - public String[] call(ProcedureContext procedureContext, String tableId, String partitions) - throws Exception { - return call(procedureContext, tableId, partitions, "", "", ""); - } - - public String[] call( - ProcedureContext procedureContext, - String tableId, - String partitions, - String orderStrategy, - String orderByColumns) - throws Exception { - return call(procedureContext, tableId, partitions, orderStrategy, orderByColumns, ""); - } - + @ProcedureHint( + argument = { + @ArgumentHint(name = "table", type = @DataTypeHint("STRING")), + @ArgumentHint( + name = "partitions", + type = @DataTypeHint("STRING"), + isOptional = true), + @ArgumentHint( + name = "order_strategy", + type = @DataTypeHint("STRING"), + isOptional = true), + @ArgumentHint(name = "order_by", type = @DataTypeHint("STRING"), isOptional = true), + @ArgumentHint(name = "options", type = @DataTypeHint("STRING"), isOptional = true) + }) public String[] call( ProcedureContext procedureContext, String tableId, @@ -81,13 +65,13 @@ public String[] call( String warehouse = ((AbstractCatalog) catalog).warehouse(); Map catalogOptions = ((AbstractCatalog) catalog).options(); Map tableConf = - StringUtils.isBlank(tableOptions) + isBlank(tableOptions) ? Collections.emptyMap() - : ParameterUtils.parseCommaSeparatedKeyValues(tableOptions); + : parseCommaSeparatedKeyValues(tableOptions); Identifier identifier = Identifier.fromString(tableId); CompactAction action; String jobName; - if (orderStrategy.isEmpty() && orderByColumns.isEmpty()) { + if (isBlank(orderStrategy) && isBlank(orderByColumns)) { action = new CompactAction( warehouse, @@ -96,7 +80,7 @@ public String[] call( catalogOptions, tableConf); jobName = "Compact Job"; - } else if (!orderStrategy.isEmpty() && !orderByColumns.isEmpty()) { + } else if (!isBlank(orderStrategy) && !isBlank(orderByColumns)) { action = new SortCompactAction( warehouse, @@ -112,8 +96,8 @@ public String[] call( "You must specify 'order strategy' and 'order by columns' both."); } - if (!(StringUtils.isBlank(partitions))) { - action.withPartitions(ParameterUtils.getPartitions(partitions.split(";"))); + if (!(isBlank(partitions))) { + action.withPartitions(getPartitions(partitions.split(";"))); } return execute(procedureContext, action, jobName); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java index 79f1079a230d..31b3ffdffcb7 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java @@ -23,7 +23,7 @@ import org.apache.paimon.flink.util.AbstractTestBase; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; -import org.apache.paimon.table.Table; +import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.utils.BlockingIterator; import org.apache.paimon.utils.SnapshotManager; @@ -183,10 +183,11 @@ protected CatalogTable table(String tableName) throws TableNotExistException { return (CatalogTable) table; } - protected Table paimonTable(String tableName) + protected FileStoreTable paimonTable(String tableName) throws org.apache.paimon.catalog.Catalog.TableNotExistException { org.apache.paimon.catalog.Catalog catalog = flinkCatalog().catalog(); - return catalog.getTable(Identifier.create(tEnv.getCurrentDatabase(), tableName)); + return (FileStoreTable) + catalog.getTable(Identifier.create(tEnv.getCurrentDatabase(), tableName)); } private FlinkCatalog flinkCatalog() { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java index d40d1aeea895..d0f10766e881 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java @@ -43,7 +43,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ThreadLocalRandom; import static org.assertj.core.api.Assertions.assertThat; @@ -77,11 +76,7 @@ public void testBatchCompact() throws Exception { checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); - if (ThreadLocalRandom.current().nextBoolean()) { - runAction(false); - } else { - callProcedure(false); - } + runAction(false); checkLatestSnapshot(table, 3, Snapshot.CommitKind.COMPACT); @@ -126,11 +121,7 @@ public void testStreamingCompact() throws Exception { TableScan.Plan plan = scan.plan(); assertThat(plan.splits()).isEmpty(); - if (ThreadLocalRandom.current().nextBoolean()) { - runAction(true); - } else { - callProcedure(true); - } + runAction(true); // first full compaction validateResult( @@ -193,11 +184,7 @@ public void testUnawareBucketStreamingCompact() throws Exception { checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); - if (ThreadLocalRandom.current().nextBoolean()) { - runAction(true); - } else { - callProcedure(true); - } + runAction(true); // first compaction, snapshot will be 3 checkFileAndRowSize(table, 3L, 30_000L, 1, 6); @@ -234,11 +221,7 @@ public void testUnawareBucketBatchCompact() throws Exception { checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); - if (ThreadLocalRandom.current().nextBoolean()) { - runAction(false); - } else { - callProcedure(false); - } + runAction(false); // first compaction, snapshot will be 3. checkFileAndRowSize(table, 3L, 0L, 1, 6); @@ -312,13 +295,4 @@ private void runAction(boolean isStreaming) throws Exception { env.execute(); } } - - private void callProcedure(boolean isStreaming) { - callProcedure( - String.format( - "CALL sys.compact('%s.%s', '%s')", - database, tableName, "dt=20221208,hh=15;dt=20221209,hh=15"), - isStreaming, - !isStreaming); - } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForDynamicBucketITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForDynamicBucketITCase.java index 3101ffb1b182..11407c587a0c 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForDynamicBucketITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForDynamicBucketITCase.java @@ -169,27 +169,15 @@ public void testDynamicBucketSortWithStringType() throws Exception { } private void zorder(List columns) throws Exception { - if (RANDOM.nextBoolean()) { - createAction("zorder", columns).run(); - } else { - callProcedure("zorder", columns); - } + createAction("zorder", columns).run(); } private void hilbert(List columns) throws Exception { - if (RANDOM.nextBoolean()) { - createAction("hilbert", columns).run(); - } else { - callProcedure("hilbert", columns); - } + createAction("hilbert", columns).run(); } private void order(List columns) throws Exception { - if (RANDOM.nextBoolean()) { - createAction("order", columns).run(); - } else { - callProcedure("order", columns); - } + createAction("order", columns).run(); } private SortCompactAction createAction(String orderStrategy, List columns) { @@ -208,15 +196,6 @@ private SortCompactAction createAction(String orderStrategy, List column String.join(",", columns)); } - private void callProcedure(String orderStrategy, List orderByColumns) { - callProcedure( - String.format( - "CALL sys.compact('%s.%s', '', '%s', '%s')", - database, tableName, orderStrategy, String.join(",", orderByColumns)), - false, - true); - } - // schema with all the basic types. private static Schema schema() { Schema.Builder schemaBuilder = Schema.newBuilder(); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java index 24c9101a3e84..317661feb8db 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/SortCompactActionForUnawareBucketITCase.java @@ -332,29 +332,17 @@ public void testSortCompactionOnEmptyData() throws Exception { private void zorder(List columns) throws Exception { String rangeStrategy = RANDOM.nextBoolean() ? "size" : "quantity"; - if (RANDOM.nextBoolean()) { - createAction("zorder", rangeStrategy, columns).run(); - } else { - callProcedure("zorder", rangeStrategy, columns); - } + createAction("zorder", rangeStrategy, columns).run(); } private void hilbert(List columns) throws Exception { String rangeStrategy = RANDOM.nextBoolean() ? "size" : "quantity"; - if (RANDOM.nextBoolean()) { - createAction("hilbert", rangeStrategy, columns).run(); - } else { - callProcedure("hilbert", rangeStrategy, columns); - } + createAction("hilbert", rangeStrategy, columns).run(); } private void order(List columns) throws Exception { String rangeStrategy = RANDOM.nextBoolean() ? "size" : "quantity"; - if (RANDOM.nextBoolean()) { - createAction("order", rangeStrategy, columns).run(); - } else { - callProcedure("order", rangeStrategy, columns); - } + createAction("order", rangeStrategy, columns).run(); } private SortCompactAction createAction( @@ -377,20 +365,6 @@ private SortCompactAction createAction( rangeStrategy); } - private void callProcedure( - String orderStrategy, String rangeStrategy, List orderByColumns) { - callProcedure( - String.format( - "CALL sys.compact('%s.%s', '', '%s', '%s','sort-compaction.range-strategy=%s')", - database, - tableName, - orderStrategy, - String.join(",", orderByColumns), - rangeStrategy), - false, - true); - } - private void createTable() throws Exception { catalog.createDatabase(database, true); catalog.createTable(identifier(), schema(), true); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java new file mode 100644 index 000000000000..a961d3b8351c --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java @@ -0,0 +1,194 @@ +/* + * 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.Snapshot; +import org.apache.paimon.flink.CatalogITCaseBase; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.table.source.StreamTableScan; +import org.apache.paimon.table.source.TableScan; +import org.apache.paimon.utils.BlockingIterator; +import org.apache.paimon.utils.SnapshotManager; +import org.apache.paimon.utils.StringUtils; + +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.types.Row; +import org.apache.flink.types.RowKind; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.assertj.core.api.Assertions.assertThat; + +/** IT Case for {@link CompactProcedure}. */ +public class CompactProcedureITCase extends CatalogITCaseBase { + + // ----------------------- Non-sort Compact ----------------------- + @Test + public void testBatchCompact() throws Exception { + sql( + "CREATE TABLE T (" + + " k INT," + + " v INT," + + " hh INT," + + " dt STRING," + + " PRIMARY KEY (k, dt, hh) NOT ENFORCED" + + ") PARTITIONED BY (dt, hh) WITH (" + + " 'write-only' = 'true'," + + " 'bucket' = '1'" + + ")"); + FileStoreTable table = paimonTable("T"); + + sql( + "INSERT INTO T VALUES (1, 100, 15, '20221208'), (1, 100, 16, '20221208'), (1, 100, 15, '20221209')"); + sql( + "INSERT INTO T VALUES (2, 100, 15, '20221208'), (2, 100, 16, '20221208'), (2, 100, 15, '20221209')"); + + checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); + + tEnv.getConfig().set(TableConfigOptions.TABLE_DML_SYNC, true); + sql( + "CALL sys.compact(`table` => 'default.T', partitions => 'dt=20221208,hh=15;dt=20221209,hh=15')"); + + checkLatestSnapshot(table, 3, Snapshot.CommitKind.COMPACT); + + List splits = table.newSnapshotReader().read().dataSplits(); + assertThat(splits.size()).isEqualTo(3); + for (DataSplit split : splits) { + if (split.partition().getInt(1) == 15) { + // compacted + assertThat(split.dataFiles().size()).isEqualTo(1); + } else { + // not compacted + assertThat(split.dataFiles().size()).isEqualTo(2); + } + } + } + + @Test + public void testStreamingCompact() throws Exception { + sql( + "CREATE TABLE T (" + + " k INT," + + " v INT," + + " hh INT," + + " dt STRING," + + " PRIMARY KEY (k, dt, hh) NOT ENFORCED" + + ") PARTITIONED BY (dt, hh) WITH (" + + " 'write-only' = 'true'," + + " 'bucket' = '1'," + + " 'changelog-producer' = 'full-compaction'," + + " 'full-compaction.delta-commits' = '1'," + + " 'continuous.discovery-interval' = '1s'" + + ")"); + FileStoreTable table = paimonTable("T"); + BlockingIterator select = streamSqlBlockIter("SELECT * FROM T"); + + sql( + "INSERT INTO T VALUES (1, 100, 15, '20221208'), (1, 100, 16, '20221208'), (1, 100, 15, '20221209')"); + checkLatestSnapshot(table, 1, Snapshot.CommitKind.APPEND); + + // no full compaction has happened, so plan should be empty + StreamTableScan scan = table.newReadBuilder().newStreamScan(); + TableScan.Plan plan = scan.plan(); + assertThat(plan.splits()).isEmpty(); + + // submit streaming compaction job + streamSqlIter( + "CALL sys.compact(`table` => 'default.T', partitions => 'dt=20221208,hh=15;dt=20221209,hh=15', " + + "options => 'scan.parallelism=1')") + .close(); + + // first full compaction + assertThat(select.collect(2)) + .containsExactlyInAnyOrder( + Row.of(1, 100, 15, "20221208"), Row.of(1, 100, 15, "20221209")); + + // incremental records + sql( + "INSERT INTO T VALUES (1, 101, 15, '20221208'), (1, 101, 16, '20221208'), (1, 101, 15, '20221209')"); + + // second full compaction + assertThat(select.collect(4)) + .containsExactlyInAnyOrder( + Row.ofKind(RowKind.UPDATE_BEFORE, 1, 100, 15, "20221208"), + Row.ofKind(RowKind.UPDATE_AFTER, 1, 101, 15, "20221208"), + Row.ofKind(RowKind.UPDATE_BEFORE, 1, 100, 15, "20221209"), + Row.ofKind(RowKind.UPDATE_AFTER, 1, 101, 15, "20221209")); + + select.close(); + } + + // ----------------------- Sort Compact ----------------------- + + @Test + public void testDynamicBucketSortCompact() throws Exception { + sql( + "CREATE TABLE T (" + + " f0 BIGINT PRIMARY KEY NOT ENFORCED," + + " f1 BIGINT," + + " f2 BIGINT," + + " f3 BIGINT," + + " f4 STRING" + + ") WITH (" + + " 'write-only' = 'true'," + + " 'dynamic-bucket.target-row-num' = '100'," + + " 'zorder.var-length-contribution' = '14'" + + ")"); + FileStoreTable table = paimonTable("T"); + + ThreadLocalRandom random = ThreadLocalRandom.current(); + int commitTimes = 20; + for (int i = 0; i < commitTimes; i++) { + String value = + IntStream.range(0, 100) + .mapToObj( + in -> + String.format( + "(%s, %s, %s, %s, '%s')", + random.nextLong(), + random.nextLong(), + random.nextLong(), + random.nextLong(), + StringUtils.randomNumericString(14))) + .collect(Collectors.joining(",")); + + sql("INSERT INTO T VALUES %s", value); + } + checkLatestSnapshot(table, 20, Snapshot.CommitKind.APPEND); + + tEnv.getConfig().set(TableConfigOptions.TABLE_DML_SYNC, true); + sql( + "CALL sys.compact(`table` => 'default.T', order_strategy => 'zorder', order_by => 'f2,f1')"); + + checkLatestSnapshot(table, 21, Snapshot.CommitKind.OVERWRITE); + } + + private void checkLatestSnapshot( + FileStoreTable table, long snapshotId, Snapshot.CommitKind commitKind) { + SnapshotManager snapshotManager = table.snapshotManager(); + Snapshot snapshot = snapshotManager.snapshot(snapshotManager.latestSnapshotId()); + assertThat(snapshot.id()).isEqualTo(snapshotId); + assertThat(snapshot.commitKind()).isEqualTo(commitKind); + } +} From 76abb848014fbc92d9f0e9463df7453a0af36322 Mon Sep 17 00:00:00 2001 From: Kerwin <37063904+zhuangchong@users.noreply.github.com> Date: Mon, 25 Mar 2024 17:24:52 +0800 Subject: [PATCH 53/79] [cdc] Update the latest dynamic options to the table schema file when synchronizing an existing paimon table using the cdc action (#3062) --- .../flink/action/cdc/SyncTableActionBase.java | 8 ++--- .../action/cdc/SynchronizationActionBase.java | 14 +++++++++ .../cdc/mysql/MySqlSyncDatabaseAction.java | 2 ++ .../flink/action/cdc/CdcActionITCaseBase.java | 29 +++++++++++++++++++ .../action/cdc/kafka/KafkaSchemaITCase.java | 5 ++-- .../mongodb/MongoDBSyncTableActionITCase.java | 5 ++-- .../cdc/mysql/MySqlSyncTableActionITCase.java | 8 +++-- .../PostgresSyncTableActionITCase.java | 4 +-- 8 files changed, 60 insertions(+), 15 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java index ae9301362de9..e45dd86b82a3 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java @@ -18,7 +18,6 @@ package org.apache.paimon.flink.action.cdc; -import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.catalog.AbstractCatalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkConnectorOptions; @@ -124,6 +123,8 @@ protected void beforeBuildingSourceSink() throws Exception { if (catalog.tableExists(identifier)) { fileStoreTable = (FileStoreTable) catalog.getTable(identifier); fileStoreTable = copyOptionsWithoutBucket(fileStoreTable); + // Update the latest options to schema + toOptionsChange(identifier, fileStoreTable.options()); try { Schema retrievedSchema = retrieveSchema(); computedColumns = @@ -208,11 +209,6 @@ private void checkConstraints() { } } - @VisibleForTesting - public FileStoreTable fileStoreTable() { - return fileStoreTable; - } - /** Custom exception to indicate issues with schema retrieval. */ public static class SchemaRetrievalException extends Exception { public SchemaRetrievalException(String message) { diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java index 147803c3580c..78af9050ee83 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java @@ -20,12 +20,15 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.action.Action; import org.apache.paimon.flink.action.ActionBase; import org.apache.paimon.flink.action.cdc.watermark.CdcWatermarkStrategy; import org.apache.paimon.flink.sink.cdc.EventParser; import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; import org.apache.paimon.options.Options; +import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.table.FileStoreTable; import org.apache.flink.api.common.eventtime.WatermarkStrategy; @@ -41,6 +44,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; import static org.apache.paimon.CoreOptions.TagCreationMode.WATERMARK; import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_ALIGNMENT_GROUP; @@ -178,6 +182,16 @@ protected FileStoreTable copyOptionsWithoutBucket(FileStoreTable table) { return table.copy(toCopy); } + protected void toOptionsChange(Identifier identifier, Map options) + throws Catalog.ColumnAlreadyExistException, Catalog.TableNotExistException, + Catalog.ColumnNotExistException { + List optionChanges = + options.entrySet().stream() + .map(entry -> SchemaChange.setOption(entry.getKey(), entry.getValue())) + .collect(Collectors.toList()); + catalog.alterTable(identifier, optionChanges, false); + } + @Override public void run() throws Exception { build(); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java index 70edca7e87ab..ecd9921d116c 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java @@ -161,6 +161,8 @@ protected void beforeBuildingSourceSink() throws Exception { if (shouldMonitorTable(table.schema(), fromMySql, errMsg)) { tables.add(table); monitoredTables.addAll(tableInfo.identifiers()); + // Update the latest options to schema + toOptionsChange(identifier, table.options()); } else { excludedTables.addAll(tableInfo.identifiers()); } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java index 7a621030217a..ae726fe0ee02 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java @@ -52,6 +52,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.concurrent.ThreadLocalRandom; import static org.assertj.core.api.Assertions.assertThat; @@ -154,6 +155,34 @@ protected void waitForResult( } } + protected void waitForOptions(Map expected, FileStoreTable table) + throws Exception { + + // wait for table options to become our expected options + Map expectedOptions = new HashMap<>(expected); + expectedOptions.put("path", table.options().get("path")); + while (true) { + if (table.options().size() == expectedOptions.size()) { + boolean result = + table.options().entrySet().stream() + .allMatch( + entry -> { + Object key = entry.getKey(); + Object value1 = entry.getValue(); + Object value2 = expectedOptions.get(key); + return expectedOptions.containsKey(key) + && Objects.equals(value1, value2); + }); + if (result) { + break; + } + } + + table = table.copyWithLatestSchema(); + Thread.sleep(1000); + } + } + protected Map getBasicTableConfig() { Map config = new HashMap<>(); ThreadLocalRandom random = ThreadLocalRandom.current(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java index 35eecbfad985..e95b204b2398 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java @@ -109,8 +109,9 @@ public void testTableOptionsChange() throws Exception { KafkaSyncTableAction action2 = syncTableActionBuilder(kafkaConfig).withTableConfig(tableConfig).build(); runActionWithDefaultEnv(action2); - Map dynamicOptions = action2.fileStoreTable().options(); - assertThat(dynamicOptions).containsAllEntriesOf(tableConfig).containsKey("path"); + + FileStoreTable table = getFileStoreTable(tableName); + waitForOptions(tableConfig, table); } @Test diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java index 78ad2f35042d..66e31b170ef2 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java @@ -199,8 +199,9 @@ public void testOptionsChange() throws Exception { MongoDBSyncTableAction action2 = syncTableActionBuilder(mongodbConfig).withTableConfig(tableConfig).build(); runActionWithDefaultEnv(action2); - Map dynamicOptions = action2.fileStoreTable().options(); - assertThat(dynamicOptions).containsAllEntriesOf(tableConfig); + + FileStoreTable table = getFileStoreTable(tableName); + waitForOptions(tableConfig, table); } @Test diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java index 6cd35eec8328..1874a93a37fc 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java @@ -1093,8 +1093,8 @@ public void testOptionsChange() throws Exception { .build(); runActionWithDefaultEnv(action2); - Map dynamicOptions = action2.fileStoreTable().options(); - assertThat(dynamicOptions).containsAllEntriesOf(tableConfig); + FileStoreTable table = getFileStoreTable(); + waitForOptions(tableConfig, table); } @Test @@ -1274,6 +1274,8 @@ public void testInvalidAlterBucket() throws Exception { .build(); assertThatCode(action::build).doesNotThrowAnyException(); - assertThat(action.fileStoreTable().options().get(BUCKET.key())).isEqualTo("1"); + + FileStoreTable table = getFileStoreTable(); + waitForOptions(Collections.singletonMap(BUCKET.key(), "1"), table); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionITCase.java index 55dc58764d6c..f7f0c7b06818 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionITCase.java @@ -722,8 +722,8 @@ public void testOptionsChange() throws Exception { .build(); runActionWithDefaultEnv(action2); - Map dynamicOptions = action2.fileStoreTable().options(); - assertThat(dynamicOptions).containsAllEntriesOf(tableConfig); + FileStoreTable table = getFileStoreTable(); + waitForOptions(tableConfig, table); } @Test From d77b75da40acd382ca833360f9b03c855b145183 Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Mon, 25 Mar 2024 17:36:43 +0800 Subject: [PATCH 54/79] [flink] Fix performance issue in ContinuousFileSplitEnumerator (#3071) --- .../paimon/flink/source/ContinuousFileSplitEnumerator.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/ContinuousFileSplitEnumerator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/ContinuousFileSplitEnumerator.java index 72b3742b3fb3..09f1d0d8f3c6 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/ContinuousFileSplitEnumerator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/ContinuousFileSplitEnumerator.java @@ -245,9 +245,10 @@ protected synchronized void assignSplits() { // create assignment Map> assignment = new HashMap<>(); Iterator readersAwait = readersAwaitingSplit.iterator(); + Set subtaskIds = context.registeredReaders().keySet(); while (readersAwait.hasNext()) { Integer task = readersAwait.next(); - if (!context.registeredReaders().containsKey(task)) { + if (!subtaskIds.contains(task)) { readersAwait.remove(); continue; } From 1cf8b47987d1b4665541eaff262236b5c19e4db8 Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Mon, 25 Mar 2024 19:56:51 +0800 Subject: [PATCH 55/79] [Refactor] Minor refactor update the latest dynamic options using the cdc action. (#3090) --- .../flink/action/cdc/SyncTableActionBase.java | 6 ++-- .../action/cdc/SynchronizationActionBase.java | 25 +++++++++------- .../cdc/mysql/MySqlSyncDatabaseAction.java | 4 +-- .../flink/action/cdc/CdcActionITCaseBase.java | 29 ------------------- .../action/cdc/kafka/KafkaSchemaITCase.java | 2 +- .../mongodb/MongoDBSyncTableActionITCase.java | 2 +- .../cdc/mysql/MySqlSyncTableActionITCase.java | 4 +-- .../PostgresSyncTableActionITCase.java | 2 +- 8 files changed, 23 insertions(+), 51 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java index e45dd86b82a3..b41ecd7f4be5 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java @@ -122,9 +122,7 @@ protected void beforeBuildingSourceSink() throws Exception { // Check if table exists before trying to get or create it if (catalog.tableExists(identifier)) { fileStoreTable = (FileStoreTable) catalog.getTable(identifier); - fileStoreTable = copyOptionsWithoutBucket(fileStoreTable); - // Update the latest options to schema - toOptionsChange(identifier, fileStoreTable.options()); + alterTableOptions(identifier, fileStoreTable); try { Schema retrievedSchema = retrieveSchema(); computedColumns = @@ -151,7 +149,7 @@ protected void beforeBuildingSourceSink() throws Exception { computedColumns = buildComputedColumns(computedColumnArgs, retrievedSchema.fields()); Schema paimonSchema = buildPaimonSchema(retrievedSchema); catalog.createTable(identifier, paimonSchema, false); - fileStoreTable = (FileStoreTable) catalog.getTable(identifier).copy(tableConfig); + fileStoreTable = (FileStoreTable) catalog.getTable(identifier); } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java index 78af9050ee83..684408f2438e 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java @@ -176,20 +176,25 @@ protected abstract void buildSink( DataStream input, EventParser.Factory parserFactory); - protected FileStoreTable copyOptionsWithoutBucket(FileStoreTable table) { - Map toCopy = new HashMap<>(tableConfig); - toCopy.remove(CoreOptions.BUCKET.key()); - return table.copy(toCopy); - } + protected FileStoreTable alterTableOptions(Identifier identifier, FileStoreTable table) { + // doesn't support altering bucket here + Map withoutBucket = new HashMap<>(tableConfig); + withoutBucket.remove(CoreOptions.BUCKET.key()); - protected void toOptionsChange(Identifier identifier, Map options) - throws Catalog.ColumnAlreadyExistException, Catalog.TableNotExistException, - Catalog.ColumnNotExistException { List optionChanges = - options.entrySet().stream() + withoutBucket.entrySet().stream() .map(entry -> SchemaChange.setOption(entry.getKey(), entry.getValue())) .collect(Collectors.toList()); - catalog.alterTable(identifier, optionChanges, false); + + try { + catalog.alterTable(identifier, optionChanges, false); + } catch (Catalog.TableNotExistException + | Catalog.ColumnAlreadyExistException + | Catalog.ColumnNotExistException e) { + throw new RuntimeException("This is unexpected.", e); + } + + return table.copy(withoutBucket); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java index ecd9921d116c..da3b80c5f09a 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java @@ -155,14 +155,12 @@ protected void beforeBuildingSourceSink() throws Exception { true); try { table = (FileStoreTable) catalog.getTable(identifier); - table = copyOptionsWithoutBucket(table); Supplier errMsg = incompatibleMessage(table.schema(), tableInfo, identifier); if (shouldMonitorTable(table.schema(), fromMySql, errMsg)) { + table = alterTableOptions(identifier, table); tables.add(table); monitoredTables.addAll(tableInfo.identifiers()); - // Update the latest options to schema - toOptionsChange(identifier, table.options()); } else { excludedTables.addAll(tableInfo.identifiers()); } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java index ae726fe0ee02..7a621030217a 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java @@ -52,7 +52,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.concurrent.ThreadLocalRandom; import static org.assertj.core.api.Assertions.assertThat; @@ -155,34 +154,6 @@ protected void waitForResult( } } - protected void waitForOptions(Map expected, FileStoreTable table) - throws Exception { - - // wait for table options to become our expected options - Map expectedOptions = new HashMap<>(expected); - expectedOptions.put("path", table.options().get("path")); - while (true) { - if (table.options().size() == expectedOptions.size()) { - boolean result = - table.options().entrySet().stream() - .allMatch( - entry -> { - Object key = entry.getKey(); - Object value1 = entry.getValue(); - Object value2 = expectedOptions.get(key); - return expectedOptions.containsKey(key) - && Objects.equals(value1, value2); - }); - if (result) { - break; - } - } - - table = table.copyWithLatestSchema(); - Thread.sleep(1000); - } - } - protected Map getBasicTableConfig() { Map config = new HashMap<>(); ThreadLocalRandom random = ThreadLocalRandom.current(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java index e95b204b2398..5c998c33cf6b 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchemaITCase.java @@ -111,7 +111,7 @@ public void testTableOptionsChange() throws Exception { runActionWithDefaultEnv(action2); FileStoreTable table = getFileStoreTable(tableName); - waitForOptions(tableConfig, table); + assertThat(table.options()).containsAllEntriesOf(tableConfig); } @Test diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java index 66e31b170ef2..48c6cd481103 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java @@ -201,7 +201,7 @@ public void testOptionsChange() throws Exception { runActionWithDefaultEnv(action2); FileStoreTable table = getFileStoreTable(tableName); - waitForOptions(tableConfig, table); + assertThat(table.options()).containsAllEntriesOf(tableConfig); } @Test diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java index 1874a93a37fc..2ed3b6f466c5 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java @@ -1094,7 +1094,7 @@ public void testOptionsChange() throws Exception { runActionWithDefaultEnv(action2); FileStoreTable table = getFileStoreTable(); - waitForOptions(tableConfig, table); + assertThat(table.options()).containsAllEntriesOf(tableConfig); } @Test @@ -1276,6 +1276,6 @@ public void testInvalidAlterBucket() throws Exception { assertThatCode(action::build).doesNotThrowAnyException(); FileStoreTable table = getFileStoreTable(); - waitForOptions(Collections.singletonMap(BUCKET.key(), "1"), table); + assertThat(table.options().get(BUCKET.key())).isEqualTo("1"); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionITCase.java index f7f0c7b06818..fbd441125cf6 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionITCase.java @@ -723,7 +723,7 @@ public void testOptionsChange() throws Exception { runActionWithDefaultEnv(action2); FileStoreTable table = getFileStoreTable(); - waitForOptions(tableConfig, table); + assertThat(table.options()).containsAllEntriesOf(tableConfig); } @Test From 2b894caa69b5ff113c7e478ac0003d2eaef59979 Mon Sep 17 00:00:00 2001 From: JunZhang Date: Tue, 26 Mar 2024 09:51:05 +0800 Subject: [PATCH 56/79] [cdc] Extract common code (#3050) --- .../format/debezium/DebeziumRecordParser.java | 8 +- .../format/debezium/DebeziumSchemaUtils.java | 7 +- .../action/cdc/mysql/MySqlRecordParser.java | 126 ++---------------- 3 files changed, 20 insertions(+), 121 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumRecordParser.java index d4d5b47a25d6..96f1bc07429b 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumRecordParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumRecordParser.java @@ -33,6 +33,7 @@ import javax.annotation.Nullable; +import java.time.ZoneOffset; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -198,7 +199,12 @@ protected Map extractRowData( String transformed = DebeziumSchemaUtils.transformRawValue( - rawValue, debeziumType, className, typeMapping, record.get(fieldName)); + rawValue, + debeziumType, + className, + typeMapping, + record.get(fieldName), + ZoneOffset.UTC); resultMap.put(fieldName, transformed); paimonFieldTypes.put( diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java index d3f1b1196632..1aab6653d4d4 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java @@ -43,6 +43,7 @@ import java.math.BigDecimal; import java.time.Instant; import java.time.LocalDateTime; +import java.time.ZoneId; import java.time.ZoneOffset; import java.util.Base64; import java.util.Map; @@ -61,7 +62,8 @@ public static String transformRawValue( String debeziumType, @Nullable String className, TypeMapping typeMapping, - JsonNode origin) { + JsonNode origin, + ZoneId serverTimeZone) { if (rawValue == null) { return null; } @@ -138,9 +140,8 @@ else if (Date.SCHEMA_NAME.equals(className)) { // https://dev.mysql.com/doc/refman/8.0/en/datetime.html for standard, and // RowDataDebeziumDeserializeSchema#convertToTimestamp in flink-cdc-connector // for implementation - // TODO currently we cannot get zone id LocalDateTime localDateTime = - Instant.parse(rawValue).atZone(ZoneOffset.UTC).toLocalDateTime(); + Instant.parse(rawValue).atZone(serverTimeZone).toLocalDateTime(); transformed = DateTimeUtils.formatLocalDateTime(localDateTime, 6); } else if (MicroTime.SCHEMA_NAME.equals(className)) { long microseconds = Long.parseLong(rawValue); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java index ee0049fd416c..72b42b46689b 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java @@ -21,15 +21,14 @@ import org.apache.paimon.flink.action.cdc.CdcMetadataConverter; import org.apache.paimon.flink.action.cdc.ComputedColumn; import org.apache.paimon.flink.action.cdc.TypeMapping; +import org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils; import org.apache.paimon.flink.action.cdc.mysql.format.DebeziumEvent; import org.apache.paimon.flink.sink.cdc.CdcRecord; import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; import org.apache.paimon.types.DataType; import org.apache.paimon.types.RowKind; -import org.apache.paimon.utils.DateTimeUtils; import org.apache.paimon.utils.JsonSerdeUtil; import org.apache.paimon.utils.Preconditions; -import org.apache.paimon.utils.StringUtils; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.DeserializationFeature; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; @@ -37,31 +36,17 @@ import com.ververica.cdc.connectors.mysql.source.config.MySqlSourceOptions; import io.debezium.connector.AbstractSourceInfo; -import io.debezium.data.Bits; -import io.debezium.data.geometry.Geometry; -import io.debezium.data.geometry.Point; import io.debezium.relational.Column; import io.debezium.relational.Table; import io.debezium.relational.history.TableChanges; -import io.debezium.time.Date; -import io.debezium.time.MicroTime; -import io.debezium.time.MicroTimestamp; -import io.debezium.time.Timestamp; -import io.debezium.time.ZonedTimestamp; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; -import org.apache.kafka.connect.json.JsonConverterConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.math.BigDecimal; -import java.time.Instant; -import java.time.LocalDateTime; import java.time.ZoneId; -import java.time.ZoneOffset; import java.util.ArrayList; -import java.util.Base64; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -78,8 +63,6 @@ import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.mapKeyCaseConvert; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.recordKeyDuplicateErrMsg; import static org.apache.paimon.flink.action.cdc.TypeMapping.TypeMappingMode.TO_NULLABLE; -import static org.apache.paimon.flink.action.cdc.TypeMapping.TypeMappingMode.TO_STRING; -import static org.apache.paimon.flink.action.cdc.format.debezium.DebeziumSchemaUtils.decimalLogicalName; import static org.apache.paimon.utils.JsonSerdeUtil.isNull; /** @@ -258,105 +241,14 @@ private Map extractRow(JsonNode recordRow) { String className = field.getValue().name(); String oldValue = objectValue.asText(); - String newValue = oldValue; - - if (Bits.LOGICAL_NAME.equals(className)) { - // transform little-endian form to normal order - // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-data-types - byte[] littleEndian = Base64.getDecoder().decode(oldValue); - byte[] bigEndian = new byte[littleEndian.length]; - for (int i = 0; i < littleEndian.length; i++) { - bigEndian[i] = littleEndian[littleEndian.length - 1 - i]; - } - if (typeMapping.containsMode(TO_STRING)) { - newValue = StringUtils.bytesToBinaryString(bigEndian); - } else { - newValue = Base64.getEncoder().encodeToString(bigEndian); - } - } else if (("bytes".equals(mySqlType) && className == null)) { - // MySQL binary, varbinary, blob - newValue = new String(Base64.getDecoder().decode(oldValue)); - } else if ("bytes".equals(mySqlType) && decimalLogicalName().equals(className)) { - // MySQL numeric, fixed, decimal - try { - new BigDecimal(oldValue); - } catch (NumberFormatException e) { - throw new IllegalArgumentException( - "Invalid big decimal value " - + oldValue - + ". Make sure that in the `customConverterConfigs` " - + "of the JsonDebeziumDeserializationSchema you created, set '" - + JsonConverterConfig.DECIMAL_FORMAT_CONFIG - + "' to 'numeric'", - e); - } - } - // pay attention to the temporal types - // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-temporal-types - else if (Date.SCHEMA_NAME.equals(className)) { - // MySQL date - newValue = DateTimeUtils.toLocalDate(Integer.parseInt(oldValue)).toString(); - } else if (Timestamp.SCHEMA_NAME.equals(className)) { - // MySQL datetime (precision 0-3) - - // display value of datetime is not affected by timezone, see - // https://dev.mysql.com/doc/refman/8.0/en/datetime.html for standard, and - // RowDataDebeziumDeserializeSchema#convertToTimestamp in flink-cdc-connector - // for implementation - LocalDateTime localDateTime = - DateTimeUtils.toLocalDateTime(Long.parseLong(oldValue), ZoneOffset.UTC); - newValue = DateTimeUtils.formatLocalDateTime(localDateTime, 3); - } else if (MicroTimestamp.SCHEMA_NAME.equals(className)) { - // MySQL datetime (precision 4-6) - long microseconds = Long.parseLong(oldValue); - long microsecondsPerSecond = 1_000_000; - long nanosecondsPerMicros = 1_000; - long seconds = microseconds / microsecondsPerSecond; - long nanoAdjustment = (microseconds % microsecondsPerSecond) * nanosecondsPerMicros; - - // display value of datetime is not affected by timezone, see - // https://dev.mysql.com/doc/refman/8.0/en/datetime.html for standard, and - // RowDataDebeziumDeserializeSchema#convertToTimestamp in flink-cdc-connector - // for implementation - LocalDateTime localDateTime = - Instant.ofEpochSecond(seconds, nanoAdjustment) - .atZone(ZoneOffset.UTC) - .toLocalDateTime(); - newValue = DateTimeUtils.formatLocalDateTime(localDateTime, 6); - } else if (ZonedTimestamp.SCHEMA_NAME.equals(className)) { - // MySQL timestamp - - // display value of timestamp is affected by timezone, see - // https://dev.mysql.com/doc/refman/8.0/en/datetime.html for standard, and - // RowDataDebeziumDeserializeSchema#convertToTimestamp in flink-cdc-connector - // for implementation - LocalDateTime localDateTime = - Instant.parse(oldValue).atZone(serverTimeZone).toLocalDateTime(); - newValue = DateTimeUtils.formatLocalDateTime(localDateTime, 6); - } else if (MicroTime.SCHEMA_NAME.equals(className)) { - long microseconds = Long.parseLong(oldValue); - long microsecondsPerSecond = 1_000_000; - long nanosecondsPerMicros = 1_000; - long seconds = microseconds / microsecondsPerSecond; - long nanoAdjustment = (microseconds % microsecondsPerSecond) * nanosecondsPerMicros; - - newValue = - Instant.ofEpochSecond(seconds, nanoAdjustment) - .atZone(ZoneOffset.UTC) - .toLocalTime() - .toString(); - } else if (Point.LOGICAL_NAME.equals(className) - || Geometry.LOGICAL_NAME.equals(className)) { - try { - byte[] wkb = objectValue.get(Geometry.WKB_FIELD).binaryValue(); - newValue = MySqlTypeUtils.convertWkbArray(wkb); - } catch (Exception e) { - throw new IllegalArgumentException( - String.format("Failed to convert %s to geometry JSON.", objectValue), - e); - } - } - + String newValue = + DebeziumSchemaUtils.transformRawValue( + oldValue, + mySqlType, + className, + typeMapping, + objectValue, + serverTimeZone); resultMap.put(fieldName, newValue); } From ceb0a9ccfb4118bb36a53fa40e5d57ba1c25fd91 Mon Sep 17 00:00:00 2001 From: big face cat <731030576@qq.com> Date: Tue, 26 Mar 2024 11:32:10 +0800 Subject: [PATCH 57/79] [cdc] Fix Mysql sync_table unable to synchronize newly added tables with regular expression (#2851) --- .../flink/action/cdc/mysql/MySqlSyncTableAction.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java index d9f57232d019..36feefd3c3bb 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java @@ -100,11 +100,10 @@ protected Schema retrieveSchema() throws Exception { @Override protected MySqlSource buildSource() { - String tableList = - mySqlSchemasInfo.pkTables().stream() - .map(JdbcSchemasInfo.JdbcSchemaInfo::identifier) - .map(i -> i.getDatabaseName() + "\\." + i.getObjectName()) - .collect(Collectors.joining("|")); + String tableList = + cdcSourceConfig.get(MySqlSourceOptions.DATABASE_NAME) + + "\\." + + cdcSourceConfig.get(MySqlSourceOptions.TABLE_NAME); return MySqlActionUtils.buildMySqlSource(cdcSourceConfig, tableList, typeMapping); } From 346831908374c7fba345cd022520478bda22f43e Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Tue, 26 Mar 2024 11:45:41 +0800 Subject: [PATCH 58/79] [hotfix] Apply spotless to fix checkstyle violation in flink-cdc module (#3092) --- .../flink/action/cdc/mysql/MySqlSyncTableAction.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java index 36feefd3c3bb..50b306bb03d6 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java @@ -100,10 +100,10 @@ protected Schema retrieveSchema() throws Exception { @Override protected MySqlSource buildSource() { - String tableList = - cdcSourceConfig.get(MySqlSourceOptions.DATABASE_NAME) - + "\\." - + cdcSourceConfig.get(MySqlSourceOptions.TABLE_NAME); + String tableList = + cdcSourceConfig.get(MySqlSourceOptions.DATABASE_NAME) + + "\\." + + cdcSourceConfig.get(MySqlSourceOptions.TABLE_NAME); return MySqlActionUtils.buildMySqlSource(cdcSourceConfig, tableList, typeMapping); } From 2401f7e7fd09bbe182b59efcc46729b11111d2d2 Mon Sep 17 00:00:00 2001 From: YeJunHao <41894543+leaves12138@users.noreply.github.com> Date: Tue, 26 Mar 2024 12:06:45 +0800 Subject: [PATCH 59/79] [test] Append only table can't test failing IO re-insert yet (#3089) --- .../paimon/flink/UnawareBucketAppendOnlyTableITCase.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 a99997a00e9c..d428bd29a4aa 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 @@ -242,6 +242,7 @@ public void testTimestampLzType() { .toInstant())); } + // test is not correct, append table may insert twice if always retry when file io fails @Test public void testReadWriteFailRandom() throws Exception { setFailRate(100, 1000); @@ -265,7 +266,7 @@ public void testReadWriteFailRandom() throws Exception { () -> { batchSql("SELECT * FROM append_table"); List rows = batchSql("SELECT * FROM append_table"); - assertThat(rows.size()).isEqualTo(size); + assertThat(rows.size()).isGreaterThanOrEqualTo(size); assertThat(rows).containsExactlyInAnyOrder(results.toArray(new Row[0])); }); } @@ -294,7 +295,7 @@ public void testReadWriteFailRandomString() throws Exception { () -> { batchSql("SELECT * FROM append_table"); List rows = batchSql("SELECT * FROM append_table"); - assertThat(rows.size()).isEqualTo(size); + assertThat(rows.size()).isGreaterThanOrEqualTo(size); assertThat(rows).containsExactlyInAnyOrder(results.toArray(new Row[0])); }); } From 9839efa0589bcc4a555468fb554a23aaf27b331c Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Tue, 26 Mar 2024 14:20:00 +0800 Subject: [PATCH 60/79] [core] Introduce SplitGroup for SplitGenerator to optimize more rawFiles (#3059) --- .../source/AppendOnlySplitGenerator.java | 11 ++- .../table/source/MergeTreeSplitGenerator.java | 29 +++++-- .../paimon/table/source/SplitGenerator.java | 24 +++++- .../snapshot/IncrementalStartingScanner.java | 5 +- .../source/snapshot/SnapshotReaderImpl.java | 41 ++++----- .../apache/paimon/io/DataFileTestUtils.java | 9 +- .../table/source/SplitGeneratorTest.java | 84 ++++++++++++++++++- .../source/snapshot/SnapshotReaderTest.java | 2 +- 8 files changed, 158 insertions(+), 47 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/AppendOnlySplitGenerator.java b/paimon-core/src/main/java/org/apache/paimon/table/source/AppendOnlySplitGenerator.java index 002072a38736..dbc341803075 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/AppendOnlySplitGenerator.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/AppendOnlySplitGenerator.java @@ -26,6 +26,7 @@ import java.util.Collections; import java.util.List; import java.util.function.Function; +import java.util.stream.Collectors; import static org.apache.paimon.append.AppendOnlyCompactManager.fileComparator; @@ -44,21 +45,23 @@ public AppendOnlySplitGenerator( } @Override - public List> splitForBatch(List input) { + public List splitForBatch(List input) { List files = new ArrayList<>(input); files.sort(fileComparator(bucketMode == BucketMode.UNAWARE)); Function weightFunc = file -> Math.max(file.fileSize(), openFileCost); - return BinPacking.packForOrdered(files, weightFunc, targetSplitSize); + return BinPacking.packForOrdered(files, weightFunc, targetSplitSize).stream() + .map(SplitGroup::rawConvertibleGroup) + .collect(Collectors.toList()); } @Override - public List> splitForStreaming(List files) { + public List splitForStreaming(List files) { // When the bucket mode is unaware, we spit the files as batch, because unaware-bucket table // only contains one bucket (bucket 0). if (bucketMode == BucketMode.UNAWARE) { return splitForBatch(files); } else { - return Collections.singletonList(files); + return Collections.singletonList(SplitGroup.rawConvertibleGroup(files)); } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/MergeTreeSplitGenerator.java b/paimon-core/src/main/java/org/apache/paimon/table/source/MergeTreeSplitGenerator.java index 9a06a53f4ce6..4cf4a18031f5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/MergeTreeSplitGenerator.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/MergeTreeSplitGenerator.java @@ -61,11 +61,18 @@ public MergeTreeSplitGenerator( } @Override - public List> splitForBatch(List files) { - if (deletionVectorsEnabled || mergeEngine == FIRST_ROW) { + public List splitForBatch(List files) { + boolean rawConvertible = + files.stream().allMatch(file -> file.level() != 0 && withoutDeleteRow(file)); + boolean oneLevel = + files.stream().map(DataFileMeta::level).collect(Collectors.toSet()).size() == 1; + + if (rawConvertible && (deletionVectorsEnabled || mergeEngine == FIRST_ROW || oneLevel)) { Function weightFunc = file -> Math.max(file.fileSize(), openFileCost); - return BinPacking.packForOrdered(files, weightFunc, targetSplitSize); + return BinPacking.packForOrdered(files, weightFunc, targetSplitSize).stream() + .map(SplitGroup::rawConvertibleGroup) + .collect(Collectors.toList()); } /* @@ -93,13 +100,19 @@ public List> splitForBatch(List files) { new IntervalPartition(files, keyComparator) .partition().stream().map(this::flatRun).collect(Collectors.toList()); - return packSplits(sections); + return packSplits(sections).stream() + .map( + f -> + f.size() == 1 && withoutDeleteRow(f.get(0)) + ? SplitGroup.rawConvertibleGroup(f) + : SplitGroup.nonRawConvertibleGroup(f)) + .collect(Collectors.toList()); } @Override - public List> splitForStreaming(List files) { + public List splitForStreaming(List files) { // We don't split streaming scan files - return Collections.singletonList(files); + return Collections.singletonList(SplitGroup.rawConvertibleGroup(files)); } private List> packSplits(List> sections) { @@ -129,4 +142,8 @@ private List flatFiles(List> section) { section.forEach(files::addAll); return files; } + + private boolean withoutDeleteRow(DataFileMeta dataFileMeta) { + return dataFileMeta.deleteRowCount().map(count -> count == 0L).orElse(false); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/SplitGenerator.java b/paimon-core/src/main/java/org/apache/paimon/table/source/SplitGenerator.java index f4aa7c7e2858..73cfa9826839 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/SplitGenerator.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/SplitGenerator.java @@ -25,7 +25,27 @@ /** Generate splits from {@link DataFileMeta}s. */ public interface SplitGenerator { - List> splitForBatch(List files); + List splitForBatch(List files); - List> splitForStreaming(List files); + List splitForStreaming(List files); + + /** Split group. */ + class SplitGroup { + + public final List files; + public final boolean rawConvertible; + + private SplitGroup(List files, boolean rawConvertible) { + this.files = files; + this.rawConvertible = rawConvertible; + } + + public static SplitGroup rawConvertibleGroup(List files) { + return new SplitGroup(files, true); + } + + public static SplitGroup nonRawConvertibleGroup(List files) { + return new SplitGroup(files, false); + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java index 49ab3a87e764..602a6370a76f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java @@ -25,6 +25,7 @@ import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.PlanImpl; import org.apache.paimon.table.source.ScanMode; +import org.apache.paimon.table.source.SplitGenerator; import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.SnapshotManager; @@ -65,7 +66,7 @@ public Result scan(SnapshotReader reader) { for (Map.Entry, List> entry : grouped.entrySet()) { BinaryRow partition = entry.getKey().getLeft(); int bucket = entry.getKey().getRight(); - for (List files : + for (SplitGenerator.SplitGroup splitGroup : reader.splitGenerator().splitForBatch(entry.getValue())) { // TODO pass deletion files result.add( @@ -73,7 +74,7 @@ public Result scan(SnapshotReader reader) { .withSnapshot(endingSnapshotId) .withPartition(partition) .withBucket(bucket) - .withDataFiles(files) + .withDataFiles(splitGroup.files) .build()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java index aa28fa667467..fe48557e3ce8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java @@ -64,7 +64,6 @@ import java.util.function.BiConsumer; import java.util.stream.Collectors; -import static org.apache.paimon.CoreOptions.MergeEngine.FIRST_ROW; import static org.apache.paimon.deletionvectors.DeletionVectorsIndexFile.DELETION_VECTORS_INDEX; import static org.apache.paimon.operation.FileStoreScan.Plan.groupByPartFiles; import static org.apache.paimon.predicate.PredicateBuilder.transformFieldMapping; @@ -278,18 +277,24 @@ private List generateSplits( .withPartition(partition) .withBucket(bucket) .isStreaming(isStreaming); - List> splitGroups = + List splitGroups = isStreaming ? splitGenerator.splitForStreaming(bucketFiles) : splitGenerator.splitForBatch(bucketFiles); - for (List dataFiles : splitGroups) { - builder.withDataFiles(dataFiles) - .rawFiles(convertToRawFiles(partition, bucket, dataFiles)); - if (deletionVectors) { - IndexFileMeta deletionIndexFile = - indexFileHandler + + IndexFileMeta deletionIndexFile = + deletionVectors + ? indexFileHandler .scan(snapshotId, DELETION_VECTORS_INDEX, partition, bucket) - .orElse(null); + .orElse(null) + : null; + for (SplitGenerator.SplitGroup splitGroup : splitGroups) { + List dataFiles = splitGroup.files; + builder.withDataFiles(dataFiles); + if (splitGroup.rawConvertible) { + builder.rawFiles(convertToRawFiles(partition, bucket, dataFiles)); + } + if (deletionVectors) { builder.withDataDeletionFiles( getDeletionFiles(dataFiles, deletionIndexFile)); } @@ -370,8 +375,7 @@ private Plan toChangesPlan( .withBucket(bucket) .withBeforeFiles(before) .withDataFiles(data) - .isStreaming(isStreaming) - .rawFiles(convertToRawFiles(part, bucket, data)); + .isStreaming(isStreaming); if (deletionVectors) { IndexFileMeta beforeDeletionIndex = indexFileHandler @@ -437,21 +441,6 @@ private List getDeletionFiles( private List convertToRawFiles( BinaryRow partition, int bucket, List dataFiles) { String bucketPath = pathFactory.bucketPath(partition, bucket).toString(); - - // append only or deletionVectors files can be returned - if (tableSchema.primaryKeys().isEmpty() || deletionVectors || mergeEngine == FIRST_ROW) { - return makeRawTableFiles(bucketPath, dataFiles); - } - - int maxLevel = options.numLevels() - 1; - if (dataFiles.stream().map(DataFileMeta::level).allMatch(l -> l == maxLevel)) { - return makeRawTableFiles(bucketPath, dataFiles); - } - - return Collections.emptyList(); - } - - private List makeRawTableFiles(String bucketPath, List dataFiles) { return dataFiles.stream() .map(f -> makeRawTableFile(bucketPath, f)) .collect(Collectors.toList()); diff --git a/paimon-core/src/test/java/org/apache/paimon/io/DataFileTestUtils.java b/paimon-core/src/test/java/org/apache/paimon/io/DataFileTestUtils.java index 624237b38680..b902ae967773 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/DataFileTestUtils.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/DataFileTestUtils.java @@ -72,10 +72,15 @@ public static DataFileMeta newFile() { public static DataFileMeta newFile( String name, int level, int minKey, int maxKey, long maxSequence) { + return newFile(name, level, minKey, maxKey, maxSequence, 0L); + } + + public static DataFileMeta newFile( + String name, int level, int minKey, int maxKey, long maxSequence, long deleteRowCount) { return new DataFileMeta( name, maxKey - minKey + 1, - 1, + maxKey - minKey + 1, row(minKey), row(maxKey), null, @@ -84,7 +89,7 @@ public static DataFileMeta newFile( maxSequence, 0, level, - 0L); + deleteRowCount); } public static BinaryRow row(int i) { diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java index 1278339210de..82663a72cb2e 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java @@ -21,6 +21,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.table.BucketMode; +import org.apache.paimon.utils.Pair; import org.junit.jupiter.api.Test; @@ -31,8 +32,10 @@ import java.util.stream.Collectors; import static org.apache.paimon.CoreOptions.MergeEngine.DEDUPLICATE; +import static org.apache.paimon.CoreOptions.MergeEngine.FIRST_ROW; import static org.apache.paimon.data.BinaryRow.EMPTY_ROW; import static org.apache.paimon.io.DataFileTestUtils.fromMinMax; +import static org.apache.paimon.io.DataFileTestUtils.newFile; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link AppendOnlySplitGenerator} and {@link MergeTreeSplitGenerator}. */ @@ -124,13 +127,86 @@ public void testMergeTree() { Collections.singletonList("6")); } - private List> toNames(List> splits) { - return splits.stream() + @Test + public void testSplitRawConvertible() { + Comparator comparator = Comparator.comparingInt(o -> o.getInt(0)); + MergeTreeSplitGenerator mergeTreeSplitGenerator = + new MergeTreeSplitGenerator(comparator, 100, 2, false, DEDUPLICATE); + + // When level0 exists, should not be rawConvertible + List files1 = + Arrays.asList(newFile("1", 0, 0, 10, 10L), newFile("2", 0, 10, 20, 20L)); + assertThat(toNamesAndRawConvertible(mergeTreeSplitGenerator.splitForBatch(files1))) + .containsExactlyInAnyOrder(Pair.of(Arrays.asList("1", "2"), false)); + + // When deleteRowCount > 0, should not be rawConvertible + List files2 = + Arrays.asList(newFile("1", 1, 0, 10, 10L, 1L), newFile("2", 1, 10, 20, 20L)); + assertThat(toNamesAndRawConvertible(mergeTreeSplitGenerator.splitForBatch(files2))) + .containsExactlyInAnyOrder(Pair.of(Arrays.asList("1", "2"), false)); + + // No level0 and deleteRowCount == 0: + // All in one level, should be rawConvertible + List files3 = + Arrays.asList(newFile("1", 1, 0, 10, 10L), newFile("2", 1, 10, 20, 20L)); + assertThat(toNamesAndRawConvertible(mergeTreeSplitGenerator.splitForBatch(files3))) + .containsExactlyInAnyOrder(Pair.of(Arrays.asList("1", "2"), true)); + + // Not all in one level, should not be rawConvertible + List files4 = + Arrays.asList(newFile("1", 1, 0, 10, 10L), newFile("2", 2, 10, 20, 20L)); + assertThat(toNamesAndRawConvertible(mergeTreeSplitGenerator.splitForBatch(files4))) + .containsExactlyInAnyOrder(Pair.of(Arrays.asList("1", "2"), false)); + + // Not all in one level but with deletion vectors enabled, should be rawConvertible + MergeTreeSplitGenerator splitGeneratorWithDVEnabled = + new MergeTreeSplitGenerator(comparator, 100, 2, true, DEDUPLICATE); + assertThat(toNamesAndRawConvertible(splitGeneratorWithDVEnabled.splitForBatch(files4))) + .containsExactlyInAnyOrder(Pair.of(Arrays.asList("1", "2"), true)); + + // Not all in one level but with first row merge engine, should be rawConvertible + MergeTreeSplitGenerator splitGeneratorWithFirstRow = + new MergeTreeSplitGenerator(comparator, 100, 2, false, FIRST_ROW); + assertThat(toNamesAndRawConvertible(splitGeneratorWithFirstRow.splitForBatch(files4))) + .containsExactlyInAnyOrder(Pair.of(Arrays.asList("1", "2"), true)); + + // Split with one file should be rawConvertible + List files5 = + Arrays.asList( + newFile("1", 1, 0, 10, 10L), + newFile("2", 2, 0, 12, 12L), + newFile("3", 3, 15, 60, 60L), + newFile("4", 4, 18, 40, 40L), + newFile("5", 5, 82, 85, 85L), + newFile("6", 6, 100, 200, 200L)); + assertThat(toNamesAndRawConvertible(mergeTreeSplitGenerator.splitForBatch(files5))) + .containsExactlyInAnyOrder( + Pair.of(Arrays.asList("1", "2", "3", "4", "5"), false), + Pair.of(Collections.singletonList("6"), true)); + } + + private List> toNames(List splitGroups) { + return splitGroups.stream() .map( - files -> - files.stream() + splitGroup -> + splitGroup.files.stream() .map(DataFileMeta::fileName) .collect(Collectors.toList())) .collect(Collectors.toList()); } + + private List, Boolean>> toNamesAndRawConvertible( + List splitGroups) { + return splitGroups.stream() + .map( + splitGroup -> { + List sortedFileNames = + splitGroup.files.stream() + .sorted(Comparator.comparing(DataFileMeta::fileName)) + .map(DataFileMeta::fileName) + .collect(Collectors.toList()); + return Pair.of(sortedFileNames, splitGroup.rawConvertible); + }) + .collect(Collectors.toList()); + } } 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 e448d8684af1..a34f16a0305d 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 @@ -100,7 +100,7 @@ public void testGetPrimaryKeyRawFiles() throws Exception { assertThat(dataSplit.dataFiles()).hasSize(1); DataFileMeta meta = dataSplit.dataFiles().get(0); String partition = dataSplit.partition().getString(0).toString(); - assertThat(dataSplit.convertToRawFiles()).isNotPresent(); + assertThat(dataSplit.convertToRawFiles()).isPresent(); } // write another file on level 0 From c6cb0c243fb1404c06e0755316ca3a583e6622d8 Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Tue, 26 Mar 2024 14:39:38 +0800 Subject: [PATCH 61/79] [hotfix] Fix unstable test MySqlSyncTableActionITCase#testSyncShards (#3096) --- .../flink/action/cdc/mysql/MySqlSyncTableAction.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java index 50b306bb03d6..919cdab3c096 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java @@ -101,9 +101,10 @@ protected Schema retrieveSchema() throws Exception { @Override protected MySqlSource buildSource() { String tableList = - cdcSourceConfig.get(MySqlSourceOptions.DATABASE_NAME) - + "\\." - + cdcSourceConfig.get(MySqlSourceOptions.TABLE_NAME); + String.format( + "(%s)\\.(%s)", + cdcSourceConfig.get(MySqlSourceOptions.DATABASE_NAME), + cdcSourceConfig.get(MySqlSourceOptions.TABLE_NAME)); return MySqlActionUtils.buildMySqlSource(cdcSourceConfig, tableList, typeMapping); } From 726d6b282e08be6eeea988876fbf3f66bcc4c9a7 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Tue, 26 Mar 2024 15:40:15 +0800 Subject: [PATCH 62/79] [core] Refactor FileStorePathFactory to clean test methods (#3093) --- .../org/apache/paimon/data/BinaryRow.java | 70 +++++++++---------- .../apache/paimon/io/DataFilePathFactory.java | 16 ++--- .../operation/AbstractFileStoreScan.java | 3 +- .../paimon/operation/OrphanFilesClean.java | 2 +- .../paimon/table/system/ManifestsTable.java | 49 +++++-------- .../table/system/SystemTableLoader.java | 2 +- .../paimon/utils/FileStorePathFactory.java | 27 ++----- .../paimon/append/AppendOnlyWriterTest.java | 4 +- .../paimon/format/FileFormatSuffixTest.java | 2 +- .../paimon/io/DataFilePathFactoryTest.java | 8 +-- .../paimon/io/KeyValueFileReadWriteTest.java | 3 +- .../paimon/io/RollingFileWriterTest.java | 4 +- .../paimon/mergetree/ContainsLevelsTest.java | 5 +- .../paimon/mergetree/LookupLevelsTest.java | 5 +- .../paimon/mergetree/MergeTreeTestBase.java | 3 +- .../operation/OrphanFilesCleanTest.java | 2 +- .../paimon/table/sink/TableCommitTest.java | 4 +- .../table/system/ManifestsTableTest.java | 4 +- .../utils/FileStorePathFactoryTest.java | 13 +++- .../spark/PaimonPartitionManagement.scala | 3 +- 20 files changed, 97 insertions(+), 132 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/data/BinaryRow.java b/paimon-common/src/main/java/org/apache/paimon/data/BinaryRow.java index 86e2ef0e621d..7068e25e6e60 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/BinaryRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/BinaryRow.java @@ -21,12 +21,9 @@ import org.apache.paimon.annotation.Public; import org.apache.paimon.memory.MemorySegment; import org.apache.paimon.memory.MemorySegmentUtils; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypeRoot; -import org.apache.paimon.types.DecimalType; -import org.apache.paimon.types.LocalZonedTimestampType; import org.apache.paimon.types.RowKind; -import org.apache.paimon.types.TimestampType; + +import javax.annotation.Nullable; import java.nio.ByteOrder; @@ -80,38 +77,6 @@ public static int calculateFixPartSizeInBytes(int arity) { return calculateBitSetWidthInBytes(arity) + 8 * arity; } - /** - * If it is a fixed-length field, we can call this BinaryRow's setXX method for in-place - * updates. If it is variable-length field, can't use this method, because the underlying data - * is stored continuously. - */ - public static boolean isInFixedLengthPart(DataType type) { - switch (type.getTypeRoot()) { - case BOOLEAN: - case TINYINT: - case SMALLINT: - case INTEGER: - case DATE: - case TIME_WITHOUT_TIME_ZONE: - case BIGINT: - case FLOAT: - case DOUBLE: - return true; - case DECIMAL: - return Decimal.isCompact(((DecimalType) type).getPrecision()); - case TIMESTAMP_WITHOUT_TIME_ZONE: - return Timestamp.isCompact(((TimestampType) type).getPrecision()); - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return Timestamp.isCompact(((LocalZonedTimestampType) type).getPrecision()); - default: - return false; - } - } - - public static boolean isMutable(DataType type) { - return isInFixedLengthPart(type) || type.getTypeRoot() == DataTypeRoot.DECIMAL; - } - private final int arity; private final int nullBitsSizeInBytes; @@ -446,4 +411,35 @@ public boolean equals(Object o) { public int hashCode() { return MemorySegmentUtils.hashByWords(segments, offset, sizeInBytes); } + + public static BinaryRow singleColumn(@Nullable Integer i) { + BinaryRow row = new BinaryRow(1); + BinaryRowWriter writer = new BinaryRowWriter(row); + writer.reset(); + if (i == null) { + writer.setNullAt(0); + } else { + writer.writeInt(0, i); + } + writer.complete(); + return row; + } + + public static BinaryRow singleColumn(@Nullable String string) { + BinaryString binaryString = string == null ? null : BinaryString.fromString(string); + return singleColumn(binaryString); + } + + public static BinaryRow singleColumn(@Nullable BinaryString string) { + BinaryRow row = new BinaryRow(1); + BinaryRowWriter writer = new BinaryRowWriter(row); + writer.reset(); + if (string == null) { + writer.setNullAt(0); + } else { + writer.writeString(0, string); + } + writer.complete(); + return row; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java index 83aef4f8480b..385609de3f84 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java @@ -34,16 +34,14 @@ public class DataFilePathFactory { public static final String CHANGELOG_FILE_PREFIX = "changelog-"; - public static final String BUCKET_PATH_PREFIX = "bucket-"; - - private final Path bucketDir; + private final Path parent; private final String uuid; private final AtomicInteger pathCount; private final String formatIdentifier; - public DataFilePathFactory(Path root, String partition, int bucket, String formatIdentifier) { - this.bucketDir = bucketPath(root, partition, bucket); + public DataFilePathFactory(Path parent, String formatIdentifier) { + this.parent = parent; this.uuid = UUID.randomUUID().toString(); this.pathCount = new AtomicInteger(0); @@ -60,11 +58,11 @@ public Path newChangelogPath() { private Path newPath(String prefix) { String name = prefix + uuid + "-" + pathCount.getAndIncrement() + "." + formatIdentifier; - return new Path(bucketDir, name); + return new Path(parent, name); } public Path toPath(String fileName) { - return new Path(bucketDir + "/" + fileName); + return new Path(parent + "/" + fileName); } @VisibleForTesting @@ -72,10 +70,6 @@ public String uuid() { return uuid; } - public static Path bucketPath(Path tablePath, String partition, int bucket) { - return new Path(tablePath + "/" + partition + "/" + BUCKET_PATH_PREFIX + bucket); - } - public static String formatIdentifier(String fileName) { int index = fileName.lastIndexOf('.'); if (index == -1) { diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java index 94df882ff3f9..a911b084742b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java @@ -18,6 +18,7 @@ package org.apache.paimon.operation; +import org.apache.paimon.CoreOptions; import org.apache.paimon.Snapshot; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.manifest.FileEntry; @@ -277,7 +278,7 @@ private Pair> doPlan() { ? "partition " + FileStorePathFactory.getPartitionComputer( partitionType, - FileStorePathFactory.PARTITION_DEFAULT_NAME + CoreOptions.PARTITION_DEFAULT_NAME .defaultValue()) .generatePartValues(file.partition()) : "table"; diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java b/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java index 4ddf96091a0b..8d85024cf00d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java @@ -56,7 +56,7 @@ import java.util.function.Predicate; import java.util.stream.Collectors; -import static org.apache.paimon.io.DataFilePathFactory.BUCKET_PATH_PREFIX; +import static org.apache.paimon.utils.FileStorePathFactory.BUCKET_PATH_PREFIX; /** * To remove the data files and metadata files that are not used by table (so-called "orphan diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/ManifestsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/ManifestsTable.java index d5e9f1647b56..7d7a9570e55f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/ManifestsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/ManifestsTable.java @@ -25,12 +25,11 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; import org.apache.paimon.format.FileFormat; -import org.apache.paimon.fs.FileIO; -import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.ManifestFileMeta; import org.apache.paimon.manifest.ManifestList; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.ReadonlyTable; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.InnerTableRead; @@ -74,13 +73,9 @@ public class ManifestsTable implements ReadonlyTable { new DataField(3, "num_deleted_files", new BigIntType(false)), new DataField(4, "schema_id", new BigIntType(false)))); - private final FileIO fileIO; - private final Path location; - private final Table dataTable; + private final FileStoreTable dataTable; - public ManifestsTable(FileIO fileIO, Path location, Table dataTable) { - this.fileIO = fileIO; - this.location = location; + public ManifestsTable(FileStoreTable dataTable) { this.dataTable = dataTable; } @@ -91,12 +86,12 @@ public InnerTableScan newScan() { @Override public InnerTableRead newRead() { - return new ManifestsRead(fileIO, dataTable); + return new ManifestsRead(dataTable); } @Override public String name() { - return location.getName() + SYSTEM_TABLE_SPLITTER + MANIFESTS; + return dataTable.name() + SYSTEM_TABLE_SPLITTER + MANIFESTS; } @Override @@ -111,7 +106,7 @@ public List primaryKeys() { @Override public Table copy(Map dynamicOptions) { - return new ManifestsTable(fileIO, location, dataTable.copy(dynamicOptions)); + return new ManifestsTable(dataTable.copy(dynamicOptions)); } private class ManifestsScan extends ReadOnceTableScan { @@ -125,9 +120,7 @@ public InnerTableScan withFilter(Predicate predicate) { @Override protected Plan innerPlan() { return () -> - Collections.singletonList( - new ManifestsSplit( - allManifests(fileIO, location, dataTable).size(), location)); + Collections.singletonList(new ManifestsSplit(allManifests(dataTable).size())); } } @@ -136,11 +129,9 @@ private static class ManifestsSplit implements Split { private static final long serialVersionUID = 1L; private final long rowCount; - private final Path location; - private ManifestsSplit(long rowCount, Path location) { + private ManifestsSplit(long rowCount) { this.rowCount = rowCount; - this.location = location; } @Override @@ -157,12 +148,12 @@ public boolean equals(Object o) { return false; } ManifestsSplit that = (ManifestsSplit) o; - return Objects.equals(location, that.location); + return Objects.equals(rowCount, that.rowCount); } @Override public int hashCode() { - return Objects.hash(location); + return Objects.hash(rowCount); } } @@ -170,12 +161,9 @@ private static class ManifestsRead implements InnerTableRead { private int[][] projection; - private final FileIO fileIO; + private final FileStoreTable dataTable; - private final Table dataTable; - - public ManifestsRead(FileIO fileIO, Table dataTable) { - this.fileIO = fileIO; + public ManifestsRead(FileStoreTable dataTable) { this.dataTable = dataTable; } @@ -201,8 +189,7 @@ public RecordReader createReader(Split split) throws IOException { if (!(split instanceof ManifestsSplit)) { throw new IllegalArgumentException("Unsupported split: " + split.getClass()); } - Path location = ((ManifestsSplit) split).location; - List manifestFileMetas = allManifests(fileIO, location, dataTable); + List manifestFileMetas = allManifests(dataTable); Iterator rows = Iterators.transform(manifestFileMetas.iterator(), this::toRow); @@ -224,10 +211,9 @@ private InternalRow toRow(ManifestFileMeta manifestFileMeta) { } } - private static List allManifests( - FileIO fileIO, Path location, Table dataTable) { + private static List allManifests(FileStoreTable dataTable) { CoreOptions coreOptions = CoreOptions.fromMap(dataTable.options()); - SnapshotManager snapshotManager = new SnapshotManager(fileIO, location); + SnapshotManager snapshotManager = dataTable.snapshotManager(); Long snapshotId = coreOptions.scanSnapshotId(); Snapshot snapshot = null; if (snapshotId != null && snapshotManager.snapshotExists(snapshotId)) { @@ -239,10 +225,11 @@ private static List allManifests( if (snapshot == null) { return Collections.emptyList(); } - FileStorePathFactory fileStorePathFactory = new FileStorePathFactory(location); + FileStorePathFactory fileStorePathFactory = dataTable.store().pathFactory(); FileFormat fileFormat = coreOptions.manifestFormat(); ManifestList manifestList = - new ManifestList.Factory(fileIO, fileFormat, fileStorePathFactory, null).create(); + new ManifestList.Factory(dataTable.fileIO(), fileFormat, fileStorePathFactory, null) + .create(); return snapshot.allManifests(manifestList); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java index 32aa9fa3596a..dcbe0fed1d0c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java @@ -59,7 +59,7 @@ public static Table load(String type, FileIO fileIO, FileStoreTable dataTable) { Path location = dataTable.location(); switch (type.toLowerCase()) { case MANIFESTS: - return new ManifestsTable(fileIO, location, dataTable); + return new ManifestsTable(dataTable); case SNAPSHOTS: return new SnapshotsTable(fileIO, location, dataTable); case OPTIONS: diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java index f9aaaf67b54d..903a814e13e2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java @@ -18,12 +18,10 @@ package org.apache.paimon.utils; -import org.apache.paimon.CoreOptions; import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.fs.Path; import org.apache.paimon.io.DataFilePathFactory; -import org.apache.paimon.options.ConfigOption; import org.apache.paimon.types.RowType; import javax.annotation.concurrent.ThreadSafe; @@ -33,19 +31,11 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; -import static org.apache.paimon.options.ConfigOptions.key; - /** Factory which produces {@link Path}s for manifest files. */ @ThreadSafe public class FileStorePathFactory { - public static final ConfigOption PARTITION_DEFAULT_NAME = - key("partition.default-name") - .stringType() - .defaultValue("__DEFAULT_PARTITION__") - .withDescription( - "The default partition name in case the dynamic partition" - + " column value is null/empty string."); + public static final String BUCKET_PATH_PREFIX = "bucket-"; private final Path root; private final String uuid; @@ -58,15 +48,6 @@ public class FileStorePathFactory { private final AtomicInteger indexFileCount; private final AtomicInteger statsFileCount; - public FileStorePathFactory(Path root) { - this( - root, - RowType.builder().build(), - PARTITION_DEFAULT_NAME.defaultValue(), - CoreOptions.FILE_FORMAT.defaultValue().toString()); - } - - // for tables without partition, partitionType should be a row type with 0 columns (not null) public FileStorePathFactory( Path root, RowType partitionType, String defaultPartValue, String formatIdentifier) { this.root = root; @@ -116,12 +97,12 @@ public Path toManifestListPath(String manifestListName) { } public DataFilePathFactory createDataFilePathFactory(BinaryRow partition, int bucket) { - return new DataFilePathFactory( - root, getPartitionString(partition), bucket, formatIdentifier); + return new DataFilePathFactory(bucketPath(partition, bucket), formatIdentifier); } public Path bucketPath(BinaryRow partition, int bucket) { - return DataFilePathFactory.bucketPath(root, getPartitionString(partition), bucket); + return new Path( + root + "/" + getPartitionString(partition) + "/" + BUCKET_PATH_PREFIX + bucket); } /** IMPORTANT: This method is NOT THREAD SAFE. */ diff --git a/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java b/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java index c954aec1f114..8b66daae62ff 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java @@ -515,9 +515,7 @@ private InternalRow row(int id, String name, String dt) { private DataFilePathFactory createPathFactory() { return new DataFilePathFactory( - new Path(tempDir.toString()), - "dt=" + PART, - 0, + new Path(tempDir + "/dt=" + PART + "/bucket-0"), CoreOptions.FILE_FORMAT.defaultValue().toString()); } diff --git a/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java b/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java index dcc02cde63de..0b623f90c920 100644 --- a/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java @@ -63,7 +63,7 @@ public void testFileSuffix(@TempDir java.nio.file.Path tempDir) throws Exception assertThat(path.toString().endsWith(format)).isTrue(); DataFilePathFactory dataFilePathFactory = - new DataFilePathFactory(new Path(tempDir.toString()), "dt=1", 1, format); + new DataFilePathFactory(new Path(tempDir + "/dt=1/bucket-1"), format); FileFormat fileFormat = FileFormat.fromIdentifier(format, new Options()); LinkedList toCompact = new LinkedList<>(); CoreOptions options = new CoreOptions(new HashMap<>()); diff --git a/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java b/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java index c579cabdafd1..bf741e026887 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java @@ -35,9 +35,7 @@ public class DataFilePathFactoryTest { public void testNoPartition() { DataFilePathFactory pathFactory = new DataFilePathFactory( - new Path(tempDir.toString()), - "", - 123, + new Path(tempDir + "/bucket-123"), CoreOptions.FILE_FORMAT.defaultValue().toString()); String uuid = pathFactory.uuid(); @@ -61,9 +59,7 @@ public void testNoPartition() { public void testWithPartition() { DataFilePathFactory pathFactory = new DataFilePathFactory( - new Path(tempDir.toString()), - "dt=20211224", - 123, + new Path(tempDir + "/dt=20211224/bucket-123"), CoreOptions.FILE_FORMAT.defaultValue().toString()); String uuid = pathFactory.uuid(); diff --git a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java index 552eac0a13f2..f049d07a1e8b 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java @@ -64,6 +64,7 @@ import static org.apache.paimon.TestKeyValueGenerator.KEY_TYPE; import static org.apache.paimon.TestKeyValueGenerator.createTestSchemaManager; import static org.apache.paimon.stats.StatsTestUtils.convertWithoutSchemaEvolution; +import static org.apache.paimon.utils.FileStorePathFactoryTest.createNonPartFactory; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -279,7 +280,7 @@ private KeyValueFileReaderFactory createReaderFactory( String pathStr, String format, int[][] keyProjection, int[][] valueProjection) { Path path = new Path(pathStr); FileIO fileIO = FileIOFinder.find(path); - FileStorePathFactory pathFactory = new FileStorePathFactory(path); + FileStorePathFactory pathFactory = createNonPartFactory(path); KeyValueFileReaderFactory.Builder builder = KeyValueFileReaderFactory.builder( fileIO, diff --git a/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java b/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java index 8d42d20e88d4..d91b8a9a0b94 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java @@ -69,9 +69,7 @@ public void initialize(String identifier) { LocalFileIO.create(), fileFormat.createWriterFactory(SCHEMA), new DataFilePathFactory( - new Path(tempDir.toString()), - "", - 0, + new Path(tempDir + "/bucket-0"), CoreOptions.FILE_FORMAT .defaultValue() .toString()) diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java index d53b94155229..9ec0e38e8d10 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java @@ -63,6 +63,7 @@ import static org.apache.paimon.CoreOptions.TARGET_FILE_SIZE; import static org.apache.paimon.io.DataFileTestUtils.row; +import static org.apache.paimon.utils.FileStorePathFactoryTest.createNonPartFactory; import static org.assertj.core.api.Assertions.assertThat; /** Test {@link LookupLevels} for contains. */ @@ -217,7 +218,7 @@ private KeyValueFileWriterFactory createWriterFactory() { Path path = new Path(tempDir.toUri().toString()); String identifier = "avro"; Map pathFactoryMap = new HashMap<>(); - pathFactoryMap.put(identifier, new FileStorePathFactory(path)); + pathFactoryMap.put(identifier, createNonPartFactory(path)); return KeyValueFileWriterFactory.builder( FileIOFinder.find(path), 0, @@ -239,7 +240,7 @@ private KeyValueFileReaderFactory createReaderFactory() { keyType, rowType, ignore -> new FlushingFileFormat("avro"), - new FileStorePathFactory(path), + createNonPartFactory(path), new KeyValueFieldsExtractor() { @Override public List keyFields(TableSchema schema) { diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java index f4d27784a56d..d38b516fcf4f 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java @@ -64,6 +64,7 @@ import static org.apache.paimon.CoreOptions.TARGET_FILE_SIZE; import static org.apache.paimon.KeyValue.UNKNOWN_SEQUENCE; import static org.apache.paimon.io.DataFileTestUtils.row; +import static org.apache.paimon.utils.FileStorePathFactoryTest.createNonPartFactory; import static org.assertj.core.api.Assertions.assertThat; /** Test {@link LookupLevels}. */ @@ -297,7 +298,7 @@ private KeyValueFileWriterFactory createWriterFactory() { Path path = new Path(tempDir.toUri().toString()); String identifier = "avro"; Map pathFactoryMap = new HashMap<>(); - pathFactoryMap.put(identifier, new FileStorePathFactory(path)); + pathFactoryMap.put(identifier, createNonPartFactory(path)); return KeyValueFileWriterFactory.builder( FileIOFinder.find(path), 0, @@ -319,7 +320,7 @@ private KeyValueFileReaderFactory createReaderFactory() { keyType, rowType, ignore -> new FlushingFileFormat("avro"), - new FileStorePathFactory(path), + createNonPartFactory(path), new KeyValueFieldsExtractor() { @Override public List keyFields(TableSchema schema) { 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 a3bcc9bfa32c..fef552abdabb 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 @@ -87,6 +87,7 @@ import java.util.stream.Collectors; import static java.util.Collections.singletonList; +import static org.apache.paimon.utils.FileStorePathFactoryTest.createNonPartFactory; import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link MergeTreeReaders} and {@link MergeTreeWriter}. */ @@ -108,7 +109,7 @@ public abstract class MergeTreeTestBase { @BeforeEach public void beforeEach() throws IOException { path = new Path(tempDir.toString()); - pathFactory = new FileStorePathFactory(path); + pathFactory = createNonPartFactory(path); comparator = Comparator.comparingInt(o -> o.getInt(0)); recreateMergeTree(1024 * 1024); Path bucketDir = writerFactory.pathFactory(0).toPath("ignore").getParent(); diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/OrphanFilesCleanTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/OrphanFilesCleanTest.java index cf7535f7688f..c20cdf21d784 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/OrphanFilesCleanTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/OrphanFilesCleanTest.java @@ -72,9 +72,9 @@ import java.util.function.Predicate; import java.util.stream.Collectors; -import static org.apache.paimon.io.DataFilePathFactory.BUCKET_PATH_PREFIX; import static org.apache.paimon.io.DataFilePathFactory.CHANGELOG_FILE_PREFIX; import static org.apache.paimon.io.DataFilePathFactory.DATA_FILE_PREFIX; +import static org.apache.paimon.utils.FileStorePathFactory.BUCKET_PATH_PREFIX; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link OrphanFilesClean}. */ diff --git a/paimon-core/src/test/java/org/apache/paimon/table/sink/TableCommitTest.java b/paimon-core/src/test/java/org/apache/paimon/table/sink/TableCommitTest.java index bed06293a83a..d9aa14ce6490 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/sink/TableCommitTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/sink/TableCommitTest.java @@ -38,7 +38,6 @@ import org.apache.paimon.types.RowType; import org.apache.paimon.utils.ExceptionUtils; import org.apache.paimon.utils.FailingFileIO; -import org.apache.paimon.utils.FileStorePathFactory; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -56,6 +55,7 @@ import java.util.stream.Collectors; import java.util.stream.LongStream; +import static org.apache.paimon.utils.FileStorePathFactoryTest.createNonPartFactory; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -232,7 +232,7 @@ public void testRecoverDeletedFiles() throws Exception { (CommitMessageImpl) messages0.get(0), (CommitMessageImpl) messages1.get(0))) { DataFilePathFactory pathFactory = - new FileStorePathFactory(new Path(path)) + createNonPartFactory(new Path(path)) .createDataFilePathFactory(message.partition(), message.bucket()); Path file = message.newFilesIncrement().newFiles().get(0).collectFiles(pathFactory).get(0); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/system/ManifestsTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/system/ManifestsTableTest.java index fa659430c48f..8b084ddb42fa 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/system/ManifestsTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/system/ManifestsTableTest.java @@ -35,7 +35,6 @@ import org.apache.paimon.table.Table; import org.apache.paimon.table.TableTestBase; import org.apache.paimon.types.DataTypes; -import org.apache.paimon.utils.FileStorePathFactory; import org.apache.paimon.utils.SnapshotManager; import org.junit.jupiter.api.BeforeEach; @@ -45,6 +44,7 @@ import java.util.Collections; import java.util.List; +import static org.apache.paimon.utils.FileStorePathFactoryTest.createNonPartFactory; import static org.assertj.core.api.Assertions.assertThat; /** Unit tests for {@link ManifestsTable}. */ @@ -82,7 +82,7 @@ public void before() throws Exception { FileFormat.fromIdentifier( CoreOptions.MANIFEST_FORMAT.defaultValue().toString(), new Options()), - new FileStorePathFactory(tablePath), + createNonPartFactory(tablePath), null); manifestList = factory.create(); diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java index 8e9088bb4511..e0031f7c1303 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java @@ -32,6 +32,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import static org.apache.paimon.CoreOptions.PARTITION_DEFAULT_NAME; import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link FileStorePathFactory}. */ @@ -41,7 +42,7 @@ public class FileStorePathFactoryTest { @Test public void testManifestPaths() { - FileStorePathFactory pathFactory = new FileStorePathFactory(new Path(tempDir.toString())); + FileStorePathFactory pathFactory = createNonPartFactory(new Path(tempDir.toString())); String uuid = pathFactory.uuid(); for (int i = 0; i < 20; i++) { @@ -68,7 +69,7 @@ public void testManifestPaths() { @Test public void testCreateDataFilePathFactoryNoPartition() { - FileStorePathFactory pathFactory = new FileStorePathFactory(new Path(tempDir.toString())); + FileStorePathFactory pathFactory = createNonPartFactory(new Path(tempDir.toString())); DataFilePathFactory dataFilePathFactory = pathFactory.createDataFilePathFactory(new BinaryRow(0), 123); assertThat(dataFilePathFactory.toPath("my-data-file-name")) @@ -113,4 +114,12 @@ private void assertPartition( .isEqualTo( new Path(tempDir.toString() + expected + "/bucket-123/my-data-file-name")); } + + public static FileStorePathFactory createNonPartFactory(Path root) { + return new FileStorePathFactory( + root, + RowType.builder().build(), + PARTITION_DEFAULT_NAME.defaultValue(), + CoreOptions.FILE_FORMAT.defaultValue().toString()); + } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala index 28a1dc457b62..8c5192e013d9 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala @@ -18,6 +18,7 @@ package org.apache.paimon.spark +import org.apache.paimon.CoreOptions import org.apache.paimon.operation.FileStoreCommit import org.apache.paimon.table.FileStoreTable import org.apache.paimon.table.sink.BatchWriteBuilder @@ -54,7 +55,7 @@ trait PaimonPartitionManagement extends SupportsPartitionManagement { .apply(internalRow) .asInstanceOf[Row] val rowDataPartitionComputer = new RowDataPartitionComputer( - FileStorePathFactory.PARTITION_DEFAULT_NAME.defaultValue, + CoreOptions.PARTITION_DEFAULT_NAME.defaultValue, tableRowType, partitionKeys.asScala.toArray) val partitionMap = rowDataPartitionComputer.generatePartValues(new SparkRow(tableRowType, row)) From e6df3b72e6cba499d53f91e0911564e02598ca75 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Tue, 26 Mar 2024 16:58:56 +0800 Subject: [PATCH 63/79] [core] Add TableFormatBenchmark to test format only --- .../paimon/benchmark/TableBenchmark.java | 14 +-- .../benchmark/TableFormatBenchmark.java | 104 ++++++++++++++++++ .../paimon/benchmark/TableReadBenchmark.java | 33 ++++-- 3 files changed, 135 insertions(+), 16 deletions(-) create mode 100644 paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableFormatBenchmark.java diff --git a/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableBenchmark.java b/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableBenchmark.java index 0df1bc2ec72f..c7bd2c22ead8 100644 --- a/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableBenchmark.java +++ b/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableBenchmark.java @@ -41,8 +41,6 @@ import java.util.Collections; import java.util.List; -import static java.util.Collections.singletonList; - /** Base class for table benchmark. */ public class TableBenchmark { @@ -53,6 +51,11 @@ public class TableBenchmark { private final RandomDataGenerator random = new RandomDataGenerator(); protected Table createTable(Options tableOptions, String tableName) throws Exception { + return createTable(tableOptions, tableName, Collections.singletonList("k")); + } + + protected Table createTable(Options tableOptions, String tableName, List primaryKeys) + throws Exception { Options catalogOptions = new Options(); catalogOptions.set(CatalogOptions.WAREHOUSE, tempFile.toUri().toString()); Catalog catalog = CatalogFactory.createCatalog(CatalogContext.create(catalogOptions)); @@ -66,12 +69,7 @@ protected Table createTable(Options tableOptions, String tableName) throws Excep } tableOptions.set(CoreOptions.SNAPSHOT_NUM_RETAINED_MAX, 10); Schema schema = - new Schema( - fields, - Collections.emptyList(), - singletonList("k"), - tableOptions.toMap(), - ""); + new Schema(fields, Collections.emptyList(), primaryKeys, tableOptions.toMap(), ""); Identifier identifier = Identifier.create(database, tableName); catalog.createTable(identifier, schema, false); return catalog.getTable(identifier); diff --git a/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableFormatBenchmark.java b/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableFormatBenchmark.java new file mode 100644 index 000000000000..c5648a785d82 --- /dev/null +++ b/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableFormatBenchmark.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.benchmark; + +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.options.Options; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.sink.CommitMessage; +import org.apache.paimon.table.sink.StreamTableCommit; +import org.apache.paimon.table.sink.StreamTableWrite; +import org.apache.paimon.table.sink.StreamWriteBuilder; +import org.apache.paimon.table.source.Split; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +/** Benchmark for table read. */ +public class TableFormatBenchmark extends TableBenchmark { + + private final int rowCount = 10000000; + + @Test + public void testRead() throws Exception { + innerTest(prepareData()); + /* + * OpenJDK 64-Bit Server VM 1.8.0_292-b10 on Mac OS X 10.16 + * Apple M1 Pro + * read: Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative + * ------------------------------------------------------------------------------------------------ + * OPERATORTEST_read_read-orc 11314 / 11366 2651.6 377.1 1.0X + */ + } + + private void innerTest(Table table) { + int readTime = 3; + Benchmark benchmark = + new Benchmark("read", readTime * rowCount) + .setNumWarmupIters(1) + .setOutputPerIteration(true); + + benchmark.addCase( + "read", + 5, + () -> { + for (int i = 0; i < readTime; i++) { + List splits = table.newReadBuilder().newScan().plan().splits(); + AtomicLong readCount = new AtomicLong(0); + try { + for (Split split : splits) { + RecordReader reader = + table.newReadBuilder().newRead().createReader(split); + reader.forEachRemaining(row -> readCount.incrementAndGet()); + } + System.out.printf("Finish read %d rows.\n", readCount.get()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + }); + benchmark.run(); + } + + private Table prepareData() throws Exception { + Table table = createTable(new Options(), "table", Collections.emptyList()); + StreamWriteBuilder writeBuilder = table.newStreamWriteBuilder(); + StreamTableWrite write = writeBuilder.newWrite(); + StreamTableCommit commit = writeBuilder.newCommit(); + AtomicInteger writeCount = new AtomicInteger(0); + for (int i = 0; i < rowCount; i++) { + try { + write.write(newRandomRow()); + writeCount.incrementAndGet(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + List commitMessages = write.prepareCommit(true, 1); + commit.commit(1, commitMessages); + + write.close(); + return table; + } +} diff --git a/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableReadBenchmark.java b/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableReadBenchmark.java index f089ef234c42..f0af590a00f7 100644 --- a/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableReadBenchmark.java +++ b/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/TableReadBenchmark.java @@ -31,7 +31,7 @@ import org.junit.jupiter.api.Test; -import java.util.LinkedHashMap; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; @@ -43,20 +43,37 @@ public class TableReadBenchmark extends TableBenchmark { private final int rowCount = 1000000; @Test - public void testRead() throws Exception { - Map tables = new LinkedHashMap<>(); - tables.put("orc", prepareData(orc(), "orc")); - tables.put("parquet", prepareData(parquet(), "parquet")); - tables.put("avro", prepareData(avro(), "avro")); - - innerTest(tables); + public void testOrcRead() throws Exception { + innerTest(Collections.singletonMap("orc", prepareData(orc(), "orc"))); /* * OpenJDK 64-Bit Server VM 1.8.0_292-b10 on Mac OS X 10.16 * Apple M1 Pro * read: Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative * ------------------------------------------------------------------------------------------------ * OPERATORTEST_read_read-orc 1046 / 1295 2867.3 348.8 1.0X + */ + } + + @Test + public void testParquetRead() throws Exception { + innerTest(Collections.singletonMap("parquet", prepareData(parquet(), "parquet"))); + /* + * OpenJDK 64-Bit Server VM 1.8.0_292-b10 on Mac OS X 10.16 + * Apple M1 Pro + * read: Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative + * ------------------------------------------------------------------------------------------------ * OPERATORTEST_read_read-parquet 3076 / 5295 975.4 1025.2 0.3X + */ + } + + @Test + public void testAvroRead() throws Exception { + innerTest(Collections.singletonMap("avro", prepareData(avro(), "avro"))); + /* + * OpenJDK 64-Bit Server VM 1.8.0_292-b10 on Mac OS X 10.16 + * Apple M1 Pro + * read: Best/Avg Time(ms) Row Rate(K/s) Per Row(ns) Relative + * ------------------------------------------------------------------------------------------------ * OPERATORTEST_read_read-avro 4156 / 4362 721.8 1385.5 0.3X */ } From 13de145fc92bb63f8055ef190d3044d26b144591 Mon Sep 17 00:00:00 2001 From: Yann Byron Date: Tue, 26 Mar 2024 17:32:39 +0800 Subject: [PATCH 64/79] [spark][core] Support input_file_name UDF (#3094) --- .../data/columnar/ColumnarRowIterator.java | 17 +- ...nIterator.java => FileRecordIterator.java} | 28 ++- .../apache/paimon/reader/RecordReader.java | 4 +- .../ApplyDeletionVectorReader.java | 6 +- .../apache/paimon/mergetree/LookupLevels.java | 7 +- .../paimon/format/orc/OrcReaderFactory.java | 15 +- .../format/parquet/ParquetReaderFactory.java | 23 ++- .../paimon/spark/PaimonPartitionReader.scala | 2 +- .../spark/PaimonRecordReaderIterator.scala | 105 ++++++++++++ .../scala/org/apache/spark/sql/Utils.scala | 9 + .../paimon/spark/sql/PaimonQueryTest.scala | 161 ++++++++++++++++++ .../paimon/spark/sql/WithTableOptions.scala | 2 + 12 files changed, 345 insertions(+), 34 deletions(-) rename paimon-common/src/main/java/org/apache/paimon/reader/{RecordWithPositionIterator.java => FileRecordIterator.java} (77%) create mode 100644 paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonRecordReaderIterator.scala create mode 100644 paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala diff --git a/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRowIterator.java b/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRowIterator.java index 6de861af019c..13d706cf6b6a 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRowIterator.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRowIterator.java @@ -20,8 +20,9 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.PartitionInfo; +import org.apache.paimon.fs.Path; +import org.apache.paimon.reader.FileRecordIterator; import org.apache.paimon.reader.RecordReader; -import org.apache.paimon.reader.RecordWithPositionIterator; import org.apache.paimon.utils.RecyclableIterator; import org.apache.paimon.utils.VectorMappingUtils; @@ -32,8 +33,9 @@ * {@link ColumnarRow#setRowId}. */ public class ColumnarRowIterator extends RecyclableIterator - implements RecordWithPositionIterator { + implements FileRecordIterator { + private final Path filePath; private final ColumnarRow rowData; private final Runnable recycler; @@ -41,8 +43,9 @@ public class ColumnarRowIterator extends RecyclableIterator private int nextPos; private long nextGlobalPos; - public ColumnarRowIterator(ColumnarRow rowData, @Nullable Runnable recycler) { + public ColumnarRowIterator(Path filePath, ColumnarRow rowData, @Nullable Runnable recycler) { super(recycler); + this.filePath = filePath; this.rowData = rowData; this.recycler = recycler; } @@ -74,8 +77,14 @@ public long returnedPosition() { return nextGlobalPos - 1; } + @Override + public Path filePath() { + return this.filePath; + } + public ColumnarRowIterator copy(ColumnVector[] vectors) { - ColumnarRowIterator newIterator = new ColumnarRowIterator(rowData.copy(vectors), recycler); + ColumnarRowIterator newIterator = + new ColumnarRowIterator(filePath, rowData.copy(vectors), recycler); newIterator.reset(num, nextGlobalPos); return newIterator; } diff --git a/paimon-common/src/main/java/org/apache/paimon/reader/RecordWithPositionIterator.java b/paimon-common/src/main/java/org/apache/paimon/reader/FileRecordIterator.java similarity index 77% rename from paimon-common/src/main/java/org/apache/paimon/reader/RecordWithPositionIterator.java rename to paimon-common/src/main/java/org/apache/paimon/reader/FileRecordIterator.java index e4778413a34f..0cef8cc001e6 100644 --- a/paimon-common/src/main/java/org/apache/paimon/reader/RecordWithPositionIterator.java +++ b/paimon-common/src/main/java/org/apache/paimon/reader/FileRecordIterator.java @@ -18,6 +18,7 @@ package org.apache.paimon.reader; +import org.apache.paimon.fs.Path; import org.apache.paimon.utils.Filter; import javax.annotation.Nullable; @@ -30,7 +31,7 @@ * * @param The type of the record. */ -public interface RecordWithPositionIterator extends RecordReader.RecordIterator { +public interface FileRecordIterator extends RecordReader.RecordIterator { /** * Get the row position of the row returned by {@link RecordReader.RecordIterator#next}. @@ -39,15 +40,23 @@ public interface RecordWithPositionIterator extends RecordReader.RecordIterat */ long returnedPosition(); + /** @return the file path */ + Path filePath(); + @Override - default RecordWithPositionIterator transform(Function function) { - RecordWithPositionIterator thisIterator = this; - return new RecordWithPositionIterator() { + default FileRecordIterator transform(Function function) { + FileRecordIterator thisIterator = this; + return new FileRecordIterator() { @Override public long returnedPosition() { return thisIterator.returnedPosition(); } + @Override + public Path filePath() { + return thisIterator.filePath(); + } + @Nullable @Override public R next() throws IOException { @@ -66,14 +75,19 @@ public void releaseBatch() { } @Override - default RecordWithPositionIterator filter(Filter filter) { - RecordWithPositionIterator thisIterator = this; - return new RecordWithPositionIterator() { + default FileRecordIterator filter(Filter filter) { + FileRecordIterator thisIterator = this; + return new FileRecordIterator() { @Override public long returnedPosition() { return thisIterator.returnedPosition(); } + @Override + public Path filePath() { + return thisIterator.filePath(); + } + @Nullable @Override public T next() throws IOException { diff --git a/paimon-common/src/main/java/org/apache/paimon/reader/RecordReader.java b/paimon-common/src/main/java/org/apache/paimon/reader/RecordReader.java index 276a855719d6..5c7482d9d9c1 100644 --- a/paimon-common/src/main/java/org/apache/paimon/reader/RecordReader.java +++ b/paimon-common/src/main/java/org/apache/paimon/reader/RecordReader.java @@ -149,11 +149,11 @@ default void forEachRemaining(Consumer action) throws IOException { */ default void forEachRemainingWithPosition(BiConsumer action) throws IOException { - RecordWithPositionIterator batch; + FileRecordIterator batch; T record; try { - while ((batch = (RecordWithPositionIterator) readBatch()) != null) { + while ((batch = (FileRecordIterator) readBatch()) != null) { while ((record = batch.next()) != null) { action.accept(batch.returnedPosition(), record); } diff --git a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java index 3bba07506338..dadde99eac5f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java @@ -18,8 +18,8 @@ package org.apache.paimon.deletionvectors; +import org.apache.paimon.reader.FileRecordIterator; import org.apache.paimon.reader.RecordReader; -import org.apache.paimon.reader.RecordWithPositionIterator; import javax.annotation.Nullable; @@ -62,10 +62,10 @@ public RecordIterator readBatch() throws IOException { } checkArgument( - batch instanceof RecordWithPositionIterator, + batch instanceof FileRecordIterator, "There is a bug, RecordIterator in ApplyDeletionVectorReader must be RecordWithPositionIterator"); - RecordWithPositionIterator batchWithPosition = (RecordWithPositionIterator) batch; + FileRecordIterator batchWithPosition = (FileRecordIterator) batch; return batchWithPosition.filter( a -> !deletionVector.isDeleted(batchWithPosition.returnedPosition())); diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/LookupLevels.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/LookupLevels.java index dd45e7fc19c8..d6024ebb88cf 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/LookupLevels.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/LookupLevels.java @@ -28,8 +28,8 @@ import org.apache.paimon.lookup.LookupStoreWriter; import org.apache.paimon.memory.MemorySegment; import org.apache.paimon.options.MemorySize; +import org.apache.paimon.reader.FileRecordIterator; import org.apache.paimon.reader.RecordReader; -import org.apache.paimon.reader.RecordWithPositionIterator; import org.apache.paimon.types.RowKind; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.BloomFilter; @@ -176,9 +176,8 @@ private LookupFile createLookupFile(DataFileMeta file) throws IOException { try (RecordReader reader = fileReaderFactory.apply(file)) { KeyValue kv; if (valueProcessor.withPosition()) { - RecordWithPositionIterator batch; - while ((batch = (RecordWithPositionIterator) reader.readBatch()) - != null) { + FileRecordIterator batch; + while ((batch = (FileRecordIterator) reader.readBatch()) != null) { while ((kv = batch.next()) != null) { byte[] keyBytes = keySerializer.serializeToBytes(kv.key()); byte[] valueBytes = 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 55cff92980bf..8cf95fad3290 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 @@ -28,6 +28,7 @@ import org.apache.paimon.format.fs.HadoopReadOnlyFileSystem; import org.apache.paimon.format.orc.filter.OrcFilters; import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; import org.apache.paimon.reader.RecordReader.RecordIterator; import org.apache.paimon.types.DataType; import org.apache.paimon.types.RowType; @@ -94,7 +95,7 @@ public OrcVectorizedReader createReader(FormatReaderFactory.Context context) context instanceof OrcFormatReaderContext ? ((OrcFormatReaderContext) context).poolSize() : 1; - Pool poolOfBatches = createPoolOfBatches(poolSize); + Pool poolOfBatches = createPoolOfBatches(context.filePath(), poolSize); RecordReader orcReader = createRecordReader( @@ -114,7 +115,7 @@ public OrcVectorizedReader createReader(FormatReaderFactory.Context context) * conversion from the ORC representation to the result format. */ public OrcReaderBatch createReaderBatch( - VectorizedRowBatch orcBatch, Pool.Recycler recycler) { + Path filePath, VectorizedRowBatch orcBatch, Pool.Recycler recycler) { List tableFieldNames = tableType.getFieldNames(); List tableFieldTypes = tableType.getFieldTypes(); @@ -125,17 +126,17 @@ public OrcReaderBatch createReaderBatch( DataType type = tableFieldTypes.get(i); vectors[i] = createPaimonVector(orcBatch.cols[tableFieldNames.indexOf(name)], type); } - return new OrcReaderBatch(orcBatch, new VectorizedColumnBatch(vectors), recycler); + return new OrcReaderBatch(filePath, orcBatch, new VectorizedColumnBatch(vectors), recycler); } // ------------------------------------------------------------------------ - private Pool createPoolOfBatches(int numBatches) { + private Pool createPoolOfBatches(Path filePath, int numBatches) { final Pool pool = new Pool<>(numBatches); for (int i = 0; i < numBatches; i++) { final VectorizedRowBatch orcBatch = createBatchWrapper(schema, batchSize / numBatches); - final OrcReaderBatch batch = createReaderBatch(orcBatch, pool.recycler()); + final OrcReaderBatch batch = createReaderBatch(filePath, orcBatch, pool.recycler()); pool.add(batch); } @@ -153,6 +154,7 @@ private static class OrcReaderBatch { private final ColumnarRowIterator result; protected OrcReaderBatch( + final Path filePath, final VectorizedRowBatch orcVectorizedRowBatch, final VectorizedColumnBatch paimonColumnBatch, final Pool.Recycler recycler) { @@ -160,7 +162,8 @@ protected OrcReaderBatch( this.recycler = checkNotNull(recycler); this.paimonColumnBatch = paimonColumnBatch; this.result = - new ColumnarRowIterator(new ColumnarRow(paimonColumnBatch), this::recycle); + new ColumnarRowIterator( + filePath, new ColumnarRow(paimonColumnBatch), this::recycle); } /** 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 ed778c0bf018..004a0d655e96 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 @@ -28,6 +28,7 @@ import org.apache.paimon.format.parquet.reader.ColumnReader; import org.apache.paimon.format.parquet.reader.ParquetDecimalVector; import org.apache.paimon.format.parquet.reader.ParquetTimestampVector; +import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.reader.RecordReader.RecordIterator; @@ -100,7 +101,8 @@ public ParquetReader createReader(FormatReaderFactory.Context context) throws IO checkSchema(fileSchema, requestedSchema); - Pool poolOfBatches = createPoolOfBatches(requestedSchema); + Pool poolOfBatches = + createPoolOfBatches(context.filePath(), requestedSchema); return new ParquetReader(reader, requestedSchema, reader.getRecordCount(), poolOfBatches); } @@ -174,21 +176,24 @@ private void checkSchema(MessageType fileSchema, MessageType requestedSchema) } } - private Pool createPoolOfBatches(MessageType requestedSchema) { + private Pool createPoolOfBatches( + Path filePath, MessageType requestedSchema) { // In a VectorizedColumnBatch, the dictionary will be lazied deserialized. // If there are multiple batches at the same time, there may be thread safety problems, // because the deserialization of the dictionary depends on some internal structures. // We need set poolCapacity to 1. Pool pool = new Pool<>(1); - pool.add(createReaderBatch(requestedSchema, pool.recycler())); + pool.add(createReaderBatch(filePath, requestedSchema, pool.recycler())); return pool; } private ParquetReaderBatch createReaderBatch( - MessageType requestedSchema, Pool.Recycler recycler) { + Path filePath, + MessageType requestedSchema, + Pool.Recycler recycler) { WritableColumnVector[] writableVectors = createWritableVectors(requestedSchema); VectorizedColumnBatch columnarBatch = createVectorizedColumnBatch(writableVectors); - return createReaderBatch(writableVectors, columnarBatch, recycler); + return createReaderBatch(filePath, writableVectors, columnarBatch, recycler); } private WritableColumnVector[] createWritableVectors(MessageType requestedSchema) { @@ -361,10 +366,11 @@ public void close() throws IOException { } private ParquetReaderBatch createReaderBatch( + Path filePath, WritableColumnVector[] writableVectors, VectorizedColumnBatch columnarBatch, Pool.Recycler recycler) { - return new ParquetReaderBatch(writableVectors, columnarBatch, recycler); + return new ParquetReaderBatch(filePath, writableVectors, columnarBatch, recycler); } private static class ParquetReaderBatch { @@ -376,13 +382,16 @@ private static class ParquetReaderBatch { private final ColumnarRowIterator result; protected ParquetReaderBatch( + Path filePath, WritableColumnVector[] writableVectors, VectorizedColumnBatch columnarBatch, Pool.Recycler recycler) { this.writableVectors = writableVectors; this.columnarBatch = columnarBatch; this.recycler = recycler; - this.result = new ColumnarRowIterator(new ColumnarRow(columnarBatch), this::recycle); + this.result = + new ColumnarRowIterator( + filePath, new ColumnarRow(columnarBatch), this::recycle); } public void recycle() { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala index cfb8803b4cee..c4e694814453 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala @@ -40,7 +40,7 @@ case class PaimonPartitionReader( private lazy val iterator = { val reader = readFunc(split) - new RecordReaderIterator[PaimonInternalRow](reader) + PaimonRecordReaderIterator(reader) } override def next(): Boolean = { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonRecordReaderIterator.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonRecordReaderIterator.scala new file mode 100644 index 000000000000..3debb5e1891d --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonRecordReaderIterator.scala @@ -0,0 +1,105 @@ +/* + * 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 + +import org.apache.paimon.data.{InternalRow => PaimonInternalRow} +import org.apache.paimon.fs.Path +import org.apache.paimon.reader.{FileRecordIterator, RecordReader} +import org.apache.paimon.utils.CloseableIterator + +import org.apache.spark.sql.Utils + +import java.io.IOException + +case class PaimonRecordReaderIterator(reader: RecordReader[PaimonInternalRow]) + extends CloseableIterator[PaimonInternalRow] { + + private var lastFilePath: Path = _ + private var currentIterator: RecordReader.RecordIterator[PaimonInternalRow] = readBatch() + private var advanced = false + private var currentResult: PaimonInternalRow = _ + + override def hasNext: Boolean = { + if (currentIterator == null) { + false + } else { + advanceIfNeeded() + currentResult != null + } + } + + override def next(): PaimonInternalRow = { + if (!hasNext) { + null + } else { + advanced = false + currentResult + } + } + + override def close(): Unit = { + try { + if (currentIterator != null) { + currentIterator.releaseBatch() + currentResult == null + } + } finally { + reader.close() + Utils.unsetInputFileName() + } + } + + private def readBatch(): RecordReader.RecordIterator[PaimonInternalRow] = { + val iter = reader.readBatch() + iter match { + case fileRecordIterator: FileRecordIterator[_] => + if (lastFilePath != fileRecordIterator.filePath()) { + Utils.setInputFileName(fileRecordIterator.filePath().toUri.toString) + lastFilePath = fileRecordIterator.filePath() + } + case _ => + } + iter + } + + private def advanceIfNeeded(): Unit = { + if (!advanced) { + advanced = true + try { + var stop = false + while (!stop) { + currentResult = currentIterator.next + if (currentResult != null) { + stop = true + } else { + currentIterator.releaseBatch() + currentIterator = null + currentIterator = readBatch() + if (currentIterator == null) { + stop = true + } + } + } + } catch { + case e: IOException => + throw new RuntimeException(e) + } + } + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/Utils.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/Utils.scala index 8f7e5aaf7c5c..4767dab39468 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/Utils.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/Utils.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql +import org.apache.spark.rdd.InputFileBlockHolder import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.expressions.{FieldReference, NamedReference} @@ -69,4 +70,12 @@ object Utils { def bytesToString(size: Long): String = { SparkUtils.bytesToString(size) } + + def setInputFileName(inputFileName: String): Unit = { + InputFileBlockHolder.set(inputFileName, 0, -1) + } + + def unsetInputFileName(): Unit = { + InputFileBlockHolder.unset() + } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala new file mode 100644 index 000000000000..ef683366d9d3 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala @@ -0,0 +1,161 @@ +/* + * 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.sql + +import org.apache.paimon.spark.PaimonSparkTestBase +import org.apache.paimon.table.source.DataSplit + +import org.apache.spark.sql.{Row, SparkSession} +import org.junit.jupiter.api.Assertions + +import java.util + +import scala.collection.JavaConverters._ + +class PaimonQueryTest extends PaimonSparkTestBase { + + fileFormats.foreach { + fileFormat => + bucketModes.foreach { + bucketMode => + test(s"Query input_file_name(): file.format=$fileFormat, bucket=$bucketMode") { + val _spark: SparkSession = spark + import _spark.implicits._ + + withTable("T") { + spark.sql(s""" + |CREATE TABLE T (id INT, name STRING) + |TBLPROPERTIES ('file.format'='$fileFormat', 'bucket'='$bucketMode') + |""".stripMargin) + + val location = loadTable("T").location().toUri.toString + + spark.sql("INSERT INTO T VALUES (1, 'x1'), (3, 'x3')") + + val res1 = spark.sql(s""" + |SELECT *, + |startswith(input_file_name(), '$location') AS start, + |endswith(input_file_name(), '.$fileFormat') AS end + |FROM T + |ORdER BY id + |""".stripMargin) + checkAnswer(res1, Row(1, "x1", true, true) :: Row(3, "x3", true, true) :: Nil) + + spark.sql("INSERT INTO T VALUES (2, 'x2'), (4, 'x4'), (6, 'x6')") + + val res2 = + spark.sql("SELECT input_file_name() FROM T").distinct().as[String].collect().sorted + val allDataFiles = getAllFiles("T", Seq.empty, null) + Assertions.assertTrue(res2.sameElements(allDataFiles)) + } + } + + } + } + + fileFormats.foreach { + fileFormat => + bucketModes.foreach { + bucketMode => + test( + s"Query input_file_name() for partitioned table: file.format=$fileFormat, bucket=$bucketMode") { + val _spark: SparkSession = spark + import _spark.implicits._ + + withTable("T") { + spark.sql(s""" + |CREATE TABLE T (id INT, name STRING, pt STRING) + |PARTITIONED BY (pt) + |TBLPROPERTIES ('file.format'='$fileFormat', 'bucket'='$bucketMode') + |""".stripMargin) + + val location = loadTable("T").location().toUri.toString + + spark.sql("INSERT INTO T VALUES (1, 'x1', '2024'), (3, 'x3', '2024')") + + val res1 = spark.sql(s""" + |SELECT id, name, pt, + |startswith(input_file_name(), '$location') AS start, + |endswith(input_file_name(), '.$fileFormat') AS end + |FROM T + |ORdER BY id + |""".stripMargin) + checkAnswer( + res1, + Row(1, "x1", "2024", true, true) :: Row(3, "x3", "2024", true, true) :: Nil) + + spark.sql(""" + |INSERT INTO T + |VALUES (2, 'x2', '2025'), (4, 'x4', '2025'), (6, 'x6', '2026') + |""".stripMargin) + + val res2 = + spark + .sql("SELECT input_file_name() FROM T WHERE pt='2026'") + .distinct() + .as[String] + .collect() + .sorted + val partitionFilter = new util.HashMap[String, String]() + partitionFilter.put("pt", "2026") + val partialDataFiles = getAllFiles("T", Seq("pt"), partitionFilter) + Assertions.assertTrue(res2.sameElements(partialDataFiles)) + + val res3 = + spark.sql("SELECT input_file_name() FROM T").distinct().as[String].collect().sorted + val allDataFiles = getAllFiles("T", Seq("pt"), null) + Assertions.assertTrue(res3.sameElements(allDataFiles)) + } + } + + } + } + + private def getAllFiles( + tableName: String, + partitions: Seq[String], + partitionFilter: java.util.Map[String, String]): Array[String] = { + val paimonTable = loadTable(tableName) + val location = paimonTable.location() + + val files = paimonTable + .newSnapshotReader() + .withPartitionFilter(partitionFilter) + .read() + .splits() + .asScala + .collect { case ds: DataSplit => ds } + .flatMap { + ds => + val prefix = if (partitions.isEmpty) { + s"$location/bucket-${ds.bucket}" + } else { + val partitionPath = partitions.zipWithIndex + .map { + case (pt, index) => + s"$pt=" + ds.partition().getString(index) + } + .mkString("/") + s"$location/$partitionPath/bucket-${ds.bucket}" + } + ds.dataFiles().asScala.map(f => prefix + "/" + f.fileName) + } + files.sorted.toArray + } +} diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala index 5b1c65525404..e390058bafab 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala @@ -23,6 +23,8 @@ trait WithTableOptions { // 3: fixed bucket, -1: dynamic bucket protected val bucketModes: Seq[Int] = Seq(3, -1) + protected val fileFormats: Seq[String] = Seq("orc", "parquet") + protected val withPk: Seq[Boolean] = Seq(true, false) } From 466de5a261792e658bf0520f1344dd4254bd88e3 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Tue, 26 Mar 2024 20:28:47 +0800 Subject: [PATCH 65/79] [core] Unify FileRecordReader and reduce file access (#3098) --- ...ecordReader.java => FileRecordReader.java} | 20 ++++----- .../io/KeyValueDataFileRecordReader.java | 44 +------------------ .../paimon/io/KeyValueFileReaderFactory.java | 16 ++++--- .../operation/AppendOnlyFileStoreRead.java | 12 ++--- .../org/apache/paimon/utils/FileUtils.java | 12 +++-- .../table/AppendOnlyFileStoreTableTest.java | 23 ++++++++++ 6 files changed, 58 insertions(+), 69 deletions(-) rename paimon-core/src/main/java/org/apache/paimon/io/{RowDataFileRecordReader.java => FileRecordReader.java} (86%) diff --git a/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileRecordReader.java b/paimon-core/src/main/java/org/apache/paimon/io/FileRecordReader.java similarity index 86% rename from paimon-core/src/main/java/org/apache/paimon/io/RowDataFileRecordReader.java rename to paimon-core/src/main/java/org/apache/paimon/io/FileRecordReader.java index ed891a32b2e1..0eac2961a912 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileRecordReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/FileRecordReader.java @@ -24,10 +24,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.PartitionInfo; import org.apache.paimon.data.columnar.ColumnarRowIterator; -import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.format.FormatReaderFactory; -import org.apache.paimon.fs.FileIO; -import org.apache.paimon.fs.Path; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.utils.FileUtils; import org.apache.paimon.utils.ProjectedRow; @@ -37,25 +34,26 @@ import java.io.IOException; /** Reads {@link InternalRow} from data files. */ -public class RowDataFileRecordReader implements RecordReader { +public class FileRecordReader implements RecordReader { private final RecordReader reader; @Nullable private final int[] indexMapping; @Nullable private final PartitionInfo partitionInfo; @Nullable private final CastFieldGetter[] castMapping; - public RowDataFileRecordReader( - FileIO fileIO, - Path path, - long fileSize, + public FileRecordReader( FormatReaderFactory readerFactory, + FormatReaderFactory.Context context, @Nullable int[] indexMapping, @Nullable CastFieldGetter[] castMapping, @Nullable PartitionInfo partitionInfo) throws IOException { - FileUtils.checkExists(fileIO, path); - FormatReaderContext context = new FormatReaderContext(fileIO, path, fileSize); - this.reader = readerFactory.createReader(context); + try { + this.reader = readerFactory.createReader(context); + } catch (Exception e) { + FileUtils.checkExists(context.fileIO(), context.filePath()); + throw e; + } this.indexMapping = indexMapping; this.partitionInfo = partitionInfo; this.castMapping = castMapping; diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java index 92be0ff684a5..e44ad79ff53e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java @@ -20,17 +20,9 @@ import org.apache.paimon.KeyValue; import org.apache.paimon.KeyValueSerializer; -import org.apache.paimon.PartitionSettedRow; -import org.apache.paimon.casting.CastFieldGetter; -import org.apache.paimon.casting.CastedRow; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.data.PartitionInfo; -import org.apache.paimon.data.columnar.ColumnarRowIterator; -import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.types.RowType; -import org.apache.paimon.utils.FileUtils; -import org.apache.paimon.utils.ProjectedRow; import javax.annotation.Nullable; @@ -42,27 +34,12 @@ public class KeyValueDataFileRecordReader implements RecordReader { private final RecordReader reader; private final KeyValueSerializer serializer; private final int level; - @Nullable private final int[] indexMapping; - @Nullable private final PartitionInfo partitionInfo; - @Nullable private final CastFieldGetter[] castMapping; public KeyValueDataFileRecordReader( - FormatReaderFactory readerFactory, - FormatReaderFactory.Context context, - RowType keyType, - RowType valueType, - int level, - @Nullable int[] indexMapping, - @Nullable CastFieldGetter[] castMapping, - @Nullable PartitionInfo partitionInfo) - throws IOException { - FileUtils.checkExists(context.fileIO(), context.filePath()); - this.reader = readerFactory.createReader(context); + RecordReader reader, RowType keyType, RowType valueType, int level) { + this.reader = reader; this.serializer = new KeyValueSerializer(keyType, valueType); this.level = level; - this.indexMapping = indexMapping; - this.partitionInfo = partitionInfo; - this.castMapping = castMapping; } @Nullable @@ -73,23 +50,6 @@ public RecordIterator readBatch() throws IOException { return null; } - if (iterator instanceof ColumnarRowIterator) { - iterator = ((ColumnarRowIterator) iterator).mapping(partitionInfo, indexMapping); - } else { - if (partitionInfo != null) { - final PartitionSettedRow partitionSettedRow = - PartitionSettedRow.from(partitionInfo); - iterator = iterator.transform(partitionSettedRow::replaceRow); - } - if (indexMapping != null) { - final ProjectedRow projectedRow = ProjectedRow.from(indexMapping); - iterator = iterator.transform(projectedRow::replaceRow); - } - } - if (castMapping != null) { - final CastedRow castedRow = CastedRow.from(castMapping); - iterator = iterator.transform(castedRow::replaceRow); - } return iterator.transform( internalRow -> internalRow == null diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java index 184857b45691..27ddd1ff677a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.KeyValue; import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.data.InternalRow; import org.apache.paimon.deletionvectors.ApplyDeletionVectorReader; import org.apache.paimon.deletionvectors.DeletionVector; import org.apache.paimon.format.FileFormatDiscover; @@ -125,24 +126,25 @@ private RecordReader createRecordReader( key -> formatSupplier.get()) : formatSupplier.get(); Path filePath = pathFactory.toPath(fileName); - RecordReader recordReader = - new KeyValueDataFileRecordReader( + + RecordReader fileRecordReader = + new FileRecordReader( bulkFormatMapping.getReaderFactory(), orcPoolSize == null ? new FormatReaderContext(fileIO, filePath, fileSize) : new OrcFormatReaderContext( fileIO, filePath, fileSize, orcPoolSize), - keyType, - valueType, - level, bulkFormatMapping.getIndexMapping(), bulkFormatMapping.getCastMapping(), PartitionUtils.create(bulkFormatMapping.getPartitionPair(), partition)); + Optional deletionVector = dvFactory.create(fileName); if (deletionVector.isPresent() && !deletionVector.get().isEmpty()) { - recordReader = new ApplyDeletionVectorReader<>(recordReader, deletionVector.get()); + fileRecordReader = + new ApplyDeletionVectorReader<>(fileRecordReader, deletionVector.get()); } - return recordReader; + + return new KeyValueDataFileRecordReader(fileRecordReader, keyType, valueType, level); } public static Builder builder( diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java index b67edaaf3e6b..49eea905c4b1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java @@ -23,10 +23,11 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.format.FileFormatDiscover; import org.apache.paimon.format.FormatKey; +import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.fs.FileIO; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFilePathFactory; -import org.apache.paimon.io.RowDataFileRecordReader; +import org.apache.paimon.io.FileRecordReader; import org.apache.paimon.mergetree.compact.ConcatRecordReader; import org.apache.paimon.partition.PartitionUtils; import org.apache.paimon.predicate.Predicate; @@ -174,11 +175,12 @@ public RecordReader createReader(DataSplit split) throws IOExceptio final BinaryRow partition = split.partition(); suppliers.add( () -> - new RowDataFileRecordReader( - fileIO, - dataFilePathFactory.toPath(file.fileName()), - file.fileSize(), + new FileRecordReader( bulkFormatMapping.getReaderFactory(), + new FormatReaderContext( + fileIO, + dataFilePathFactory.toPath(file.fileName()), + file.fileSize()), bulkFormatMapping.getIndexMapping(), bulkFormatMapping.getCastMapping(), PartitionUtils.create( diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/FileUtils.java b/paimon-core/src/main/java/org/apache/paimon/utils/FileUtils.java index f1278ab717c8..eddc7273e76b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/FileUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/FileUtils.java @@ -134,10 +134,14 @@ public static void checkExists(FileIO fileIO, Path file) throws IOException { public static RecordReader createFormatReader( FileIO fileIO, FormatReaderFactory format, Path file, @Nullable Long fileSize) throws IOException { - checkExists(fileIO, file); - if (fileSize == null) { - fileSize = fileIO.getFileSize(file); + try { + if (fileSize == null) { + fileSize = fileIO.getFileSize(file); + } + return format.createReader(new FormatReaderContext(fileIO, file, fileSize)); + } catch (Exception e) { + checkExists(fileIO, file); + throw e; } - return format.createReader(new FormatReaderContext(fileIO, file, fileSize)); } } 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 9e0c6d2ab4cb..3846ab234bba 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 @@ -23,6 +23,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.serializer.InternalRowSerializer; import org.apache.paimon.fs.FileIOFinder; +import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.options.Options; import org.apache.paimon.predicate.Predicate; @@ -57,10 +58,32 @@ import static org.apache.paimon.table.sink.KeyAndBucketExtractor.bucket; import static org.apache.paimon.table.sink.KeyAndBucketExtractor.bucketKeyHashCode; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link AppendOnlyFileStoreTable}. */ public class AppendOnlyFileStoreTableTest extends FileStoreTableTestBase { + @Test + public void testReadDeletedFiles() throws Exception { + writeData(); + FileStoreTable table = createFileStoreTable(); + List splits = toSplits(table.newSnapshotReader().read().dataSplits()); + TableRead read = table.newRead(); + + // delete one file + DataSplit split = (DataSplit) splits.get(0); + Path path = + table.store() + .pathFactory() + .createDataFilePathFactory(split.partition(), split.bucket()) + .toPath(split.dataFiles().get(0).fileName()); + table.fileIO().deleteQuietly(path); + + // read + assertThatThrownBy(() -> getResult(read, splits, BATCH_ROW_TO_STRING)) + .hasMessageContaining("snapshot expires too fast"); + } + @Test public void testBatchReadWrite() throws Exception { writeData(); From 85843e9bb5d270c9ef465cad46d08659dcc0d038 Mon Sep 17 00:00:00 2001 From: Xiaojian Sun Date: Tue, 26 Mar 2024 20:31:09 +0800 Subject: [PATCH 66/79] [core] Support common jdbc catalog lock for filesystem catalog. (#3076) --- docs/content/how-to/creating-catalogs.md | 10 ++ .../paimon/catalog/FileSystemCatalog.java | 16 ++- .../paimon/catalog/LockContextUtils.java | 98 +++++++++++++++++++ .../apache/paimon/jdbc/JdbcCatalogLock.java | 3 +- .../paimon/catalog/FileSystemCatalogTest.java | 68 +++++++++++++ 5 files changed, 193 insertions(+), 2 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/catalog/LockContextUtils.java create mode 100644 paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java diff --git a/docs/content/how-to/creating-catalogs.md b/docs/content/how-to/creating-catalogs.md index 536a6b165ff7..fc2927d4153a 100644 --- a/docs/content/how-to/creating-catalogs.md +++ b/docs/content/how-to/creating-catalogs.md @@ -53,6 +53,16 @@ USE CATALOG my_catalog; You can define any default table options with the prefix `table-default.` for tables created in the catalog. +The FileSystem catalog supports jdbc lock and can take effect through the following configuration: + +> ```shell +> 'uri' = 'jdbc:mysql://:/' +> 'jdbc.user' = '...', +> 'jdbc.password' = '...', +> ``` + + + {{< /tab >}} {{< tab "Spark3" >}} diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java index e71c92dc4007..2c1d1e8df4dc 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java @@ -18,6 +18,7 @@ package org.apache.paimon.catalog; +import org.apache.paimon.client.ClientPool; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; @@ -35,6 +36,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.Callable; import static org.apache.paimon.catalog.FileSystemCatalogOptions.CASE_SENSITIVE; @@ -46,6 +48,8 @@ public class FileSystemCatalog extends AbstractCatalog { private final Path warehouse; + private ClientPool.ClientPoolImpl clientPool; + public FileSystemCatalog(FileIO fileIO, Path warehouse) { super(fileIO); this.warehouse = warehouse; @@ -156,6 +160,14 @@ private SchemaManager schemaManager(Identifier identifier) { .withLock(catalogLock == null ? null : Lock.fromCatalog(catalogLock, identifier)); } + @Override + public Optional lockContext() { + if (clientPool == null) { + this.clientPool = LockContextUtils.tryInitializeClientPool(catalogOptions); + } + return LockContextUtils.lockContext(this.clientPool, catalogOptions, "filesystem"); + } + @Override public void renameTableImpl(Identifier fromTable, Identifier toTable) { Path fromPath = getDataTableLocation(fromTable); @@ -187,7 +199,9 @@ private static String database(Path path) { } @Override - public void close() throws Exception {} + public void close() throws Exception { + LockContextUtils.close(clientPool); + } @Override public String warehouse() { diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/LockContextUtils.java b/paimon-core/src/main/java/org/apache/paimon/catalog/LockContextUtils.java new file mode 100644 index 000000000000..699c54de4474 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/LockContextUtils.java @@ -0,0 +1,98 @@ +/* + * 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.catalog; + +import org.apache.paimon.client.ClientPool; +import org.apache.paimon.jdbc.JdbcCatalogFactory; +import org.apache.paimon.jdbc.JdbcCatalogLock; +import org.apache.paimon.jdbc.JdbcClientPool; +import org.apache.paimon.jdbc.JdbcUtils; +import org.apache.paimon.options.CatalogOptions; +import org.apache.paimon.options.Options; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.SQLException; +import java.util.Optional; + +/** Utils for {@link org.apache.paimon.catalog.CatalogLock.LockContext}. */ +public class LockContextUtils { + + private static final Logger LOG = LoggerFactory.getLogger(FileSystemCatalog.class); + + public static Optional lockContext( + ClientPool.ClientPoolImpl clientPool, Options catalogOptions, String catalogKey) { + if (clientPool == null) { + return Optional.of(new AbstractCatalog.OptionLockContext(catalogOptions)); + } + String lockType = catalogOptions.get(CatalogOptions.LOCK_TYPE); + switch (lockType) { + case JdbcCatalogFactory.IDENTIFIER: + JdbcClientPool connections = (JdbcClientPool) clientPool; + return Optional.of( + new JdbcCatalogLock.JdbcLockContext( + connections, catalogKey, catalogOptions)); + default: + LOG.warn("Unsupported lock type:" + lockType); + return Optional.of(new AbstractCatalog.OptionLockContext(catalogOptions)); + } + } + + public static ClientPool.ClientPoolImpl tryInitializeClientPool(Options catalogOptions) { + String lockType = catalogOptions.get(CatalogOptions.LOCK_TYPE); + if (lockType == null) { + return null; + } + switch (lockType) { + case JdbcCatalogFactory.IDENTIFIER: + JdbcClientPool connections = + new JdbcClientPool( + catalogOptions.get(CatalogOptions.CLIENT_POOL_SIZE), + catalogOptions.get(CatalogOptions.URI.key()), + catalogOptions.toMap()); + try { + JdbcUtils.createDistributedLockTable(connections, catalogOptions); + } catch (SQLException e) { + throw new RuntimeException("Cannot initialize JDBC distributed lock.", e); + } catch (InterruptedException e) { + throw new RuntimeException("Interrupted in call to initialize", e); + } + return connections; + default: + LOG.warn("Unsupported lock type:" + lockType); + return null; + } + } + + public static void close(ClientPool.ClientPoolImpl clientPool) { + if (clientPool == null) { + return; + } + if (clientPool instanceof JdbcClientPool) { + JdbcClientPool connections = (JdbcClientPool) clientPool; + if (!connections.isClosed()) { + connections.close(); + } + } else { + clientPool.close(); + } + clientPool = null; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java index d713feb7e49f..2d409f12a693 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java @@ -109,7 +109,8 @@ public CatalogLock create(LockContext context) { } } - static class JdbcLockContext implements LockContext { + /** Jdbc lock context. */ + public static class JdbcLockContext implements LockContext { private final JdbcClientPool connections; private final String catalogKey; private final Options conf; diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java new file mode 100644 index 000000000000..0948ab07c4c2 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java @@ -0,0 +1,68 @@ +/* + * 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.catalog; + +import org.apache.paimon.fs.Path; +import org.apache.paimon.jdbc.JdbcCatalog; +import org.apache.paimon.options.CatalogOptions; +import org.apache.paimon.options.Options; + +import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; + +import org.junit.jupiter.api.BeforeEach; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link FileSystemCatalog}. */ +public class FileSystemCatalogTest extends CatalogTestBase { + + @BeforeEach + public void setUp() throws Exception { + super.setUp(); + catalog = initCatalog(Maps.newHashMap()); + } + + private FileSystemCatalog initCatalog(Map props) { + Map properties = Maps.newHashMap(); + properties.put( + CatalogOptions.URI.key(), + "jdbc:sqlite:file::memory:?ic" + UUID.randomUUID().toString().replace("-", "")); + + properties.put(JdbcCatalog.PROPERTY_PREFIX + "username", "user"); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "password", "password"); + properties.put(CatalogOptions.WAREHOUSE.key(), warehouse); + properties.put(CatalogOptions.LOCK_ENABLED.key(), "true"); + properties.put(CatalogOptions.LOCK_TYPE.key(), "jdbc"); + properties.putAll(props); + FileSystemCatalog catalog = + new FileSystemCatalog(fileIO, new Path(warehouse), Options.fromMap(properties)); + return catalog; + } + + @Override + public void testListDatabasesWhenNoDatabases() { + List databases = catalog.listDatabases(); + assertThat(databases).isEqualTo(new ArrayList<>()); + } +} From 983c78e3dc4694407ffbe55bc63b9aef125caecb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=90=B4=E7=A5=A5=E5=B9=B3?= <408317717@qq.com> Date: Wed, 27 Mar 2024 12:11:55 +0800 Subject: [PATCH 67/79] [core] Fix OrphanFilesClean has a chance to delete normal file (#3075) --- .../main/java/org/apache/paimon/operation/OrphanFilesClean.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java b/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java index 8d85024cf00d..13ae240b6563 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java @@ -125,8 +125,8 @@ public int clean() throws IOException, ExecutionException, InterruptedException nonSnapshotFiles.forEach(this::deleteFileOrDirQuietly); deletedFilesNum += nonSnapshotFiles.size(); - Set usedFiles = getUsedFiles(); Map candidates = getCandidateDeletingFiles(); + Set usedFiles = getUsedFiles(); Set deleted = new HashSet<>(candidates.keySet()); deleted.removeAll(usedFiles); From cfeb22ff3b985a78a99fa71ceecc3fe808ef9440 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Wed, 27 Mar 2024 13:07:31 +0800 Subject: [PATCH 68/79] [catalog] Refactor Catalog Factory and revert jdbc lock in FileSystemCatalog (#3099) --- docs/content/how-to/creating-catalogs.md | 10 -- .../paimon/catalog/AbstractCatalog.java | 39 ++++---- .../org/apache/paimon/catalog/Catalog.java | 4 +- .../apache/paimon/catalog/CatalogLock.java | 10 -- .../paimon/catalog/CatalogLockContext.java | 33 +++++++ .../paimon/catalog/CatalogLockFactory.java | 29 ++++++ .../paimon/catalog/FileSystemCatalog.java | 18 +--- .../paimon/catalog/LockContextUtils.java | 98 ------------------- .../org/apache/paimon/jdbc/JdbcCatalog.java | 12 ++- .../paimon/jdbc/JdbcCatalogFactory.java | 8 -- .../apache/paimon/jdbc/JdbcCatalogLock.java | 36 ------- .../paimon/jdbc/JdbcCatalogLockContext.java | 49 ++++++++++ .../paimon/jdbc/JdbcCatalogLockFactory.java | 52 ++++++++++ .../org/apache/paimon/operation/Lock.java | 22 +++-- .../org.apache.paimon.factories.Factory | 2 +- .../paimon/catalog/FileSystemCatalogTest.java | 68 ------------- .../paimon/flink/FileSystemCatalogITCase.java | 19 +--- .../org/apache/paimon/hive/HiveCatalog.java | 32 +++--- .../apache/paimon/hive/HiveCatalogLock.java | 33 ------- .../paimon/hive/HiveCatalogLockContext.java | 50 ++++++++++ .../paimon/hive/HiveCatalogLockFactory.java | 52 ++++++++++ .../org.apache.paimon.factories.Factory | 4 +- .../paimon/hive/HiveCatalogITCaseBase.java | 6 +- 23 files changed, 331 insertions(+), 355 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLockContext.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLockFactory.java delete mode 100644 paimon-core/src/main/java/org/apache/paimon/catalog/LockContextUtils.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLockContext.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLockFactory.java delete mode 100644 paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java create mode 100644 paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLockContext.java create mode 100644 paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLockFactory.java diff --git a/docs/content/how-to/creating-catalogs.md b/docs/content/how-to/creating-catalogs.md index fc2927d4153a..536a6b165ff7 100644 --- a/docs/content/how-to/creating-catalogs.md +++ b/docs/content/how-to/creating-catalogs.md @@ -53,16 +53,6 @@ USE CATALOG my_catalog; You can define any default table options with the prefix `table-default.` for tables created in the catalog. -The FileSystem catalog supports jdbc lock and can take effect through the following configuration: - -> ```shell -> 'uri' = 'jdbc:mysql://:/' -> 'jdbc.user' = '...', -> 'jdbc.password' = '...', -> ``` - - - {{< /tab >}} {{< tab "Spark3" >}} diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index c69a72b0db6a..4a4fb04fd4e2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -83,26 +83,31 @@ protected AbstractCatalog(FileIO fileIO, Options options) { this.tableDefaultOptions = convertToPropertiesPrefixKey(options.toMap(), TABLE_DEFAULT_OPTION_PREFIX); this.catalogOptions = options; + } - if (lockEnabled()) { - checkArgument(options.contains(LOCK_TYPE), "No lock type when lock is enabled."); + @Override + public Optional lockFactory() { + if (!lockEnabled()) { + return Optional.empty(); } + + String lock = catalogOptions.get(LOCK_TYPE); + if (lock == null) { + return defaultLockFactory(); + } + + return Optional.of( + FactoryUtil.discoverFactory( + AbstractCatalog.class.getClassLoader(), CatalogLockFactory.class, lock)); } - @Override - public Optional lockFactory() { - return lockEnabled() - ? Optional.of( - FactoryUtil.discoverFactory( - AbstractCatalog.class.getClassLoader(), - CatalogLock.LockFactory.class, - catalogOptions.get(LOCK_TYPE))) - : Optional.empty(); + public Optional defaultLockFactory() { + return Optional.empty(); } @Override - public Optional lockContext() { - return Optional.of(new OptionLockContext(catalogOptions)); + public Optional lockContext() { + return Optional.of(CatalogLockContext.fromOptions(catalogOptions)); } protected boolean lockEnabled() { @@ -492,12 +497,4 @@ private void validateAutoCreateClose(Map options) { "The value of %s property should be %s.", CoreOptions.AUTO_CREATE.key(), Boolean.FALSE)); } - - static class OptionLockContext implements CatalogLock.LockContext { - private final Options catalogOptions; - - public OptionLockContext(Options catalogOptions) { - this.catalogOptions = catalogOptions; - } - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java index 0168891b008a..99b71e8deb1e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java @@ -49,10 +49,10 @@ public interface Catalog extends AutoCloseable { * Get lock factory from catalog. Lock is used to support multiple concurrent writes on the * object store. */ - Optional lockFactory(); + Optional lockFactory(); /** Get lock context for lock factory to create a lock. */ - default Optional lockContext() { + default Optional lockContext() { return Optional.empty(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLock.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLock.java index 0e547037e3a2..4f147c6aada7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLock.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLock.java @@ -19,10 +19,8 @@ package org.apache.paimon.catalog; import org.apache.paimon.annotation.Public; -import org.apache.paimon.factories.Factory; import java.io.Closeable; -import java.io.Serializable; import java.util.concurrent.Callable; /** @@ -35,12 +33,4 @@ public interface CatalogLock extends Closeable { /** Run with catalog lock. The caller should tell catalog the database and table name. */ T runWithLock(String database, String table, Callable callable) throws Exception; - - /** Factory to create {@link CatalogLock}. */ - interface LockFactory extends Factory, Serializable { - CatalogLock create(LockContext context); - } - - /** Context for lock factory to create lock. */ - interface LockContext extends Serializable {} } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLockContext.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLockContext.java new file mode 100644 index 000000000000..442409c81be8 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLockContext.java @@ -0,0 +1,33 @@ +/* + * 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.catalog; + +import org.apache.paimon.options.Options; + +import java.io.Serializable; + +/** Context for lock factory to create lock. */ +public interface CatalogLockContext extends Serializable { + + Options options(); + + static CatalogLockContext fromOptions(Options options) { + return () -> options; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLockFactory.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLockFactory.java new file mode 100644 index 000000000000..d964ebda4ee2 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogLockFactory.java @@ -0,0 +1,29 @@ +/* + * 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.catalog; + +import org.apache.paimon.factories.Factory; + +import java.io.Serializable; + +/** Factory to create {@link CatalogLock}. */ +public interface CatalogLockFactory extends Factory, Serializable { + + CatalogLock createLock(CatalogLockContext context); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java index 2c1d1e8df4dc..8ffe0f271916 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java @@ -18,7 +18,6 @@ package org.apache.paimon.catalog; -import org.apache.paimon.client.ClientPool; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; @@ -36,7 +35,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.Callable; import static org.apache.paimon.catalog.FileSystemCatalogOptions.CASE_SENSITIVE; @@ -48,8 +46,6 @@ public class FileSystemCatalog extends AbstractCatalog { private final Path warehouse; - private ClientPool.ClientPoolImpl clientPool; - public FileSystemCatalog(FileIO fileIO, Path warehouse) { super(fileIO); this.warehouse = warehouse; @@ -149,7 +145,7 @@ private SchemaManager schemaManager(Identifier identifier) { lockFactory() .map( fac -> - fac.create( + fac.createLock( lockContext() .orElseThrow( () -> @@ -160,14 +156,6 @@ private SchemaManager schemaManager(Identifier identifier) { .withLock(catalogLock == null ? null : Lock.fromCatalog(catalogLock, identifier)); } - @Override - public Optional lockContext() { - if (clientPool == null) { - this.clientPool = LockContextUtils.tryInitializeClientPool(catalogOptions); - } - return LockContextUtils.lockContext(this.clientPool, catalogOptions, "filesystem"); - } - @Override public void renameTableImpl(Identifier fromTable, Identifier toTable) { Path fromPath = getDataTableLocation(fromTable); @@ -199,9 +187,7 @@ private static String database(Path path) { } @Override - public void close() throws Exception { - LockContextUtils.close(clientPool); - } + public void close() throws Exception {} @Override public String warehouse() { diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/LockContextUtils.java b/paimon-core/src/main/java/org/apache/paimon/catalog/LockContextUtils.java deleted file mode 100644 index 699c54de4474..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/LockContextUtils.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.catalog; - -import org.apache.paimon.client.ClientPool; -import org.apache.paimon.jdbc.JdbcCatalogFactory; -import org.apache.paimon.jdbc.JdbcCatalogLock; -import org.apache.paimon.jdbc.JdbcClientPool; -import org.apache.paimon.jdbc.JdbcUtils; -import org.apache.paimon.options.CatalogOptions; -import org.apache.paimon.options.Options; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.SQLException; -import java.util.Optional; - -/** Utils for {@link org.apache.paimon.catalog.CatalogLock.LockContext}. */ -public class LockContextUtils { - - private static final Logger LOG = LoggerFactory.getLogger(FileSystemCatalog.class); - - public static Optional lockContext( - ClientPool.ClientPoolImpl clientPool, Options catalogOptions, String catalogKey) { - if (clientPool == null) { - return Optional.of(new AbstractCatalog.OptionLockContext(catalogOptions)); - } - String lockType = catalogOptions.get(CatalogOptions.LOCK_TYPE); - switch (lockType) { - case JdbcCatalogFactory.IDENTIFIER: - JdbcClientPool connections = (JdbcClientPool) clientPool; - return Optional.of( - new JdbcCatalogLock.JdbcLockContext( - connections, catalogKey, catalogOptions)); - default: - LOG.warn("Unsupported lock type:" + lockType); - return Optional.of(new AbstractCatalog.OptionLockContext(catalogOptions)); - } - } - - public static ClientPool.ClientPoolImpl tryInitializeClientPool(Options catalogOptions) { - String lockType = catalogOptions.get(CatalogOptions.LOCK_TYPE); - if (lockType == null) { - return null; - } - switch (lockType) { - case JdbcCatalogFactory.IDENTIFIER: - JdbcClientPool connections = - new JdbcClientPool( - catalogOptions.get(CatalogOptions.CLIENT_POOL_SIZE), - catalogOptions.get(CatalogOptions.URI.key()), - catalogOptions.toMap()); - try { - JdbcUtils.createDistributedLockTable(connections, catalogOptions); - } catch (SQLException e) { - throw new RuntimeException("Cannot initialize JDBC distributed lock.", e); - } catch (InterruptedException e) { - throw new RuntimeException("Interrupted in call to initialize", e); - } - return connections; - default: - LOG.warn("Unsupported lock type:" + lockType); - return null; - } - } - - public static void close(ClientPool.ClientPoolImpl clientPool) { - if (clientPool == null) { - return; - } - if (clientPool instanceof JdbcClientPool) { - JdbcClientPool connections = (JdbcClientPool) clientPool; - if (!connections.isClosed()) { - connections.close(); - } - } else { - clientPool.close(); - } - clientPool = null; - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java index 91c11ac24828..7e7718b5bee9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java @@ -20,7 +20,8 @@ import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.catalog.AbstractCatalog; -import org.apache.paimon.catalog.CatalogLock; +import org.apache.paimon.catalog.CatalogLockContext; +import org.apache.paimon.catalog.CatalogLockFactory; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; @@ -343,8 +344,13 @@ public boolean caseSensitive() { } @Override - public Optional lockContext() { - return Optional.of(new JdbcCatalogLock.JdbcLockContext(connections, catalogKey, options)); + public Optional defaultLockFactory() { + return Optional.of(new JdbcCatalogLockFactory()); + } + + @Override + public Optional lockContext() { + return Optional.of(new JdbcCatalogLockContext(connections, catalogKey, options)); } private Lock lock(Identifier identifier) { diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogFactory.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogFactory.java index adaaf3f43632..6c3c1d0e41cc 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogFactory.java @@ -25,9 +25,6 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; -import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED; -import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE; - /** Factory to create {@link JdbcCatalog}. */ public class JdbcCatalogFactory implements CatalogFactory { @@ -42,11 +39,6 @@ public String identifier() { public Catalog create(FileIO fileIO, Path warehouse, CatalogContext context) { Options options = context.options(); String catalogKey = options.get(JdbcCatalogOptions.CATALOG_KEY); - if (options.get(LOCK_ENABLED)) { - if (!options.getOptional(LOCK_TYPE).isPresent()) { - options.set(LOCK_TYPE, JdbcCatalogLock.JdbcCatalogLockFactory.IDENTIFIER); - } - } return new JdbcCatalog(fileIO, catalogKey, context.options(), warehouse.toString()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java index 2d409f12a693..307f92f0a570 100644 --- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLock.java @@ -19,7 +19,6 @@ package org.apache.paimon.jdbc; import org.apache.paimon.catalog.CatalogLock; -import org.apache.paimon.options.Options; import org.apache.paimon.utils.TimeUtils; import java.io.IOException; @@ -87,41 +86,6 @@ public void close() throws IOException { // Do nothing } - /** Jdbc catalog lock factory. */ - public static class JdbcCatalogLockFactory implements LockFactory { - - private static final long serialVersionUID = 1L; - public static final String IDENTIFIER = "jdbc"; - - @Override - public String identifier() { - return IDENTIFIER; - } - - @Override - public CatalogLock create(LockContext context) { - JdbcLockContext lockContext = (JdbcLockContext) context; - return new JdbcCatalogLock( - lockContext.connections, - lockContext.catalogKey, - checkMaxSleep(lockContext.conf.toMap()), - acquireTimeout(lockContext.conf.toMap())); - } - } - - /** Jdbc lock context. */ - public static class JdbcLockContext implements LockContext { - private final JdbcClientPool connections; - private final String catalogKey; - private final Options conf; - - public JdbcLockContext(JdbcClientPool connections, String catalogKey, Options conf) { - this.connections = connections; - this.catalogKey = catalogKey; - this.conf = conf; - } - } - public static long checkMaxSleep(Map conf) { return TimeUtils.parseDuration( conf.getOrDefault( diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLockContext.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLockContext.java new file mode 100644 index 000000000000..e56b3474ccb3 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLockContext.java @@ -0,0 +1,49 @@ +/* + * 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.jdbc; + +import org.apache.paimon.catalog.CatalogLockContext; +import org.apache.paimon.options.Options; + +/** Jdbc lock context. */ +public class JdbcCatalogLockContext implements CatalogLockContext { + + private final JdbcClientPool connections; + private final String catalogKey; + private final Options options; + + public JdbcCatalogLockContext(JdbcClientPool connections, String catalogKey, Options options) { + this.connections = connections; + this.catalogKey = catalogKey; + this.options = options; + } + + @Override + public Options options() { + return options; + } + + public JdbcClientPool connections() { + return connections; + } + + public String catalogKey() { + return catalogKey; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLockFactory.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLockFactory.java new file mode 100644 index 000000000000..ce0a2d24eea5 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalogLockFactory.java @@ -0,0 +1,52 @@ +/* + * 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.jdbc; + +import org.apache.paimon.catalog.CatalogLock; +import org.apache.paimon.catalog.CatalogLockContext; +import org.apache.paimon.catalog.CatalogLockFactory; + +import java.util.Map; + +import static org.apache.paimon.jdbc.JdbcCatalogLock.acquireTimeout; +import static org.apache.paimon.jdbc.JdbcCatalogLock.checkMaxSleep; + +/** Jdbc catalog lock factory. */ +public class JdbcCatalogLockFactory implements CatalogLockFactory { + + private static final long serialVersionUID = 1L; + + public static final String IDENTIFIER = "jdbc"; + + @Override + public String identifier() { + return IDENTIFIER; + } + + @Override + public CatalogLock createLock(CatalogLockContext context) { + JdbcCatalogLockContext lockContext = (JdbcCatalogLockContext) context; + Map optionsMap = lockContext.options().toMap(); + return new JdbcCatalogLock( + lockContext.connections(), + lockContext.catalogKey(), + checkMaxSleep(optionsMap), + acquireTimeout(optionsMap)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/Lock.java b/paimon-core/src/main/java/org/apache/paimon/operation/Lock.java index a9f27e70ae6a..76cd8b178d2c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/Lock.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/Lock.java @@ -20,6 +20,8 @@ import org.apache.paimon.annotation.Public; import org.apache.paimon.catalog.CatalogLock; +import org.apache.paimon.catalog.CatalogLockContext; +import org.apache.paimon.catalog.CatalogLockFactory; import org.apache.paimon.catalog.Identifier; import javax.annotation.Nullable; @@ -44,12 +46,12 @@ interface Factory extends Serializable { } static Factory factory( - @Nullable CatalogLock.LockFactory lockFactory, - @Nullable CatalogLock.LockContext lockContext, + @Nullable CatalogLockFactory lockFactory, + @Nullable CatalogLockContext lockContext, Identifier tablePath) { return lockFactory == null ? new EmptyFactory() - : new CatalogLockFactory(lockFactory, lockContext, tablePath); + : new LockFactory(lockFactory, lockContext, tablePath); } static Factory emptyFactory() { @@ -57,17 +59,17 @@ static Factory emptyFactory() { } /** A {@link Factory} creating lock from catalog. */ - class CatalogLockFactory implements Factory { + class LockFactory implements Factory { private static final long serialVersionUID = 1L; - private final CatalogLock.LockFactory lockFactory; - private final CatalogLock.LockContext lockContext; + private final CatalogLockFactory lockFactory; + private final CatalogLockContext lockContext; private final Identifier tablePath; - public CatalogLockFactory( - CatalogLock.LockFactory lockFactory, - CatalogLock.LockContext lockContext, + public LockFactory( + CatalogLockFactory lockFactory, + CatalogLockContext lockContext, Identifier tablePath) { this.lockFactory = lockFactory; this.lockContext = lockContext; @@ -76,7 +78,7 @@ public CatalogLockFactory( @Override public Lock create() { - return fromCatalog(lockFactory.create(lockContext), tablePath); + return fromCatalog(lockFactory.createLock(lockContext), tablePath); } } diff --git a/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index 34de4106bada..0f87c96b0d4e 100644 --- a/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -15,4 +15,4 @@ org.apache.paimon.catalog.FileSystemCatalogFactory org.apache.paimon.jdbc.JdbcCatalogFactory -org.apache.paimon.jdbc.JdbcCatalogLock$JdbcCatalogLockFactory +org.apache.paimon.jdbc.JdbcCatalogLockFactory diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java deleted file mode 100644 index 0948ab07c4c2..000000000000 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.catalog; - -import org.apache.paimon.fs.Path; -import org.apache.paimon.jdbc.JdbcCatalog; -import org.apache.paimon.options.CatalogOptions; -import org.apache.paimon.options.Options; - -import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; - -import org.junit.jupiter.api.BeforeEach; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.UUID; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for {@link FileSystemCatalog}. */ -public class FileSystemCatalogTest extends CatalogTestBase { - - @BeforeEach - public void setUp() throws Exception { - super.setUp(); - catalog = initCatalog(Maps.newHashMap()); - } - - private FileSystemCatalog initCatalog(Map props) { - Map properties = Maps.newHashMap(); - properties.put( - CatalogOptions.URI.key(), - "jdbc:sqlite:file::memory:?ic" + UUID.randomUUID().toString().replace("-", "")); - - properties.put(JdbcCatalog.PROPERTY_PREFIX + "username", "user"); - properties.put(JdbcCatalog.PROPERTY_PREFIX + "password", "password"); - properties.put(CatalogOptions.WAREHOUSE.key(), warehouse); - properties.put(CatalogOptions.LOCK_ENABLED.key(), "true"); - properties.put(CatalogOptions.LOCK_TYPE.key(), "jdbc"); - properties.putAll(props); - FileSystemCatalog catalog = - new FileSystemCatalog(fileIO, new Path(warehouse), Options.fromMap(properties)); - return catalog; - } - - @Override - public void testListDatabasesWhenNoDatabases() { - List databases = catalog.listDatabases(); - assertThat(databases).isEqualTo(new ArrayList<>()); - } -} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java index b68d65dd2251..50d28cd112fc 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java @@ -21,6 +21,8 @@ import org.apache.paimon.catalog.AbstractCatalog; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogLock; +import org.apache.paimon.catalog.CatalogLockContext; +import org.apache.paimon.catalog.CatalogLockFactory; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.util.AbstractTestBase; import org.apache.paimon.fs.Path; @@ -154,18 +156,6 @@ public void testCatalogOptionsInheritAndOverride() throws Exception { @Test void testCatalogWithLockForSchema() throws Exception { LOCK_COUNT.set(0); - assertThatThrownBy( - () -> - tEnv.executeSql( - String.format( - "CREATE CATALOG fs_with_lock WITH (" - + "'type'='paimon', " - + "'warehouse'='%s', " - + "'lock.enabled'='true'" - + ")", - path)) - .await()) - .hasRootCauseMessage("No lock type when lock is enabled."); tEnv.executeSql( String.format( "CREATE CATALOG fs_with_lock WITH (" @@ -203,7 +193,8 @@ private List collect(String sql) throws Exception { } /** Lock factory for file system catalog. */ - public static class FileSystemCatalogDummyLockFactory implements CatalogLock.LockFactory { + public static class FileSystemCatalogDummyLockFactory implements CatalogLockFactory { + private static final String IDENTIFIER = "DUMMY"; @Override @@ -212,7 +203,7 @@ public String identifier() { } @Override - public CatalogLock create(CatalogLock.LockContext context) { + public CatalogLock createLock(CatalogLockContext context) { return new CatalogLock() { @Override public T runWithLock(String database, String table, Callable callable) diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index 589e920370e0..372bfedefb88 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -23,7 +23,8 @@ import org.apache.paimon.catalog.AbstractCatalog; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; -import org.apache.paimon.catalog.CatalogLock; +import org.apache.paimon.catalog.CatalogLockContext; +import org.apache.paimon.catalog.CatalogLockFactory; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; @@ -77,15 +78,12 @@ import java.util.stream.Collectors; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREWAREHOUSE; -import static org.apache.paimon.hive.HiveCatalogLock.LOCK_IDENTIFIER; import static org.apache.paimon.hive.HiveCatalogLock.acquireTimeout; import static org.apache.paimon.hive.HiveCatalogLock.checkMaxSleep; import static org.apache.paimon.hive.HiveCatalogOptions.HADOOP_CONF_DIR; import static org.apache.paimon.hive.HiveCatalogOptions.HIVE_CONF_DIR; import static org.apache.paimon.hive.HiveCatalogOptions.IDENTIFIER; import static org.apache.paimon.hive.HiveCatalogOptions.LOCATION_IN_PROPERTIES; -import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED; -import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE; import static org.apache.paimon.options.CatalogOptions.TABLE_TYPE; import static org.apache.paimon.options.OptionsUtils.convertToPropertiesPrefixKey; import static org.apache.paimon.utils.Preconditions.checkArgument; @@ -93,6 +91,7 @@ /** A catalog implementation for Hive. */ public class HiveCatalog extends AbstractCatalog { + private static final Logger LOG = LoggerFactory.getLogger(HiveCatalog.class); // Reserved properties @@ -149,10 +148,15 @@ public HiveCatalog( } @Override - public Optional lockContext() { + public Optional defaultLockFactory() { + return Optional.of(new HiveCatalogLockFactory()); + } + + @Override + public Optional lockContext() { return Optional.of( - new HiveCatalogLock.HiveLockContext( - new SerializableHiveConf(hiveConf), clientClassName)); + new HiveCatalogLockContext( + new SerializableHiveConf(hiveConf), clientClassName, catalogOptions)); } @Override @@ -635,7 +639,7 @@ public static HiveConf createHiveConf( try (InputStream inputStream = hiveSite.getFileSystem(hadoopConf).open(hiveSite)) { hiveConf.addResource(inputStream, hiveSite.toString()); // trigger a read from the conf to avoid input stream is closed - isEmbeddedMetastore(hiveConf); + hiveConf.getVar(HiveConf.ConfVars.METASTOREURIS); } catch (IOException e) { throw new RuntimeException( "Failed to load hive-site.xml from specified path:" + hiveSite, e); @@ -656,10 +660,6 @@ public static HiveConf createHiveConf( } } - public static boolean isEmbeddedMetastore(HiveConf hiveConf) { - return isNullOrWhitespaceOnly(hiveConf.getVar(HiveConf.ConfVars.METASTOREURIS)); - } - public static Catalog createHiveCatalog(CatalogContext context) { HiveConf hiveConf = createHiveConf(context); Options options = context.options(); @@ -680,14 +680,6 @@ public static Catalog createHiveCatalog(CatalogContext context) { } catch (IOException e) { throw new UncheckedIOException(e); } - - /** Hive catalog only support hive lock. */ - if (options.getOptional(LOCK_ENABLED).orElse(false)) { - Optional lockType = options.getOptional(LOCK_TYPE); - if (!lockType.isPresent()) { - options.set(LOCK_TYPE, LOCK_IDENTIFIER); - } - } return new HiveCatalog( fileIO, hiveConf, diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLock.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLock.java index c49cd020c654..8c3d3829ea28 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLock.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLock.java @@ -39,7 +39,6 @@ import static org.apache.paimon.options.CatalogOptions.LOCK_ACQUIRE_TIMEOUT; import static org.apache.paimon.options.CatalogOptions.LOCK_CHECK_MAX_SLEEP; -import static org.apache.paimon.utils.Preconditions.checkArgument; /** Hive {@link CatalogLock}. */ public class HiveCatalogLock implements CatalogLock { @@ -114,28 +113,6 @@ public void close() { this.client.close(); } - /** Catalog lock factory for hive. */ - public static class HiveCatalogLockFactory implements LockFactory { - - private static final long serialVersionUID = 1L; - - @Override - public CatalogLock create(LockContext context) { - checkArgument(context instanceof HiveLockContext); - HiveLockContext hiveLockContext = (HiveLockContext) context; - HiveConf conf = hiveLockContext.hiveConf.conf(); - return new HiveCatalogLock( - HiveCatalog.createClient(conf, hiveLockContext.clientClassName), - checkMaxSleep(conf), - acquireTimeout(conf)); - } - - @Override - public String identifier() { - return LOCK_IDENTIFIER; - } - } - public static long checkMaxSleep(HiveConf conf) { return TimeUtils.parseDuration( conf.get( @@ -151,14 +128,4 @@ public static long acquireTimeout(HiveConf conf) { TimeUtils.getStringInMillis(LOCK_ACQUIRE_TIMEOUT.defaultValue()))) .toMillis(); } - - static class HiveLockContext implements LockContext { - private final SerializableHiveConf hiveConf; - private final String clientClassName; - - public HiveLockContext(SerializableHiveConf hiveConf, String clientClassName) { - this.hiveConf = hiveConf; - this.clientClassName = clientClassName; - } - } } diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLockContext.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLockContext.java new file mode 100644 index 000000000000..ecffd7f1e633 --- /dev/null +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLockContext.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.hive; + +import org.apache.paimon.catalog.CatalogLockContext; +import org.apache.paimon.options.Options; + +/** Hive {@link CatalogLockContext}. */ +public class HiveCatalogLockContext implements CatalogLockContext { + + private final SerializableHiveConf hiveConf; + private final String clientClassName; + private final Options options; + + public HiveCatalogLockContext( + SerializableHiveConf hiveConf, String clientClassName, Options options) { + this.hiveConf = hiveConf; + this.clientClassName = clientClassName; + this.options = options; + } + + @Override + public Options options() { + return options; + } + + public SerializableHiveConf hiveConf() { + return hiveConf; + } + + public String clientClassName() { + return clientClassName; + } +} diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLockFactory.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLockFactory.java new file mode 100644 index 000000000000..7c05ce3ee520 --- /dev/null +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogLockFactory.java @@ -0,0 +1,52 @@ +/* + * 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.hive; + +import org.apache.paimon.catalog.CatalogLock; +import org.apache.paimon.catalog.CatalogLockContext; +import org.apache.paimon.catalog.CatalogLockFactory; + +import org.apache.hadoop.hive.conf.HiveConf; + +import static org.apache.paimon.hive.HiveCatalogLock.LOCK_IDENTIFIER; +import static org.apache.paimon.hive.HiveCatalogLock.acquireTimeout; +import static org.apache.paimon.hive.HiveCatalogLock.checkMaxSleep; +import static org.apache.paimon.utils.Preconditions.checkArgument; + +/** Catalog lock factory for hive. */ +public class HiveCatalogLockFactory implements CatalogLockFactory { + + private static final long serialVersionUID = 1L; + + @Override + public CatalogLock createLock(CatalogLockContext context) { + checkArgument(context instanceof HiveCatalogLockContext); + HiveCatalogLockContext hiveLockContext = (HiveCatalogLockContext) context; + HiveConf conf = hiveLockContext.hiveConf().conf(); + return new HiveCatalogLock( + HiveCatalog.createClient(conf, hiveLockContext.clientClassName()), + checkMaxSleep(conf), + acquireTimeout(conf)); + } + + @Override + public String identifier() { + return LOCK_IDENTIFIER; + } +} diff --git a/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index d4af13cc08e6..baab92184129 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -14,6 +14,4 @@ # limitations under the License. org.apache.paimon.hive.HiveCatalogFactory - -# Hive catalog lock factory -org.apache.paimon.hive.HiveCatalogLock$HiveCatalogLockFactory +org.apache.paimon.hive.HiveCatalogLockFactory diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java index 668c88f1f32c..4d9753babc34 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java @@ -20,6 +20,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogLock; +import org.apache.paimon.catalog.CatalogLockFactory; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkCatalog; import org.apache.paimon.hive.annotation.Minio; @@ -695,7 +696,7 @@ public void testHiveLock() throws InterruptedException { tEnv.executeSql("CREATE TABLE t (a INT)"); Catalog catalog = ((FlinkCatalog) tEnv.getCatalog(tEnv.getCurrentCatalog()).get()).catalog(); - CatalogLock.LockFactory lockFactory = catalog.lockFactory().get(); + CatalogLockFactory lockFactory = catalog.lockFactory().get(); AtomicInteger count = new AtomicInteger(0); List threads = new ArrayList<>(); @@ -710,7 +711,8 @@ public void testHiveLock() throws InterruptedException { Thread thread = new Thread( () -> { - CatalogLock lock = lockFactory.create(catalog.lockContext().get()); + CatalogLock lock = + lockFactory.createLock(catalog.lockContext().get()); for (int j = 0; j < 10; j++) { try { lock.runWithLock("test_db", "t", unsafeIncrement); From 6f02230ef464238ef58e8811d094270dfb1a6762 Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Wed, 27 Mar 2024 13:29:09 +0800 Subject: [PATCH 69/79] [core] Shade roaringbitmap dependency into paimon-common (#3100) --- paimon-common/pom.xml | 11 ++++ .../apache/paimon/utils/RoaringBitmap32.java | 62 +++++++++++++++++++ .../src/main/resources/META-INF/NOTICE | 3 + paimon-core/pom.xml | 7 --- .../deletionvectors/BitmapDeletionVector.java | 17 +++-- 5 files changed, 84 insertions(+), 16 deletions(-) create mode 100644 paimon-common/src/main/java/org/apache/paimon/utils/RoaringBitmap32.java diff --git a/paimon-common/pom.xml b/paimon-common/pom.xml index 22d51c924cc7..95d6458d6f8d 100644 --- a/paimon-common/pom.xml +++ b/paimon-common/pom.xml @@ -135,6 +135,12 @@ under the License. 8.5.12 + + org.roaringbitmap + RoaringBitmap + 1.0.5 + + @@ -266,6 +272,7 @@ under the License. org.antlr:antlr4-runtime org.codehaus.janino:* it.unimi.dsi:fastutil + org.roaringbitmap:RoaringBitmap @@ -301,6 +308,10 @@ under the License. io.airlift org.apache.paimon.shade.io.airlift + + org.roaringbitmap + org.apache.paimon.shade.org.roaringbitmap + true diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/RoaringBitmap32.java b/paimon-common/src/main/java/org/apache/paimon/utils/RoaringBitmap32.java new file mode 100644 index 000000000000..31eb710ae8c8 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/utils/RoaringBitmap32.java @@ -0,0 +1,62 @@ +/* + * 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.utils; + +import org.roaringbitmap.RoaringBitmap; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +/** A compressed bitmap for 32-bit integer. */ +public class RoaringBitmap32 { + + public static final int MAX_VALUE = Integer.MAX_VALUE; + + private final RoaringBitmap roaringBitmap; + + public RoaringBitmap32() { + this.roaringBitmap = new RoaringBitmap(); + } + + public void add(int x) { + roaringBitmap.add(x); + } + + public boolean checkedAdd(int x) { + return roaringBitmap.checkedAdd(x); + } + + public boolean contains(int x) { + return roaringBitmap.contains(x); + } + + public boolean isEmpty() { + return roaringBitmap.isEmpty(); + } + + public void serialize(DataOutput out) throws IOException { + roaringBitmap.runOptimize(); + roaringBitmap.serialize(out); + } + + public void deserialize(DataInput in) throws IOException { + roaringBitmap.deserialize(in); + } +} diff --git a/paimon-common/src/main/resources/META-INF/NOTICE b/paimon-common/src/main/resources/META-INF/NOTICE index 6eadf7db9634..1f8f8a05858e 100644 --- a/paimon-common/src/main/resources/META-INF/NOTICE +++ b/paimon-common/src/main/resources/META-INF/NOTICE @@ -4,6 +4,9 @@ Copyright 2023-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) +- org.roaringbitmap:RoaringBitmap:1.0.5 + This project bundles the following dependencies under the BSD 3-clause license. You find them under licenses/LICENSE.antlr-runtime and licenses/LICENSE.janino. diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index 965f2de672ea..5e7339e61e92 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -33,7 +33,6 @@ under the License. 6.20.3-ververica-2.0 - 1.0.1 @@ -64,12 +63,6 @@ under the License. provided - - org.roaringbitmap - RoaringBitmap - ${RoaringBitmap.version} - - diff --git a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/BitmapDeletionVector.java b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/BitmapDeletionVector.java index 9ef2182e10bd..d15c0d210575 100644 --- a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/BitmapDeletionVector.java +++ b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/BitmapDeletionVector.java @@ -18,7 +18,7 @@ package org.apache.paimon.deletionvectors; -import org.roaringbitmap.RoaringBitmap; +import org.apache.paimon.utils.RoaringBitmap32; import java.io.ByteArrayOutputStream; import java.io.DataInput; @@ -26,20 +26,20 @@ import java.io.IOException; /** - * A {@link DeletionVector} based on {@link RoaringBitmap}, it only supports files with row count - * not exceeding {@link Integer#MAX_VALUE}. + * A {@link DeletionVector} based on {@link RoaringBitmap32}, it only supports files with row count + * not exceeding {@link RoaringBitmap32#MAX_VALUE}. */ public class BitmapDeletionVector implements DeletionVector { public static final int MAGIC_NUMBER = 1581511376; - private final RoaringBitmap roaringBitmap; + private final RoaringBitmap32 roaringBitmap; BitmapDeletionVector() { - roaringBitmap = new RoaringBitmap(); + roaringBitmap = new RoaringBitmap32(); } - private BitmapDeletionVector(RoaringBitmap roaringBitmap) { + private BitmapDeletionVector(RoaringBitmap32 roaringBitmap) { this.roaringBitmap = roaringBitmap; } @@ -71,7 +71,6 @@ public byte[] serializeToBytes() { try (ByteArrayOutputStream bos = new ByteArrayOutputStream(); DataOutputStream dos = new DataOutputStream(bos)) { dos.writeInt(MAGIC_NUMBER); - roaringBitmap.runOptimize(); roaringBitmap.serialize(dos); return bos.toByteArray(); } catch (Exception e) { @@ -80,13 +79,13 @@ public byte[] serializeToBytes() { } public static DeletionVector deserializeFromDataInput(DataInput bis) throws IOException { - RoaringBitmap roaringBitmap = new RoaringBitmap(); + RoaringBitmap32 roaringBitmap = new RoaringBitmap32(); roaringBitmap.deserialize(bis); return new BitmapDeletionVector(roaringBitmap); } private void checkPosition(long position) { - if (position > Integer.MAX_VALUE) { + if (position > RoaringBitmap32.MAX_VALUE) { throw new IllegalArgumentException( "The file has too many rows, RoaringBitmap32 only supports files with row count not exceeding 2147483647."); } From 3237e1ab198b4101b547179f0c0ddd21974ec936 Mon Sep 17 00:00:00 2001 From: Kerwin <37063904+zhuangchong@users.noreply.github.com> Date: Wed, 27 Mar 2024 13:44:33 +0800 Subject: [PATCH 70/79] [cdc] Fix cdc job mistakenly changes immutable options of existing table (#3095) --- .../action/cdc/SynchronizationActionBase.java | 23 +++++++--- .../cdc/mysql/MySqlSyncTableActionITCase.java | 43 +++++++++++++++++++ .../test/resources/mysql/sync_table_setup.sql | 9 +++- 3 files changed, 69 insertions(+), 6 deletions(-) diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java index 684408f2438e..5c04d5707e1c 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java @@ -44,6 +44,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.stream.Collectors; import static org.apache.paimon.CoreOptions.TagCreationMode.WATERMARK; @@ -178,11 +179,23 @@ protected abstract void buildSink( protected FileStoreTable alterTableOptions(Identifier identifier, FileStoreTable table) { // doesn't support altering bucket here - Map withoutBucket = new HashMap<>(tableConfig); - withoutBucket.remove(CoreOptions.BUCKET.key()); - + Map dynamicOptions = new HashMap<>(tableConfig); + dynamicOptions.remove(CoreOptions.BUCKET.key()); + + // remove immutable options and options with equal values + Map oldOptions = table.options(); + Set immutableOptionKeys = CoreOptions.getImmutableOptionKeys(); + dynamicOptions + .entrySet() + .removeIf( + entry -> + immutableOptionKeys.contains(entry.getKey()) + || Objects.equals( + oldOptions.get(entry.getKey()), entry.getValue())); + + // alter the table dynamic options List optionChanges = - withoutBucket.entrySet().stream() + dynamicOptions.entrySet().stream() .map(entry -> SchemaChange.setOption(entry.getKey(), entry.getValue())) .collect(Collectors.toList()); @@ -194,7 +207,7 @@ protected FileStoreTable alterTableOptions(Identifier identifier, FileStoreTable throw new RuntimeException("This is unexpected.", e); } - return table.copy(withoutBucket); + return table.copy(dynamicOptions); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java index 2ed3b6f466c5..fb54763cb7c6 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java @@ -1097,6 +1097,49 @@ public void testOptionsChange() throws Exception { assertThat(table.options()).containsAllEntriesOf(tableConfig); } + @Test + public void testOptionsChangeInExistingTable() throws Exception { + Map options = new HashMap<>(); + options.put("bucket", "1"); + options.put("sink.parallelism", "1"); + options.put("sequence.field", "_timestamp"); + + createFileStoreTable( + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), DataTypes.DATE(), DataTypes.TIMESTAMP(0) + }, + new String[] {"pk", "_date", "_timestamp"}), + Collections.emptyList(), + Collections.singletonList("pk"), + options); + + Map mySqlConfig = getBasicMySqlConfig(); + mySqlConfig.put("database-name", DATABASE_NAME); + mySqlConfig.put("table-name", "test_exist_options_change"); + Map tableConfig = new HashMap<>(); + // update immutable options + tableConfig.put("sequence.field", "_date"); + // update existing options + tableConfig.put("sink.parallelism", "2"); + // add new options + tableConfig.put("snapshot.expire.limit", "1000"); + + MySqlSyncTableAction action = + syncTableActionBuilder(mySqlConfig) + .withPrimaryKeys("pk") + .withTableConfig(tableConfig) + .build(); + runActionWithDefaultEnv(action); + + FileStoreTable table = getFileStoreTable(); + + assertThat(table.options().get("bucket")).isEqualTo("1"); + assertThat(table.options().get("sequence.field")).isEqualTo("_timestamp"); + assertThat(table.options().get("sink.parallelism")).isEqualTo("2"); + assertThat(table.options().get("snapshot.expire.limit")).isEqualTo("1000"); + } + @Test @Timeout(60) public void testMetadataColumns() throws Exception { diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/mysql/sync_table_setup.sql b/paimon-flink/paimon-flink-cdc/src/test/resources/mysql/sync_table_setup.sql index 949f1c99dac5..b69661b60fa7 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/mysql/sync_table_setup.sql +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/mysql/sync_table_setup.sql @@ -309,6 +309,13 @@ CREATE TABLE test_options_change ( PRIMARY KEY (pk) ); +CREATE TABLE test_exist_options_change ( + pk INT, + _date DATE, + _timestamp TIMESTAMP, + PRIMARY KEY (pk) +); + -- ################################################################################ -- testSyncShard -- ################################################################################ @@ -405,4 +412,4 @@ USE invalid_alter_bucket; CREATE TABLE t ( k INT PRIMARY KEY -); \ No newline at end of file +); From 629edfed75622f19fef55d620660530970118c94 Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Wed, 27 Mar 2024 16:26:41 +0800 Subject: [PATCH 71/79] [core] Support dv with avro format (#3105) --- .../paimon/reader/FileRecordIterator.java | 3 +- .../paimon/utils/IteratorResultIterator.java | 26 ++++++++++- .../paimon/schema/SchemaValidation.java | 6 --- .../paimon/format/avro/AvroBulkFormat.java | 9 +++- .../format/avro/AvroFileFormatTest.java | 43 +++++++++++++++++++ .../paimon/spark/sql/DeletionVectorTest.scala | 2 +- 6 files changed, 78 insertions(+), 11 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/reader/FileRecordIterator.java b/paimon-common/src/main/java/org/apache/paimon/reader/FileRecordIterator.java index 0cef8cc001e6..d22b27053f98 100644 --- a/paimon-common/src/main/java/org/apache/paimon/reader/FileRecordIterator.java +++ b/paimon-common/src/main/java/org/apache/paimon/reader/FileRecordIterator.java @@ -27,7 +27,8 @@ import java.util.function.Function; /** - * Wrap {@link RecordReader.RecordIterator} to support returning the record's row position. + * Wrap {@link RecordReader.RecordIterator} to support returning the record's row position and file + * Path. * * @param The type of the record. */ diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/IteratorResultIterator.java b/paimon-common/src/main/java/org/apache/paimon/utils/IteratorResultIterator.java index a2bffd31da05..cb42a371fa2f 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/IteratorResultIterator.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/IteratorResultIterator.java @@ -18,6 +18,8 @@ package org.apache.paimon.utils; +import org.apache.paimon.fs.Path; +import org.apache.paimon.reader.FileRecordIterator; import org.apache.paimon.reader.RecordReader; import javax.annotation.Nullable; @@ -25,22 +27,42 @@ import java.util.Iterator; /** A simple {@link RecordReader.RecordIterator} that returns the elements of an iterator. */ -public final class IteratorResultIterator extends RecyclableIterator { +public final class IteratorResultIterator extends RecyclableIterator + implements FileRecordIterator { private final Iterator records; + private final Path filePath; + private long nextFilePos; - public IteratorResultIterator(final Iterator records, final @Nullable Runnable recycler) { + public IteratorResultIterator( + final Iterator records, + final @Nullable Runnable recycler, + final Path filePath, + long pos) { super(recycler); this.records = records; + this.filePath = filePath; + this.nextFilePos = pos; } @Nullable @Override public E next() { if (records.hasNext()) { + nextFilePos++; return records.next(); } else { return null; } } + + @Override + public long returnedPosition() { + return nextFilePos - 1; + } + + @Override + public Path filePath() { + return filePath; + } } 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 7e59bb7b51b8..18c95cd2f048 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 @@ -49,8 +49,6 @@ import static org.apache.paimon.CoreOptions.CHANGELOG_PRODUCER; import static org.apache.paimon.CoreOptions.FIELDS_PREFIX; import static org.apache.paimon.CoreOptions.FULL_COMPACTION_DELTA_COMMITS; -import static org.apache.paimon.CoreOptions.FileFormatType.ORC; -import static org.apache.paimon.CoreOptions.FileFormatType.PARQUET; import static org.apache.paimon.CoreOptions.INCREMENTAL_BETWEEN; import static org.apache.paimon.CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP; import static org.apache.paimon.CoreOptions.SCAN_FILE_CREATION_TIME_MILLIS; @@ -473,10 +471,6 @@ private static void validateForDeletionVectors(TableSchema schema, CoreOptions o !schema.primaryKeys().isEmpty(), "Deletion vectors mode is only supported for tables with primary keys."); - checkArgument( - options.formatType().equals(ORC) || options.formatType().equals(PARQUET), - "Deletion vectors mode is only supported for orc or parquet file format now."); - checkArgument( options.changelogProducer() == ChangelogProducer.NONE || options.changelogProducer() == ChangelogProducer.LOOKUP, 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 abf82342a5aa..0838912141c1 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 @@ -61,6 +61,8 @@ private class AvroReader implements RecordReader { private final long end; private final Pool pool; + private final Path filePath; + private long currentRowPosition; private AvroReader(FileIO fileIO, Path path, long fileSize) throws IOException { this.fileIO = fileIO; @@ -69,6 +71,8 @@ private AvroReader(FileIO fileIO, Path path, long fileSize) throws IOException { this.reader.sync(0); this.pool = new Pool<>(1); this.pool.add(new Object()); + this.filePath = path; + this.currentRowPosition = 0; } private DataFileReader createReaderFromPath(Path path, long fileSize) @@ -101,8 +105,11 @@ public RecordIterator readBatch() throws IOException { return null; } + long rowPosition = currentRowPosition; + currentRowPosition += reader.getBlockCount(); Iterator iterator = new AvroBlockIterator(reader.getBlockCount(), reader); - return new IteratorResultIterator<>(iterator, () -> pool.recycler().recycle(ticket)); + return new IteratorResultIterator<>( + iterator, () -> pool.recycler().recycle(ticket), filePath, rowPosition); } private boolean readNextBlock() throws IOException { diff --git a/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java b/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java index 0aa15f7f227a..e9caf3b24b91 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java @@ -18,19 +18,35 @@ package org.apache.paimon.format.avro; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.format.FileFormat; +import org.apache.paimon.format.FormatReaderContext; +import org.apache.paimon.format.FormatWriter; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.PositionOutputStream; +import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.options.Options; +import org.apache.paimon.reader.RecordReader; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import java.io.IOException; import java.util.ArrayList; +import java.util.UUID; + +import static org.assertj.core.api.Assertions.assertThat; /** Test for avro file format. */ public class AvroFileFormatTest { + @TempDir java.nio.file.Path tempPath; + private static AvroFileFormat fileFormat; @BeforeAll @@ -85,4 +101,31 @@ public void testSupportedComplexDataTypes() { RowType rowType = new RowType(dataFields); fileFormat.validateDataFields(rowType); } + + @Test + void testReadRowPosition() throws IOException { + RowType rowType = DataTypes.ROW(DataTypes.INT().notNull()); + FileFormat format = new AvroFileFormat(new Options()); + + LocalFileIO fileIO = LocalFileIO.create(); + Path file = new Path(new Path(tempPath.toUri()), UUID.randomUUID().toString()); + + try (PositionOutputStream out = fileIO.newOutputStream(file, false)) { + FormatWriter writer = format.createWriterFactory(rowType).create(out, null); + for (int i = 0; i < 1000000; i++) { + writer.addElement(GenericRow.of(i)); + } + writer.flush(); + writer.finish(); + } + + try (RecordReader reader = + format.createReaderFactory(rowType) + .createReader( + new FormatReaderContext( + fileIO, file, fileIO.getFileSize(file))); ) { + reader.forEachRemainingWithPosition( + (rowPosition, row) -> assertThat(row.getInt(0) == rowPosition).isTrue()); + } + } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala index 887b6c8dfe61..f45a22caee2d 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala @@ -101,7 +101,7 @@ class DeletionVectorTest extends PaimonSparkTestBase { test("Paimon deletionVector: e2e random write") { val bucket = Random.shuffle(Seq("-1", "1", "3")).head val changelogProducer = Random.shuffle(Seq("none", "lookup")).head - val format = Random.shuffle(Seq("orc", "parquet")).head + val format = Random.shuffle(Seq("orc", "parquet", "avro")).head val batchSize = Random.nextInt(1024) + 1 val dvTbl = "deletion_vector_tbl" From f70f9d02b50cfe4438c7b67da1fecac078879ce6 Mon Sep 17 00:00:00 2001 From: YeJunHao <41894543+leaves12138@users.noreply.github.com> Date: Wed, 27 Mar 2024 16:42:19 +0800 Subject: [PATCH 72/79] [core] Introduce File index format for data skipping (#3068) --- .../paimon/fileindex/FileIndexFormat.java | 280 ++++++++++++++++++ .../paimon/fileindex/FileIndexPredicate.java | 165 +++++++++++ .../paimon/fileindex/FileIndexReader.java | 108 +++++++ .../paimon/fileindex/FileIndexWriter.java | 27 ++ .../apache/paimon/fileindex/FileIndexer.java | 36 +++ .../paimon/fs/ByteArraySeekableStream.java | 105 +++++++ .../fileindex/FileIndexFormatFormatTest.java | 66 +++++ .../fs/ByteArraySeekableStreamTest.java | 72 +++++ .../org/apache/paimon/utils/RandomUtil.java | 43 +++ .../paimon/sort/zorder/ZIndexerTest.java | 12 +- .../paimon/utils/TestZOrderByteUtil.java | 28 +- 11 files changed, 911 insertions(+), 31 deletions(-) create mode 100644 paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexFormat.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexPredicate.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexReader.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexWriter.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexer.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/fs/ByteArraySeekableStream.java create mode 100644 paimon-common/src/test/java/org/apache/paimon/fileindex/FileIndexFormatFormatTest.java create mode 100644 paimon-common/src/test/java/org/apache/paimon/fs/ByteArraySeekableStreamTest.java create mode 100644 paimon-common/src/test/java/org/apache/paimon/utils/RandomUtil.java 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 new file mode 100644 index 000000000000..c9b827ee72f0 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexFormat.java @@ -0,0 +1,280 @@ +/* + * 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.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.fs.SeekableInputStream; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.IOUtils; +import org.apache.paimon.utils.Pair; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.DataInputStream; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.IOException; +import java.io.OutputStream; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +/** + * File index file format. Put all column and offset in the header. + * + *
+ * _______________________________________    _____________________
+ * |     magic    |version|head length |
+ * |-------------------------------------|
+ * |   index type        |body info size|
+ * |-------------------------------------|
+ * | column name 1 |start pos |length  |
+ * |-------------------------------------|            HEAD
+ * | column name 2 |start pos |length  |
+ * |-------------------------------------|
+ * | column name 3 |start pos |length  |
+ * |-------------------------------------|
+ * |                 ...                 |
+ * |-------------------------------------|
+ * |                 ...                 |
+ * |-------------------------------------|
+ * |  redundant length |redundant bytes |
+ * |-------------------------------------|    ---------------------
+ * |                BODY                 |
+ * |                BODY                 |
+ * |                BODY                 |             BODY
+ * |                BODY                 |
+ * |_____________________________________|    _____________________
+ *
+ * magic:                            8 bytes long
+ * version:                          4 bytes int
+ * head length:                      4 bytes int
+ * index type:                       var bytes utf (length + bytes)
+ * body info size:                   4 bytes int (how many column items below)
+ * column name:                      var bytes utf
+ * start pos:                        4 bytes int
+ * length:                           4 bytes int
+ * redundant length:                 4 bytes int (for compatibility with later versions, in this version, content is zero)
+ * redundant bytes:                  var bytes (for compatibility with later version, in this version, is empty)
+ * BODY:                             column bytes + column bytes + column bytes + .......
+ *
+ * 
+ */ +public final class FileIndexFormat { + + private static final long MAGIC = 1493475289347502L; + + enum Version { + V_1(1); + + private final int version; + + Version(int version) { + this.version = version; + } + + public int version() { + return version; + } + } + + public static Writer createWriter(OutputStream outputStream) { + return new Writer(outputStream); + } + + public static Reader createReader(SeekableInputStream inputStream, RowType fileRowType) { + return new Reader(inputStream, fileRowType); + } + + /** Writer for file index file. */ + public static class Writer implements Closeable { + + private final DataOutputStream dataOutputStream; + + // for version compatible + private static final int REDUNDANT_LENGTH = 0; + + public Writer(OutputStream outputStream) { + this.dataOutputStream = new DataOutputStream(outputStream); + } + + public void writeColumnIndex(String indexType, Map bytesMap) + throws IOException { + + Map> bodyInfo = new HashMap<>(); + + // construct body + ByteArrayOutputStream baos = new ByteArrayOutputStream(256); + for (Map.Entry entry : bytesMap.entrySet()) { + int startPosition = baos.size(); + baos.write(entry.getValue()); + bodyInfo.put(entry.getKey(), Pair.of(startPosition, baos.size() - startPosition)); + } + byte[] body = baos.toByteArray(); + + writeHead(indexType, bodyInfo); + + // writeBody + dataOutputStream.write(body); + } + + private void writeHead(String indexType, Map> bodyInfo) + throws IOException { + + int headLength = calculateHeadLength(indexType, bodyInfo); + + // writeMagic + dataOutputStream.writeLong(MAGIC); + // writeVersion + dataOutputStream.writeInt(Version.V_1.version()); + // writeHeadLength + dataOutputStream.writeInt(headLength); + // writeIndexType + dataOutputStream.writeUTF(indexType); + // writeColumnSize + dataOutputStream.writeInt(bodyInfo.size()); + // writeColumnInfo, offset = headLength + for (Map.Entry> entry : bodyInfo.entrySet()) { + dataOutputStream.writeUTF(entry.getKey()); + dataOutputStream.writeInt(entry.getValue().getLeft() + headLength); + dataOutputStream.writeInt(entry.getValue().getRight()); + } + // writeRedundantLength + dataOutputStream.writeInt(REDUNDANT_LENGTH); + } + + private int calculateHeadLength( + String indexType, Map> bodyInfo) throws IOException { + // magic 8 bytes, version 4 bytes, head length 4 bytes, + // column size 4 bytes, body info start&end 8 bytes per + // item, redundant length 4 bytes; + int baseLength = 8 + 4 + 4 + 4 + bodyInfo.size() * 8 + 4; + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutput dataOutput = new DataOutputStream(baos); + dataOutput.writeUTF(indexType); + for (String s : bodyInfo.keySet()) { + dataOutput.writeUTF(s); + } + + return baseLength + baos.size(); + } + + @Override + public void close() throws IOException { + IOUtils.closeQuietly(dataOutputStream); + } + } + + /** Reader for file index file. */ + public static class Reader implements Closeable { + + private final SeekableInputStream seekableInputStream; + // get header and cache it. + private final Map> header = new HashMap<>(); + private final Map fields = new HashMap<>(); + private final String type; + + public Reader(SeekableInputStream seekableInputStream, RowType fileRowType) { + this.seekableInputStream = seekableInputStream; + DataInputStream dataInputStream = new DataInputStream(seekableInputStream); + fileRowType.getFields().forEach(field -> this.fields.put(field.name(), field)); + try { + long magic = dataInputStream.readLong(); + if (magic != MAGIC) { + throw new RuntimeException("This file is not file index file."); + } + + int version = dataInputStream.readInt(); + if (version != Version.V_1.version()) { + throw new RuntimeException( + "This index file is version of " + + version + + ", not in supported version list [" + + Version.V_1.version() + + "]"); + } + + int headLength = dataInputStream.readInt(); + byte[] head = new byte[headLength - 8 - 4 - 4]; + dataInputStream.readFully(head); + + try (DataInputStream dataInput = + new DataInputStream(new ByteArrayInputStream(head))) { + this.type = dataInput.readUTF(); + int columnSize = dataInput.readInt(); + for (int i = 0; i < columnSize; i++) { + this.header.put( + dataInput.readUTF(), + Pair.of(dataInput.readInt(), dataInput.readInt())); + } + } + + } catch (IOException e) { + IOUtils.closeQuietly(seekableInputStream); + throw new RuntimeException( + "Exception happens while construct file index reader.", e); + } + } + + public FileIndexReader readColumnIndex(String columnName) { + + return readColumnInputStream(columnName) + .map( + serializedBytes -> + FileIndexer.create(type, fields.get(columnName).type()) + .createReader() + .recoverFrom(serializedBytes)) + .orElse(null); + } + + @VisibleForTesting + Optional readColumnInputStream(String columnName) { + return Optional.ofNullable(header.getOrDefault(columnName, null)) + .map( + startAndLength -> { + byte[] b = new byte[startAndLength.getRight()]; + try { + seekableInputStream.seek(startAndLength.getLeft()); + int n = 0; + int len = b.length; + // read fully until b is full else throw. + while (n < len) { + int count = seekableInputStream.read(b, n, len - n); + if (count < 0) { + throw new EOFException(); + } + n += count; + } + } catch (IOException e) { + throw new RuntimeException(e); + } + return b; + }); + } + + @Override + public void close() throws IOException { + IOUtils.closeQuietly(seekableInputStream); + } + } +} 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 new file mode 100644 index 000000000000..b07c6b8f08fd --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexPredicate.java @@ -0,0 +1,165 @@ +/* + * 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.apache.paimon.fs.ByteArraySeekableStream; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.SeekableInputStream; +import org.apache.paimon.predicate.CompoundPredicate; +import org.apache.paimon.predicate.FieldRef; +import org.apache.paimon.predicate.LeafPredicate; +import org.apache.paimon.predicate.Or; +import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.predicate.PredicateVisitor; +import org.apache.paimon.types.RowType; + +import javax.annotation.Nullable; + +import java.io.Closeable; +import java.io.IOException; +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; + +/** Utils to check secondary index (e.g. bloom filter) predicate. */ +public class FileIndexPredicate implements Closeable { + + private final FileIndexFormat.Reader reader; + private final Map fieldPredicates = new HashMap<>(); + + public FileIndexPredicate(Path path, FileIO fileIO, RowType fileRowType) throws IOException { + this(fileIO.newInputStream(path), fileRowType); + } + + public FileIndexPredicate(byte[] serializedBytes, RowType fileRowType) { + this(new ByteArraySeekableStream(serializedBytes), fileRowType); + } + + public FileIndexPredicate(SeekableInputStream inputStream, RowType fileRowType) { + this.reader = FileIndexFormat.createReader(inputStream, fileRowType); + } + + public boolean testPredicate(@Nullable Predicate filePredicate) { + if (filePredicate == null) { + return true; + } + + 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; + } + } + return true; + } + + private Set getRequiredNames(Predicate filePredicate) { + return filePredicate.visit( + new PredicateVisitor>() { + final Set names = new HashSet<>(); + + @Override + public Set visit(LeafPredicate predicate) { + names.add(predicate.fieldName()); + return names; + } + + @Override + public Set visit(CompoundPredicate predicate) { + for (Predicate child : predicate.children()) { + child.visit(this); + } + return names; + } + }); + } + + @Override + public void close() throws IOException { + this.reader.close(); + } + + /** Predicate test worker. */ + private static class FileIndexFieldPredicate implements PredicateVisitor { + + private final String columnName; + private final FileIndexReader fileIndexReader; + + public FileIndexFieldPredicate(String columnName, FileIndexReader fileIndexReader) { + this.columnName = columnName; + this.fileIndexReader = fileIndexReader; + } + + public Boolean test(Predicate predicate) { + return predicate.visit(this); + } + + @Override + public Boolean visit(LeafPredicate predicate) { + if (columnName.equals(predicate.fieldName())) { + return predicate + .function() + .visit( + fileIndexReader, + new FieldRef( + predicate.index(), predicate.fieldName(), predicate.type()), + predicate.literals()); + } + return true; + } + + @Override + public Boolean visit(CompoundPredicate predicate) { + + if (predicate.function() instanceof Or) { + for (Predicate predicate1 : predicate.children()) { + if (predicate1.visit(this)) { + return true; + } + } + return false; + + } else { + for (Predicate predicate1 : predicate.children()) { + if (!predicate1.visit(this)) { + return false; + } + } + return true; + } + } + } +} 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 new file mode 100644 index 000000000000..6d9404564127 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexReader.java @@ -0,0 +1,108 @@ +/* + * 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.apache.paimon.predicate.FieldRef; +import org.apache.paimon.predicate.FunctionVisitor; + +import java.util.List; + +/** + * Read file index from serialized bytes. Return true, means we need to search this file, else means + * needn't. + */ +public interface FileIndexReader extends FunctionVisitor { + + FileIndexReader recoverFrom(byte[] serializedBytes); + + @Override + default Boolean visitIsNotNull(FieldRef fieldRef) { + return true; + } + + @Override + default Boolean visitIsNull(FieldRef fieldRef) { + return true; + } + + @Override + default Boolean visitStartsWith(FieldRef fieldRef, Object literal) { + return true; + } + + @Override + default Boolean visitLessThan(FieldRef fieldRef, Object literal) { + return true; + } + + @Override + default Boolean visitGreaterOrEqual(FieldRef fieldRef, Object literal) { + return true; + } + + @Override + default Boolean visitNotEqual(FieldRef fieldRef, Object literal) { + return true; + } + + @Override + default Boolean visitLessOrEqual(FieldRef fieldRef, Object literal) { + return true; + } + + @Override + default Boolean visitEqual(FieldRef fieldRef, Object literal) { + return true; + } + + @Override + default Boolean visitGreaterThan(FieldRef fieldRef, Object literal) { + return true; + } + + @Override + default Boolean visitIn(FieldRef fieldRef, List literals) { + for (Object key : literals) { + if (visitEqual(fieldRef, key)) { + return true; + } + } + return false; + } + + @Override + default Boolean visitNotIn(FieldRef fieldRef, List literals) { + for (Object key : literals) { + if (visitNotEqual(fieldRef, key)) { + return true; + } + } + return false; + } + + @Override + default Boolean visitAnd(List children) { + throw new UnsupportedOperationException("Should not invoke this"); + } + + @Override + default Boolean visitOr(List children) { + throw new UnsupportedOperationException("Should not invoke this"); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexWriter.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexWriter.java new file mode 100644 index 000000000000..9eab19cde83b --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexWriter.java @@ -0,0 +1,27 @@ +/* + * 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; + +/** To write file index. */ +public interface FileIndexWriter { + + void write(Object key); + + byte[] serializedBytes(); +} 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 new file mode 100644 index 000000000000..e7e3d40bfe6f --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexer.java @@ -0,0 +1,36 @@ +/* + * 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.apache.paimon.types.DataType; + +/** File index interface. To build a file index. */ +public interface FileIndexer { + + FileIndexWriter createWriter(); + + FileIndexReader createReader(); + + static FileIndexer create(String type, DataType dataType) { + switch (type) { + default: + throw new RuntimeException("Doesn't support filter type: " + type); + } + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/fs/ByteArraySeekableStream.java b/paimon-common/src/main/java/org/apache/paimon/fs/ByteArraySeekableStream.java new file mode 100644 index 000000000000..d6536927b100 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/fs/ByteArraySeekableStream.java @@ -0,0 +1,105 @@ +/* + * 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.fs; + +import java.io.ByteArrayInputStream; +import java.io.EOFException; +import java.io.IOException; + +/** Wrap byte buf to a seekable input stream. */ +public class ByteArraySeekableStream extends SeekableInputStream { + + private final ByteArrayStream byteArrayStream; + + public ByteArraySeekableStream(byte[] buf) { + this.byteArrayStream = new ByteArrayStream(buf); + } + + @Override + public void seek(long desired) throws IOException { + byteArrayStream.seek((int) desired); + } + + @Override + public long getPos() throws IOException { + return byteArrayStream.getPos(); + } + + @Override + public int read() throws IOException { + return byteArrayStream.read(); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + return byteArrayStream.read(b, off, len); + } + + @Override + public int read(byte[] b) throws IOException { + return byteArrayStream.read(b); + } + + @Override + public long skip(long n) throws IOException { + return byteArrayStream.skip(n); + } + + @Override + public int available() throws IOException { + return byteArrayStream.available(); + } + + @Override + public synchronized void mark(int readlimit) { + byteArrayStream.mark(readlimit); + } + + @Override + public synchronized void reset() throws IOException { + byteArrayStream.reset(); + } + + @Override + public boolean markSupported() { + return byteArrayStream.markSupported(); + } + + @Override + public void close() throws IOException { + byteArrayStream.close(); + } + + private static class ByteArrayStream extends ByteArrayInputStream { + public ByteArrayStream(byte[] buf) { + super(buf); + } + + public void seek(int position) throws IOException { + if (position >= count) { + throw new EOFException("Can't seek position: " + position + ", length is " + count); + } + pos = position; + } + + public long getPos() { + return pos; + } + } +} diff --git a/paimon-common/src/test/java/org/apache/paimon/fileindex/FileIndexFormatFormatTest.java b/paimon-common/src/test/java/org/apache/paimon/fileindex/FileIndexFormatFormatTest.java new file mode 100644 index 000000000000..0f157ae99545 --- /dev/null +++ b/paimon-common/src/test/java/org/apache/paimon/fileindex/FileIndexFormatFormatTest.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.fileindex; + +import org.apache.paimon.fs.ByteArraySeekableStream; +import org.apache.paimon.types.RowType; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; + +import static org.apache.paimon.utils.RandomUtil.randomBytes; +import static org.apache.paimon.utils.RandomUtil.randomString; + +/** Test for {@link FileIndexFormat}. */ +public class FileIndexFormatFormatTest { + + private static final Random RANDOM = new Random(); + + @Test + public void testWriteRead() throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + FileIndexFormat.Writer writer = FileIndexFormat.createWriter(baos); + + String type = randomString(RANDOM.nextInt(100)); + Map indexes = new HashMap<>(); + for (int i = 0; i < RANDOM.nextInt(1000); i++) { + indexes.put(randomString(RANDOM.nextInt(20)), randomBytes(RANDOM.nextInt(100000))); + } + + writer.writeColumnIndex(type, indexes); + writer.close(); + + byte[] indexBytes = baos.toByteArray(); + + FileIndexFormat.Reader reader = + FileIndexFormat.createReader( + new ByteArraySeekableStream(indexBytes), RowType.builder().build()); + + for (String s : indexes.keySet()) { + byte[] b = reader.readColumnInputStream(s).orElseThrow(RuntimeException::new); + Assertions.assertThat(b).containsExactly(indexes.get(s)); + } + } +} diff --git a/paimon-common/src/test/java/org/apache/paimon/fs/ByteArraySeekableStreamTest.java b/paimon-common/src/test/java/org/apache/paimon/fs/ByteArraySeekableStreamTest.java new file mode 100644 index 000000000000..e725e2a3dea2 --- /dev/null +++ b/paimon-common/src/test/java/org/apache/paimon/fs/ByteArraySeekableStreamTest.java @@ -0,0 +1,72 @@ +/* + * 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.fs; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Random; + +import static org.apache.paimon.utils.RandomUtil.randomBytes; + +/** Test for {@link ByteArraySeekableStream}. */ +public class ByteArraySeekableStreamTest { + + private static final Random RANDOM = new Random(); + + @Test + public void testBasic() throws IOException { + int bl = 100000; + byte[] b = randomBytes(bl); + ByteArraySeekableStream byteArraySeekableStream = new ByteArraySeekableStream(b); + + Assertions.assertThat(byteArraySeekableStream.available()).isEqualTo(b.length); + + for (int i = 0; i < RANDOM.nextInt(1000); i++) { + int position = RANDOM.nextInt(bl); + int length = RANDOM.nextInt(b.length - position - 1); + byte[] expected = new byte[length]; + System.arraycopy(b, position, expected, 0, length); + + byte[] actual = new byte[length]; + byteArraySeekableStream.seek(position); + byteArraySeekableStream.read(actual); + Assertions.assertThat(actual).containsExactly(expected); + } + + for (int i = 0; i < RANDOM.nextInt(1000); i++) { + int position = RANDOM.nextInt(bl); + byteArraySeekableStream.seek(position); + for (int j = 0; j < 100; j++) { + Assertions.assertThat(b[position + j]) + .isEqualTo((byte) byteArraySeekableStream.read()); + } + } + } + + @Test + public void testThrow() { + int bl = 10; + byte[] b = randomBytes(bl); + ByteArraySeekableStream byteArraySeekableStream = new ByteArraySeekableStream(b); + Assertions.assertThatCode(() -> byteArraySeekableStream.seek(10)) + .hasMessage("Can't seek position: 10, length is 10"); + } +} diff --git a/paimon-common/src/test/java/org/apache/paimon/utils/RandomUtil.java b/paimon-common/src/test/java/org/apache/paimon/utils/RandomUtil.java new file mode 100644 index 000000000000..e6d6805b2b34 --- /dev/null +++ b/paimon-common/src/test/java/org/apache/paimon/utils/RandomUtil.java @@ -0,0 +1,43 @@ +/* + * 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.utils; + +import java.util.Random; + +/** Utils for tests. */ +public class RandomUtil { + + private static final Random RANDOM = new Random(); + + public static byte[] randomBytes(int length) { + byte[] b = new byte[length]; + RANDOM.nextBytes(b); + return b; + } + + public static String randomString(int length) { + byte[] buffer = new byte[length]; + + for (int i = 0; i < length; i += 1) { + buffer[i] = (byte) ('a' + RANDOM.nextInt(26)); + } + + return new String(buffer); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/sort/zorder/ZIndexerTest.java b/paimon-core/src/test/java/org/apache/paimon/sort/zorder/ZIndexerTest.java index 801a1927db79..c94327817b36 100644 --- a/paimon-core/src/test/java/org/apache/paimon/sort/zorder/ZIndexerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/sort/zorder/ZIndexerTest.java @@ -34,6 +34,8 @@ import java.util.Arrays; import java.util.Random; +import static org.apache.paimon.utils.RandomUtil.randomString; + /** Tests for {@link ZIndexer}. */ public class ZIndexerTest { @@ -135,14 +137,4 @@ public void testZIndexerForVarcharWithNull() { } } } - - public static String randomString(int length) { - byte[] buffer = new byte[length]; - - for (int i = 0; i < length; i += 1) { - buffer[i] = (byte) ('a' + RANDOM.nextInt(26)); - } - - return new String(buffer); - } } diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/TestZOrderByteUtil.java b/paimon-core/src/test/java/org/apache/paimon/utils/TestZOrderByteUtil.java index dded0cf7ee39..92786b701c4f 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/TestZOrderByteUtil.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/TestZOrderByteUtil.java @@ -26,6 +26,9 @@ import java.util.Arrays; import java.util.Random; +import static org.apache.paimon.utils.RandomUtil.randomBytes; +import static org.apache.paimon.utils.RandomUtil.randomString; + /* This file is based on source code from the Iceberg Project (http://iceberg.apache.org/), licensed by the Apache * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for * additional information regarding copyright ownership. */ @@ -352,8 +355,8 @@ public void testStringOrdering() { ByteBuffer aBuffer = ByteBuffer.allocate(128); ByteBuffer bBuffer = ByteBuffer.allocate(128); for (int i = 0; i < NUM_TESTS; i++) { - String aString = randomString(); - String bString = randomString(); + String aString = randomString(50); + String bString = randomString(50); int stringCompare = Integer.signum(aString.compareTo(bString)); byte[] aBytes = ZOrderByteUtils.stringToOrderedBytes(aString, 128, aBuffer).array(); byte[] bBytes = ZOrderByteUtils.stringToOrderedBytes(bString, 128, bBuffer).array(); @@ -380,8 +383,8 @@ public void testByteTruncateOrFill() { ByteBuffer aBuffer = ByteBuffer.allocate(128); ByteBuffer bBuffer = ByteBuffer.allocate(128); for (int i = 0; i < NUM_TESTS; i++) { - byte[] aBytesRaw = randomBytes(); - byte[] bBytesRaw = randomBytes(); + byte[] aBytesRaw = randomBytes(50); + byte[] bBytesRaw = randomBytes(50); int stringCompare = Integer.signum( UnsignedBytes.lexicographicalComparator() @@ -405,21 +408,4 @@ public void testByteTruncateOrFill() { byteCompare); } } - - private byte[] randomBytes() { - byte[] binary = new byte[random.nextInt(50)]; - random.nextBytes(binary); - return binary; - } - - private String randomString() { - int length = random.nextInt(50); - byte[] buffer = new byte[length]; - - for (int i = 0; i < length; i += 1) { - buffer[i] = (byte) ('a' + random.nextInt(26)); - } - - return new String(buffer); - } } From 241fb8fe7b3f2e9c9a3884a755a079cb5fe9cbca Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Thu, 28 Mar 2024 13:28:26 +0800 Subject: [PATCH 73/79] [test] Fix flaky test in `KeyValueFileStoreScanTest.testWithValueFilter` (#3109) --- .../operation/KeyValueFileStoreScanTest.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java index 7f1fc0d3c90e..8ea35082a5cb 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java @@ -150,13 +150,17 @@ public void testWithKeyFilter() throws Exception { @Test public void testWithValueFilter() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); - List data = generateData(100, Math.abs(random.nextInt(1000))); + // 0 <= hr <= 999 + List data = generateData(100, random.nextInt(1000)); writeData(data, 0); - data = generateData(100, Math.abs(random.nextInt(1000)) + 1000); + // 1000 <= hr <= 1999 + data = generateData(100, random.nextInt(1000) + 1000); writeData(data, 1); - data = generateData(100, Math.abs(random.nextInt(1000)) + 2000); + // 2000 <= hr <= 2999 + data = generateData(100, random.nextInt(1000) + 2000); writeData(data, 2); - generateData(100, Math.abs(random.nextInt(1000)) + 3000); + // 3000 <= hr <= 3999 + data = generateData(100, random.nextInt(1000) + 3000); Snapshot snapshot = writeData(data, 3); KeyValueFileStoreScan scan = store.newScan(); @@ -167,7 +171,7 @@ public void testWithValueFilter() throws Exception { scan.withSnapshot(snapshot.id()); scan.withValueFilter( new PredicateBuilder(TestKeyValueGenerator.DEFAULT_ROW_TYPE) - .between(1, 1000, 2000)); + .between(1, 1000, 1999)); List filesFiltered = scan.plan().files(); From 4d5e676e1e8037b923ae54628e3f24be88604657 Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Thu, 28 Mar 2024 13:51:18 +0800 Subject: [PATCH 74/79] [doc] Introduce deletion vector documentation page (#3107) --- .../primary-key-table/deletion-vectors.md | 51 ++++++++++++++++++ docs/static/img/deletion-vectors-overview.png | Bin 0 -> 197804 bytes 2 files changed, 51 insertions(+) create mode 100644 docs/content/concepts/primary-key-table/deletion-vectors.md create mode 100644 docs/static/img/deletion-vectors-overview.png diff --git a/docs/content/concepts/primary-key-table/deletion-vectors.md b/docs/content/concepts/primary-key-table/deletion-vectors.md new file mode 100644 index 000000000000..a726b5f26ec5 --- /dev/null +++ b/docs/content/concepts/primary-key-table/deletion-vectors.md @@ -0,0 +1,51 @@ +--- +title: "Deletion Vectors" +weight: 6 +type: docs +aliases: +- /concepts/primary-key-table/deletion-vectors.html +--- + + +# Deletion Vectors + +## Overview + +The Deletion Vectors mode is designed to takes into account both data reading and writing efficiency. + +In this mode, additional overhead (looking up LSM Tree and generating the corresponding Deletion File) will be introduced during writing, +but during reading, data can be directly retrieved by employing data with deletion vectors, avoiding additional merge costs between different files. + +Furthermore, data reading concurrency is no longer limited, and non-primary key columns can also be used for filter push down. +Generally speaking, in this mode, we can get a huge improvement in read performance without losing too much write performance. + +{{< img src="/img/deletion-vectors-overview.png">}} + +## Usage + +By specifying `'deletion-vectors.enabled' = 'true'`, the Deletion Vectors mode can be enabled. + +## Limitation + +- Only primary key table supports this feature. +- `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/static/img/deletion-vectors-overview.png b/docs/static/img/deletion-vectors-overview.png new file mode 100644 index 0000000000000000000000000000000000000000..9b236bb7a5195cf9fabb5b3da3e1b69c1f12ee8f GIT binary patch literal 197804 zcmc$_Wl&sE7bV)b2ZCGB;BE;n9fE5E!6CRyaCZrTK!CAt7$Is0r`Ywbgnnu;tACOIYu1j3P*lZJpm$bujcA{RO$@DA0d zhfWX(bHV1_J2e|KGZ08FDk%j`EpdT3==k}0_cgNI=OE~K#s#sY!VUs*L1=N<>u+jc zC3H*=i%%%pC@){Kv;AWIzP5+iuZM^Afso<~MVAl?H|>lVybqS0Uw(Vs>TR{X?dR%p z#Nv6BB_siYs#QoQV3dKrF^tB`px$wB5a6fXAOwNPCCfV^h7#rHU%t#`4rO`N_6{VG z+_+1g@%ja`;igaksL)-!cH^+w~%S}C3|6Oc`H3=^F=%{xG*Qu&$szTWnwAe11qFd8hmPSYyvAE@EqN$pYfr5NUMY&#H~(J2u|){q5}ID^>M zP=6s&g;0&-8ikmgP{n}p!XlaRa3;epXw3TYYa(^NF>T_8gX7f@q1Z4fQ@Ix$p|ju~ zH5N^b!(iLooZO9ZvK{CgAMmN9KzU)@c>cdJO=#QX1QR|N&_T>mAYoAY89AiMFK#pq z->A&dZ^JrGtVtPrF*jl{x_*E3N`0S7l!}w8FpQu?uuZv5Tu8zfa~lEsSXZ5_^OhZr zB2KxRvzOR(sQSLDuL@r6S@k%7El3#~%-7Sn7UGQY%WVm=gu0|+Kwp6~7rx*9b?wwk zxrJc`i8>bk*Yza)=I|!;X6h#6Cg4Ff1YaudqbfShYy96~0%0!TSL=l9jO#)s&|)w@ zW7Es-C_huGPXwR1s*}93=2a;fN)nXhsS2zi!ZrdWA=8q=sy@;_Wd#N1-#1J7wdD(b zDwKeCt0v#Wj9{wf0yQWLDVfsAiEk6G6Bqlh2b@?4zhLxaB*+cq+80biMj_d1;;d89 z@+6@_zpbP}twH6%w@Jq8QJQ9~8BIaysvJdBFr%UuC3sp&ntocpSlKj7HDk-%%3_Kq zH4inrwQ4or7SEO#L0+kKm!%comYvREPTNk8mR?TarPGe2^yZht4q;hEms6kIG=w9w zRk#YN^E@XbXJ)2f&6v%k6l<5{Xdau;U=yH5#t+Q@7VP>pbR+uC3e`8_WA7aq4xi)b z&37J+ZvA)P)e9zyKgn*?GHOg2^S!H;uGKxSCYBij+f~e(mK*B9>mn>Ct(wL*N7&Q7 z&9{*pbQZxP5Z@AyuUA@!FJBpENxq62^~kCfe%FL;=(Oy%%)6}76zan-rYaV1l(u5i zN}}Y>n2|Q{<72kji+#I&wB4c0>mAR%IPz5T-=sfytogm{<#^4>3dxhn>M34O?2?V} z=vityU38>c5Aza}JMkTHKYcW{#kZ;%Em$z`(Cmhqr90_4-Sx+qZd=s%|M+uIFEpBI zk!{j8^$>w#ziv+Zv$*D$BZj`P{ug#nJ&%#DQI{-+Y%Q^}#V__f^@DY#4SRK#I+1#n z)sFQ67G~y&)*7Z$Cg&rw18=50Di7a@2sD3e7Vs$da5~^P@V?TzYIw!`0^tSQD?Lgh z5j|l{N>8B`*F2$-PkQj_ql=k^9^JjAHPIc@&4h!RZdk^9tM^rJsyLfW4gC$Yo3foB zH@6RxH#~kH;(D@OV_yqj2QL)XqSRiG9-VWatoa`%}WgqG|w$%v-_tUD=Cefbi6W6x}Fz~}R5wy*n94N=Bafecb?Qqb{)&#E!XODTf9V}UEP-jDJs^5OEH@QhJo z-sefga+EYI70v?AO-|AD80A>S3dN@`9QX%US%@Hu2qeP0t;L}w#m^~#EKqHYmYGOY z^-_*Q`IX$c++xBT<_r8}tVrp$JcWt|7lYST-dW{9aA|*2g;S8{H~TqZ!RDo&pE6 z3x+NC=W%(|$A=h8Q<@GCL!;2LfpYKqw;oP?sy}jUuWb(5R<7OKy#mLA`;p+bhPHP5 zKlk6vbYGs`b%x5LIZ>X8OizBCsVa(}(D5UsOr{Pvbv+yNj0>Z*kr)*Ia}s^ZB@#Ok zOC%tr?5*5A(zTtP9F|1CzpNkkX#2STL~zGcs#2P>B1v)cpjzRNigoK0CftAeU| z+wD-N$kl4fJ!PNct%1`|kC7u`_o@BuH2r`E&ZX_8HW#J?mAU2A=If@!Cf}peW1OY2 zmghUEqtJ=SGAa{lBC!wN22axW@|RBA?E+VV*O>>G`>l8Ud%NA@Myo@s7VUw%TwXHX zAD#@3y$^#PJEA&+Iw`oeTXzCkyoH1{Z*XS<54D-d}7SnmLBNvNPKoUp))(58pgJz z5`&1|v^K2?S~-w~Zy_lUjj&^)zq6sf?iP)?ix1IyuSj=Cia<`IgNliQa~ql=JxhKE$t8NaP$t| zf&QE^!tXv+JAi`jI|P`gk3VHiu%5h4%e;M_-dGhNt?YRDVE9bXuu3_VgwbmRU|kwB zIfya{>!fe%nk6=kJC&;Op^TZqPGWe?<#W158+Dqv&e(QIBb$`rhHAg>Tw@{Vv!#Lq_-TiUlg!#Q< zMM**5*uzp)zt)9Y%B9!oL7Z>Cb1~%dDXZ6&R=&=(@np~VF7rFdcFV|(*C&dz0QJ4= zfSmTb_PDteEh;RQZ12ydXY;uzEuZTOrthznxT3%u{l+=?BM%3v>#P*zFI_bGVPItyk`Kis}c* z=0epD>w#@g+_6t_;oOHmZauXMM5!%pw0>H6a}rW>{k63qoQvNo4;_ zRu~Xvv{tl4Y#z0=0@(TM0_0wP`@R<5kabL6l=G$-`-$+|EByO;{_qA**!zp2C&7{yG&h!~O^49$wL zO5T^p@{3YGP&)f2bk!lNUt= zO&Ed!6z#gA?g4QD;rwi2Q5g5%zG0B?wixz@BFhU(o5VBGQ^p{E3;FB#-qb?MA(3)d z8l{vaZ!c`ECe<{NfN?KFO`rDNI`3$I34t)Niz98m52Ge8Z9+V&=bsKcvebc{^iQ6> zBi9m|>|i4*kKorR$4Qg#N=<7P^U?R=>oAr+ZTC*E*m)~5F+dVek?Ee&B( zP*rDiU@})Gi5f#{?$AU4=`j^rp`!Da$jj57ew1{6qN>ADz<%xHc!f-vCv8g-UT+Ye zj-VHl+V{oUz2@-5;#X4-#FBjk!B#p zDGeSip4GOIZ05YxC>hh0VbCmuSnu*<>U)D%B+(Ac&oWJ4mFsPmkf&FOUpaaGnZO#m zxIfux9>RgCNxVRT%F(}N{Ejfc&I+o+sA835rn5Im{;U~n9mP-89R1_{$S$HIhK{>D zZxa9e1nMd`b;HcaB5UfrvYdm2;WYzm%_C786%;Fz(q_tDBbnBMgNRTYTL_zOFQU70 z>0&~%0Xgloa0ok*NKf*6o|aS&1Z6oYWTjht!F)H0j%p-2I(njy3+CM4K7T`U;3H$I z<48n-mww!azE`TyGp|1VGmpN3iEN06ja>;IM8`zjGF-|NEdAvaLb9SQ*QRn+Aw`8Z z@a3Lc9l7>XcmX}7mUtA7voQg^24behB~(TcBZZ}AZ~bF^Vny=jM`$M;`|XfM1=Jtr zIvzAqIO)l{I7xecNr;nC0=Ckho@RbiJ~C~o`2G5j-i)4t`N#t{%G+uS)?$rj;#bMX zOKvTyrg%|~p^(JLI+8Wn)oHFsr$T_9Sf5GY{A;pMI{5Vs6QflS0ePFy52|gg3&~9c zY9&QNV%{dzJU4pWB+Lm5b8I~>?5+<)-D-JIm*NvI2g89AByuRPAoKUkx$SY>G_I;{ z@t=Y)9JU!@^k!#ud%>JtHGEx)6S)9>k#t_U6%MEBsNg|5pb=A(VnKev8|7`0N2AD4Ddql= zV?zehZj6+jdaq*SlvkF0-C&>-RohBZkc8E4H#3#evDjxzfmOZNl^$8&uyG_F2g(zx z^+u(qtdXzWGaW8k^@p3BDKgDEXx>2XC!Xg_^4w2sUW?c~@3^F3KUS|{LQpX=*qknU zrWkdN%So8~%SUb@XV)#xe$V{R_cIO5=u?V$n6K3CHJKW}mJS!3iXTJ|!~P zd=bAnHQ{AH#e_b=OlPBSqX=VPZaatRp^C(fF2548OIyyZ$Xs3{wd=SoVaY&x%Jw1Z z+kBzn%>7DOCO5Ld-tS$D>=03GQ0TyREDd*m7;V17XM3BfR0XsGqj#jiGlX|S*}W8HCF zU|yy%#*gVKU0ob>=s10r$c~x}5^^q;nT%%=YTmN(B>!s^Dn-`IfABFgn9B}n;4|xd zR+?(acU@y-0Ue1Z3X5^ZcZJ&e(xh7b#lH3TjC#ahUbr5UU=sv1GneVHl%seR>8{^^ z%lKJg30eh081p_y2Pj&dpsZBdK@Zu|vIh26^qiQLe4w1uYb$x)qWMq9XhZe!GmncdjLjB31>=qosXq=f`fwIK4hvUJ&3Ht0-*;4gk zw2WOX^RlY@Rv2-C;SE+8Y5PXSjQ>a9ey4ttGxr-3_IXSOKThT`mz`iwl{e5K?wt>@ zA_M`6EM|5z=6^oU6~@?K$`kLschJB0yF}r7gCHp%h8ly>M|VZ?gsgdw%8!r|+!{QK zbG$sn$e!!)&}7(M=MOI6L;I2?)xcEXRYV`?mH)H|0xbKrXTveRfkVy2vvU3sNkg> zovjk7lRjhGvyH|6H0S3ugpl;f$on`fIur!ORf&z?+*4+>qdyyH#RfL)jO+C6jfsx~ z5>DDGpL7Jv^&t!KL5s; zTezCJ*f_h{I5~o(ZCsu0O}t4cl70b~2>!d|os)xug`*n?Xl~G$IXQY*xVTxE(|Fpr zS<_g%xjDNEb8%VtSlL)|nmIXed0Ln{bGSKiI6GSD|2XUiz61Z?cW5|Sx_O$oSb+Z5 zmEXH-bpTcfl9!g!^!jt~=;>y#;7xNEAd;!s_)QA*x`VQLAa;*Ygem+e`G@&JP5G(d z@~Of7^496_58dOH#@ZD<&!hRajeE=NzT`tT#ScR@1<4HV`gh}PlA%wq>_lrdFnLnt08(HbN8mj?=yw#+RqOqf6i9nVfl;GA~yu$tUnp8-Z!fa zwFWrfD>grdcRd;!-A6C>9nPIyk=QmKAG@qRfep*f#@}-=fYE8g@WAN)$fU(I|6b6B z*}gz!{qLK(>7l`x|Gl7i`{VbJ|9p&BD)fKw((bxhEFT0~b9Jnr^NYsC;j^mchfd`) zaPUwl9KOYIeW!bH#&rd6nB5D^wJjIb$(d`|3V75l-U<-s@*HXvw>!LdEBPpc@}EW0 zYPxc(N&R6HAxacLwoEV&6=y8$VTeebGnHd-$`+%d4R zd1BA1dQ8>LJKFzg`p<34Qe|YcL!TN5ZGN(-79rG<_FNut9*#Nkfl2ds-I39@|R%NL#k!vjRhFAwxTs{ONRt{u|9 zQSbEZ%K3AVLCg$EH_~M%l^{^&OVBt;5B2%Mk-L;2VJ~aa=l>Xfht=eJ$hf;A8xBZP zawtuFUO*;4msHbW-8B>EfGA>SXRgP|&a_O80+E zHbMe;{meITiFrCIT!z8_<+k8Y)Ky&eAvH{cIV}ZD+N1+(GMw?Xk6I_^dUkkOiJw_t;@R=zzq_rJJH<^r6wlt|BR ziXSA&0<+X$%Y=YV!kANr-o3NpOS55lkB2_z?!VeFk``A5Va3)oC#@|`dq(K+&Z_4} z0r6%arD33fUdw^Tm5@MX<_hS!wW`0&c{l%}aUC7K?+-UK%A0WQl;=(R@sOz)*;GdlGCZ`2CR*Wcfrd=8WC#erWmY2MMfY zV5pQ6R6@AxO%PWsQ;uGk2hr~X~?2`Ey6@N)`7GTyG7ysh`DB0JQFMBE|m<| z>ljneM{zPD8Mzx5n7|C*E>w}~^B{2-p#pgt}*G8W4fh9Mv^2Z zh@6~!?V*avPhOPxEjCq@G)^dAz)|*)H(Anq@j%-{Kcl0$?wBLm>)eX3&@9S1-zt zStaE!o9)UGiXln11v5mxoXH>rP&zXRJ#uCY8GGH@&t*5Q1HJWP#Rk@`qiDE$(paJI zjgcAs!}3X0UiTKRv|BbQvc%0kR9ac)WL@lJ;~=IDY!~r}i<2Pd^}cuctJR*Bl&yd^ zj%k@%ovU@n6!Tw>7S*huiq@<~-sm#e@ z-Q3tyZuX1Z$}%h)F?f1u>Zypst%ahXpr)OXUGHWJkNl=3V&)RZ3**L`5I9#A^X&yP^XtJs(TYUJT6R!Znj}ipkfXNSJ%Kl~Lox5Nl%P5YKNagCV6JoX&sSshJ z4AlTzUBDQIH#iSTsgU^^`*jAkpZ=dm7Y3>V%+$%ri7wvS!9l)gpBeVwdNf255KaH{ zH?w4}&VhvM_GVO~6yEE^l86Y@#aQl99)(71mQCO>tr>yWWI00KTWpg)!ZFfgx_+_S z#%Fbx?7Kn@%%`c-LGx>wUZ%@ zf5M7N%j3tQ!N_@P?A)Upr#kQayDu)>=!lQ^_9&^TsYyxI6bdFwbolG1d!+v*#SoJ@ zuCQbXV%XG(pi&=3fryAO%t*2N`Voe zNf0H?$Y;pBNl?Ah;&((=sn?m>XMx*dJ$hKVeo6a~G}c!=IWTQBhGY zjhZh0j-+gQ`mT)zxoQS`ymbF8ZUVpo>!_FD^IJstaQz`$`9mq^ofIEy^%s*9NQt13 z(AIsMP3mrwaQWr$&EF6bynp+I^$!#&{$S1T;>mImurwu-_}sSvJQnQV;7Rf0+Za}x z+Pb)uC`SG(f8hY2$Yqd3SJ%@m+!7r9{TrEg(VC- z0G9aUXf6mV9wAyC! z!LV80ztJZZ_I-mMV*R>iIviX)`!drVk5mBAqgRrKoh>;{S|!KyL)r< zOVjpzd*oEduO#}-I~G_HBedhz%FFBOgG9}eD?8RJ&Fb;CexK1hjMdw&cFvz~NSdE! z5T9%2bGT5Pl^vm(BCTVUE;Swd=ca+i(|@k#z%zQjKW6u8X8}Al^XJyPd@$5Z6Hm_+ zGI&A)`wdCi^7+2mD3fiI9`k9jo*)W&Ls0@*jzvIB?D1ks^&0nr%Uu^QorjnAY+6GV z2M5RI7b*@HLlP8>*^*8hR)O#jNX~IEz^HGa-*ot!rY#Ua-=Rk?g4KG(%dP zDd+yH*Zw>i^?EJ}EOYKj5RfGE#NM6IFGV7|b@+zuuL`@x`dOc|Kji|`%f7Z=_CF7N zvC(xmj|$Oq_bL7bEl;_-curj3Mgb{{5_0O)nUa^!LAX1qK?D$GA&;Koi?fI*N?fvhj6fs zz1!b9?bX!G;o(cl-c+E%4%~Jz%}o@Xbq86$C;is2>H}AO8VbCve4cw25aOZb$qK=1 z5*FMi0oN&NA2n#>!vdS^Yb@1f*>^+vx!s_Qo)7d&acvRw6JgBF3~@VMr%3Z zQ`hGyMoJ8cmF>4O{_naAl{3W8cU9*Y_Q{wW7qk_GMjKPDDy~iq98{Fcj-(?%r1KkMqkRr|MG*0T-_I8(Sb_b1&!-+xwwvxLHr_d?6|MDD6}_>)w= z#>Q-8-Oh_9L*f<0?soFU#Ki8ekH?f$<6>f|#C$H&aztI9Td!B$=M48}D-`D#DjD;^ zR@X0y#RUrB-bE@|e_DOV()b>)mut&PN_O{b@BX0MiJ&9HV;~9$rM~x2gFqi2pXaB? zsi`zHYQN%=lGFUGm>DUR>FRa!FGy1hLT6$eiNz>MaQDDgY45s;{Hpr zv0l=he(c7-5`ZsURqs({L9;nNtiF4UWG z=YbH%uxeAJ@WLKJTw~{mp%hRl#taFU&Cez$ZBI|9r>AXg!B0PMmITW-HXUe#Qpd15$Vf+{{boEq>xoq%tKsO`BMClbet%uASkX z!?fh_$t%2!0m5*LUwa8*hH8GRb9#NI>fQVZG^wklFHGpi>JKj|-nhBB-LDL9~T$* za1m|Gg(o8`RyrR@P=;BCz8=Tk$3WZ!z+Zv9cGe>`2Bzx-1+`0g7*@SDMsOuWJ^pSE z)a}oSJ&bqHN`_2frDPEeF4$NxV~7k`v!68CM6>S zLeiQwX%v3G<-(Gt`FgV@^sit%4SI;A>!{Kc_LS^-6a>^! zgUvu|%>LM8{Lpi{(TC7+DON(5FB&fg&^hezU#q_Rxth^yBKG(HOj z8Z?%p17+ToRg)o-UfD%;Z?LDh1wol_E0=2qkLbc|5&i>V%LO(7>;&uwb}!itB8U*d z|7rQI>YXMYGR&pX2dfA2$;GyY(=cVkl50hr_O$`^-QPa0`Ll#L%Xo!|=r6-=(doK% zkNK;Q*Zw(>WFo`~)}f zrn{|es{NG0cwcXZXYSYMA^tvtRRRT}YGw->fpOOosN(1M2dBFSX>HBU*Ago(t8^P#roK_yFpSlBe+D#rsjvr0k+%9}opVzkaJzkd~CiPfTMMR+} zq?tGyBWHH(?ChX8)jh8vl(@fhogxXm9V!#n3r9@Fz$U&j%AO>)+L&4ht_@u%vCma| zIO)bkT0N$wyrDLJZc3Qs##Z}t{n$t4dvG22Y|LRDDTD&b1EB}Kk4k;qVZn(BG9v>{ z>>5StAz0wW$(-Fj^v_%K6K};hwc0F>=r>dvw>uvGsZ)=}3aZ&}eV9~?o{k^8Y&C8C z!6IKc#VwnsYKLBQ94qq|ZsJoFB=XB1+bqR^XEC8h_qqb|AF)o5XNqKSjDt6 zeHc2+V+Zc>hY#7u-bUiapWmASI?vthx?pmeC+%hkMs#=T=D|WOaNZfoP&_?6HmvdH zHm0WL_`{QTPewC|Q!pN{Mj9yi*sfg2D&88(Cxec2PO4pOP9sHw9m)qL3)*iA#_IvA zvuJ6j1U#Jfc^>|`t<4d;?!w|aSZTGgWj()df4;YyD^K%3Y&lxD3!*_UfF$o)2Vv#+ zS7|8EgyOF)20mZQ#!_NPtFbmo;URa$392;eybcqNotmAUU0Dg(FK;OP8+Cu&9C-Oh zpxN`7ic*v!y0>CGD4R`vX21hs&Rqx&o&0C=lSo-KG_?D>`=Xi!O*&ogQ3|J)m*~|d zj}hhi4G5qjJh}2Z(*t}PaU7qD?dk*<_KfG=x9#31Yf=rHy2)Do@Nm4s0T^DrLq_4k zY?JA7le_1#`yBXM9f8oWQj?t(v084r+GAl5Jp2F&8l#{D}fJZ9M0tth-jhaFLppZu$|!xVS7IG=5OK8Y2;g_Tv5 zHqURldr!p1xJS%+(-auH0iYlA^()WltHo{9%F$|MDc~>>TM|2wVu>c}(q{*ewqvZ9 z(sUEnum5^eXT;4e}X<{e(EK)is2g-g15k6TMYsG#w#BChyOymsae&3 zKO^v2;&K1kMr-j)0=2YK_uLnPy`hftmPg+!DnaKC_vJdfxeBA!Dv6bOya5VA=71swjn>BkKWU#x$h$rvhWkBfdOC^cSV0T-_ul+4B#C|> zA_m)Eb{Mw9!ijN_F$S5r|}>*rvX#)BpcSco~Vc+!AQ@_9df z+`*qZyODH(^CN`u zW^!tKj5e*7BjZs%2adb6oR5kC1c6sv-1}^E;CeMM@OY&af=O$hs|FBzGlIl(3{9FC zt)Fchu_DIahX5`=O^c-VAi@LXB7kHfWc^piENi7`l&0pZ9LpW_V}`M>&~Z%}6U71^ zy!go%fKY@^ZYr@51%T>i^>!hR&@s412_bxme}o2IN6U*lYhcSIytlU(5UTQ|0I4?Y z+lfqsi5eyiXb^@Ek~UK>fy$><7NCcuxt+4!kRG>8y5$7mO=?0K1k4Nw@Fi!O7S&*p$R^i*-T+D53a1N5}~d!uRsiO z>HQMkHN9!>>)SlpWWed}>4_w{|78{k2(9b2GoX2PYutWQ##<*xJG*oL=QOKqjF7e5NGia~vFP#SygLW- zL89vvOW(u1g1zp`8&yS0(y-BQiN!LYx3)8yaZI{O2;NS|avH~@|eE`rm z8gQ=q?3V+xK?Rir;|YHHyU8~u>Uy!W?kBFw;(fjBDIzS~b~j_Z+~(hG;cQ#At8`RS zl>MhY@Oh!OJKj2sstB+}n_3m%Xjvkj9zfiCxG%d5aYVkL@&iWi?~xxj!dH~{weRtv z>AJNHewC)xHyhN?o8!5FZ9>6vy~QS}`*W3xjjnS*u7s!I69mLU1^FP zdGfqvpc=r2s+3|ruXDI!Vz47~1bS~Onk7zu6R3hZYRAMEmzExGHsatejg^6q`)IgS zRSx$Jf%gr^%XI{x0w@$h`x_v5RZgpIN{k!#0ismxTgm!It=C3APqys0enxh$qNK&f z^7&0&K{OTP^->lybAdoL8LVLs=kOT>o-8O6GX}YiE3arJ;Jb+5ufzY{6O^n5V(#$V6fvF17eW2NEt&z$YfEkNw~xeX*}dq}H0H{0*~g(kZ7{hKiaUaJTF0u}q(oIW~XFpug`%QE^=_|78SG9F@u(r0g z%H!d+wKdQ7`#8^)01A+KVa@8p_S;pjbDVOwR5Vg5ioq?H%C@_`dWW}3Fq*jwAZc3dAJziOd>JEDqFy`QxDqXo zS0Zf}FfMnupIL$det-r5M!x(yf-CmGozI;#aIo0w3(ySx)jLMG4C%T~@@;eby)VUE zM{JGn=Mv#iOO)ytDwQ3R@Ve>OE@O=NvmX^1mbk z0@P6CdE4p91GtOcKBVL!pZf7voqJD{fZ;lPpGo3rK;ro#2T;b?iKBZ=A0Hn9NJB1Ur-93Jl!AH)%hE-c;{UYuRe`Dy8_(Snm>IE z4m2@nqh=f%JMS;&Zwa0#>)X3zBHY>xY(uRFV=x1 z8i*0~v2$!kYgjjPkNZ%}B&50+O8JUxWXmPS72N;(x4E679UgL`hKAMMUBNWp{sO>G zw5dl0G})6y+bLsfI8qc!G=(Xfjai9{YIS97fEI39CA{|4unUm)Q`k1o(@I!|YQpeJ z+?v|jo&?DY89VC28$ zKIaZN|E!xkW-daIx@~$P+-8W5eC>A>LEz(DpfZMTZTiGh znE|j%Tx=|rNC3N1gn_z-h6d0;bF;kbkEf^mrm#O}-0q@{?;a`)uXU0yivq8TWM5E|QBe(&C{9>!E?DLGA3t7$*6J6Xd^gwonwpxj zZ+H2|B%YUUZoD{C^lM9TK3bZ{mJ{W>x!8Vz3<|2 z{tkC$X6BWM#{rNXDgjLyAbsoJE7VV&&up2rBlU-?kISpgkM)Py)TxC$w~fX==Ob@@ ze0>3f_xAO{G(paHZcf5kdbO4~Ae~;nF}9M|d^*)S%t;C{)!=G1)|BE1x27WispvI0 z8v6SB0>|^QyRRR2O9nu(9*+Z`f%eA1!9mM$Fd8dLYGx*45TKv^>uh9ZWPYMiKtKH( z%~he_a1?lJ{KRdjT~tR*H?F5&NWlE6upeMPPa~^O9JJ;DcsIWPx;kI=X``c~;~(tG zIyw~du30>PP_o<;0ifDMzHDO2bm3PDGI%yz*^2kkYV=oyzHts_3Z#=6h1MF}$?yjGasKU9Q^6V1>3v5hIiQuOHV3skXpI#84Q>_poqmV}^q# zJ{{cMH36j-WNmMsnVRlqs4bTZwFRgR-Dv*_aGoM%2#faTtb*M*zcI6tm5W}mg9sF# z1%{WW>4wzYMB60BtCZs@&*v-p79? zxaqnMKSs;s8>)enOs5O%gJ>vUXO38oWs8aM2pxGFd0r2UYGtiovCx|6G5(b%%^@S2 zT$&UW7Z(@wsy}E75O>lgw)#=gHo7K5gQ$T1)=S(NI44O5;2|Z(L%N$T9>pyUNf=dLN*xMhh1;1NAiLmA`1QixB78Nmagz87wq!pxcv6bgM zty}{Mv(aD_{^kCfIz6@jV70ydcsNb&H6*8^Otomq<>O)mDmUr*GCsW|x=9Qoz+2V( zfPbvF?fpE!zFQWb8{dJWn@LF-{;XRk>~xu$I8USM6p)?jC%Xl3njHMYyYEjN}k==j0OO! z#>K@^3%PxR_}r`oR2uxb-BrcsKu^uyZ@)Mj0?;Ywi<+1~lK*ilEvI8*c3Fn6AYDb`mubV(KPO$ZWHTxeK!|hE!o+f0Ny~1+J?PGe zdZExSJcFcW$fO&3cB{DAerE$?{!dsyoA8D6kVC^Uz{eXKckfr9J9npA`&2WX4Yd&$ zYHWsjFYW3O`}qA}85H|DHwRo>E{!gY524f#Ys;RiBP2rr9l}B z=i!84nFBUAhrBREAXb7L7Ay9JUlsu!h%)2bnWmn}hOEfJbK5ceyaJ5{VO}iaU$Wq^)r35zS=4QKI zl{-E9(+J4jUU9AHH<*~+f0;cAC8<_UoKGW1;D5{##peQ~@{YXuPe)w@{@ z00RZC>9iOy>qU@NJD^H4^YXqq9gfRp=uqm`#0MHQ$zvqOKD%2QVnqmYhK9nX8BUbK zEuP0KZiah6Q4Tz)?wj0&9!F(oXFs1YJquCV&&_!@)w*iY5sL-lJ^!tzwk`LsSz1K; zcM7I{X)I^UvEs2O2I$2DAoH5pXu5(O^>VlOs^2!JjA7dHzf< zy@p|je9z6L&5WFjk~V0dylLBSGI>h`Gf(|?`q;tkh1U=0N+l&MGB_m#5HQP@pvr6f zuQGYraNtZ2+0Q|k%e*WKNU3`>f`?ZW$PHp+(d0afZyr0ezj$Sne3j{X$rYo%SPv); zkP^*H@i%YYu)+k3G}1~;b9I;rLEm+l5rZsr>koC3dr0Y_+=I|r_uVD{Aq;C40%7A^ z;3Uv+U|BomxWEu5oj2hn`l8lMv1`Q`gtvlT88ai~#1nbTgYCFjZ*p>UlnCJ9{W+)N zVNHEz!nju=SA%So5e<3DQyv`EZxNo!2{U0n{WkA$f*B_7$1q7rd1!<7(k{qOlE-C3w=}eoYhhAg~LYX)is`HZ$r5)V&8k}JP z{{As9?{P-jXObj5GS^bR;9dpmq4SIQja#>P89zmLZ>K+toySyTVa> za$jyIDf~8`g z!kQ{C4)Sll78e5`$D@I_uYMbJXCb>S4>O+-6;iRK_bAjHZHj+i*#`ragv7f+^!#UY zE31fatzO!&^=GW9r;|75R`RDyF99_*cz&*}llVsdXiQ)J=4_BLbgwXEXL@{M;!i?* zPDt1Dib%2|N;uRWa*DJ%N8&RDeQ9iz>~)qn`-1mM%^flX!B)WQ5DF~R)rFd;Q{j9A zMC-m<#GxnrZS(Vpq0v!cAt4!2M9}(L23R8CS)j|Uu3V%n@ga@_BwsIfnp<~$2adOA z8$9NlszT2ao-)HnDmT;nkg>6`03Ark5BcW}NrB{_^_SkvMqGG;$9)XS4}f_qw0^-* zd-|PR>H}a)&dksMl1%O@nxB8DlP`DQ!4kBxmX_!G`R9H;N8nbkm*oNS4sWI{b?tJ|CalEoE7-iS}HpWo;fQ;kqJI`G_%b0dM1}~P85h!dvnvI`< zJ+k24ovZZ9>nVrQD|m;$B@27u<=^4r)?LqquV<-oOmcyLhrZpy2lG@?dip?RB`E%W zzp@Pd0%)XI$P!fTsyUIhCcK#?SP-y^F%U8gbqIRX_i6i|+5x{!p+v+&SXl4gZR;(U zEe3-mwu9zrFlQYXwyr8b8B1nnzLg=U# z7*2QB*U6RM!+HSYP-#-#AwsGOlu681oKDE_-IMTU6W*4+4oxsX4}WM0{*{Hl8}avU z{+IbXwUix8bApL!54ABHyu_Ik#hWH1B&=n`X`v)X_vVo*(3!|(+=8&If`cP3cmtK~ z*}4vPVg`&%sUxg`qzvnQtb!B!3-QHJVTPxozGeD8-ynXZ3Rn_YOUl^IXMZZP#oVR* zH^%UHTTdotJspMa%AXI*gBjtX?H^BD@Wpn+heqRMW2mrpbc&5n3FzLdtB3vPdn{TF zxb1~9L0RTT@b$e`WlOhrw`kLb0+AJI=aAWRe}CDt)il6QEam1}g4r%Dlb>1>$VzhP z#pi1)Dl3!7|1vS0U!SyP^gPkz>fnC{K&qK5K#`A(>H1v#`Qrf1gmY6ToY8S&aVc_B z-A*J|I&EdrUMz)!AszN|H@-722`9?U#RVf2!K{$0ixc$0nfmu>0OAhhHcLs(&_;b) z1=B1&i!vuCN(q`+cn zsRjz9bcfC2*!muFaQIF;SNMz2)qwi=Mi8n(My5pCX)-*)5yP1dO}9rV_td)&1Qi0+ z3%_Ik-69KqL?1iy8Moo&$cP(_!$3^>L1kzvgQWqt7gK=4EX7D;^MOe zxtDU(ic(EPs4I2Ehvdl?#&2s<>?Fam9?{_-yzf&g=s{`h!1{}}p;^}2|D^bXR#?LL zUKa&iZ&aj9g{L>eQd~N>Lc7#LuvK#Fui@3te%80dZjVTllNN(#D|TBiJR%})mOIt7 zbEW*x{TBE3bPILg<>dwBsPj^XfCCAvS`Fmk`e<0Jjn}|)putE`*=^ok@M5TUpWSzQ zkSHP}Fl0u^(r4db!s}cC3JJ<0z_OtZxmMY{+q5=eUIzmOrJsDd{{CsN5DLZ|{XUuu zN?5441UkiA3Wck!H`f=-IHSwaHSx+X$b^K1KEll!eZab5xPSi)ynLpFu90;Sv`j`{ z%jxfgkmIchGtap%%ir*s&Vk13BixSx&0}@8mXvfGrRZu>=R$>c14O->8<2S(z4k{! zSX2LQKcDknTEgmFO1Bpk6`hZ`xw3&%sE2J$d3Sri?ODBbM@E4FEay1cL47FAoZAN- zJc=hTYisL$ItgAZmC9e3PW#JWp>(=dyp`025MySrUh9B0T3qW{F=YQqpG~WhD@LI* zTYta%k)(Nb5g&y&kh6pz7DojCuE_^gzI7L%U)P7z`3!QQKM%eFJaQD^vk}|l#%!rS z4-XH69El_G_8xE|F;(7kwyCbB#&x#jXjZHuVs%(8KjQkVGwu$c!N4?nWf>OcvOVJu zx0Fgrg9uyse(8U}!`QR+?2lHqSj>8{W8Rqt^Z5A@y-+4fd?C;d3{{Y!>FMdIDVj>9m5b}| z-=lXmQDZ9Rk`b5Z3mW&>p zsHGhCv>pR)6|Rc@5gx?bJbXyHJ-JdRNoNY+q5IZ+stus*X=^@fgn)aYk0pQeV`G!P zmMG(%0UV~Jz=e80W^hYk+W&s{u$gqy=+F@J`#qV#dUvq5DYpOqwa&qB^`wb4;AWV; zzjNEE44(Jy9kg^aRDLOu7OyxlDJcmI6V=s^kZ|WLJn@ctGYLtPMvL2}+ht=U;u82C z?@C~=eV=ktob_-8Z*fLrr93k5YbwxS;ZAOT{utfVg+>MN&O!!E#F0LT#P$)o#y>Ktjyia{%H z#N}=p9|lPQ0B&VJ5Q$L5V(24~@N{x2fRo(5ovw*gW+nG#sPNa(@jU6#2UMpRDaD&< z#0#hvL_uTCgaq;)yYF++@RaoK*T-CXP>kRX^2vX*Uhvo)?9M5bhxvn5md)pt%{%9q z1tKW8*LJgeQ(alf2q`JysGI0{=7S!{&A*s)%+97?+5ZA}0q3;z^PRHjKtO3#RPX@6 z42a3ubsG!1Os{m13Pg1_+p&Mn|177b+7^<%;h6YnT5;4*luSx`q!gHzW(R14kjeRc zb#~S>Y(N~7f9NG#95Faq6Np&L4?pchq53riS}y^@Q)SlpxwyE(zw5O!YrSn37|ycn zssl|j-M1@p?*91eJ9x#L{R?n;fPb7FcG_ya*g3%VM&~|n1$?Bw{)JV@c3nfn{!^h$ zI1Dgm^l(uT5t+lG;$->jFK^AbQlfrpId<;2FZUj@PbN>5V9QEL{Tmsy3S4#^bf5mb z20m_c%j|CQ!~DSC@yExt4*PNAp$B z!~D=cnfZq&CpW()Z?ecbI#Is~tfuS#)#%8W=4&_Gi&MOzPy*$vgIGm`dKBva9BBJW zJOd_3|45{IA+x-qvH-T8BQrfeI(o2sJ>Qm_>pct5?Ag4)-}_(`wo8^h-u9F~x;z+^ z-T%HaE^a00ph&<+iID-l`E>6tL>Jnbr=5{=HYV~5U}J#8v3dE!dw>*w(oAvuAF~;c z(kmO9bbC<4f}WyG{M@81;Fi$LL3C+pskxaMUmrK(b-Mj62$>4Ouo<_?`z(!8nw;j! z&7h~gzP?TTmYkGeVA^$`6c>>oSJOaB0D2EJOv^O%KLS}(IVn|T20#$s{|E8WF8FZZ;GNMzguk(o#ZY@BhhE8E{i2sBqcy zGA4K1r3cv{4Ns2#$t3^!@i5V{kyxMk2y}c_5m!fEUYax)G5dseervr5Fpn*S9%%9L zRouUSKixjsG7nM!*qjP?6*;TZb)ASAKzoE;96O2pz6k#=7RI7AFhG)XB;$ECGY)#L zP|#VOUkpC0F`f0EsIa=Z#%ChAeGEpQ#T(C?efccBNxI0$Fa_RV&x1TSDoq*=qnI;E2^tdMfjjA2El(NdW%F0Tx;I4oIO_M|aKr&Mj*JC6p zY2rLKR`}`)ULD66s_ROCukAHpRI!n$Toj|U40u=#j_?Ar|;rPzMxwpwNb#F01Xd-GuiorGEo6tkpjm zz~_bs9B&PQ|7q_h+Z}n(6+?Rf#aU?Zk2@iJAqbtDEcfqc$s5-zg>Jj!AEfjPK^NPp zsi`|VJACG1m0N$OfD7UD^g9NwzS?KHG1vZ9zbyzQC#szo9UTQruiU5txK-}G@B(_j z%YPYdUwzS)&x8soL0to#1(Rg>rO7+|#{`ARiM+KZ$8W@l6)S!P>$%pr@SgnRt#3RQ z-^x?w+TPh=q@f|m&x9g5#j%*5_;Yd*vO~*4St9Bqm_hq4CidfQ{D#VezhQd22R5R;e3IQmJj8AxKl`SE4oaadT| zZV*g^oB)N5q-1$tUA+*n1&*MX^kAR_v)PE5xR_XgpPxmF9$}{#%i8L9W$=5A7I+Yk zz1}C^qEo;h0J>pi8G!r;&3nG@M<3*&k+AhZ8M*|Okl^5%nc2yTr^5M<#pEQ9G9u10 zlzu)FnjRUM8?S3Zp-`2T{EW$ue#i`ple~O{4Jcer`)D$4mXvdh3E|tFR(Vmxu$lbf z(94F!aJ2`&pXfN=VUz)oezWD|xY<=HVBYE%Yq#GtETqHS7sH7i`{So>&g5^HK%+|< z@;z+Fx2?9x^^J2DxRcz@hZ!pTNz(!*w|>6*aZ) ziHcdU_CPt%qm6-aHYQesKqCkW3-Y?lwon%*(u6nbl}hLMISrd%0AI#l>g!L2Qn`a& zMURBi+`r5{G={nh!j4~nda@g~%Oow`%ouUG@1-n(p0j?xdM?m*If7|5^U#eRXW%HA ztG2^O|5*J#21u*pt?8BYJ{SsI*Hy&xfC`+*w1>e9AvagwzJ1%@_g;M7-QCUC2PT5; zh*SIs1p+FVM2V=JC1;{ zAij^rC+>hXQRj&~P$@s34?_c)Lgxwrq|Buhbge)df&`PCuM0O1@I$Zye1l52S4v7M z{NyA&WPU?*9PNIDmd)UKK=BBd83{ZBfQyetdOTCE$js0k^gI6H zM3t4}>@k3qfkowHd-k5T`DNR6)y*#0+w1FYtf?l=0q(4+f?AKc_@b>zK2Lufk;=*c z^kQOSt?`d_h_&;$*@X?pTqEcx`AE0%t68(x*i4LdV}@h(uZOylA;-LS9}T$2MaAC5 zlGa2)1NHNT`4JR(p0i?w5B9D$+>P+Lb3ngp-~*rrFR!74uO%gaj*sj8%ECiKWh_#j z5&${F(A-?klM>!OtjJF`Mx3iLWYBm8oTR%w4yBr1Fz#HJ51vD2#9VNLE*MuXR5Ix< z0eNL7K=I<#RH0q}fKE&w@Dw@5ehcw=Q>va65~Eaj;^$ed%iCP}7Fa@)PQ^*=RAyh-Q@N+xTQYp0BvP5= zD}_0auEs0>_FG&B-S-&|cWa8U_!tlUsBl;g9J|7Qs>Gm14?0zFCmZpU8L;7Xpat8d z2EXawX+VO;h^rbmqm8%lZ$F@jH!L?2o0wc9>Qa9|hod-kdK^Pi z`WF}SX!dUjC7eh{eTWYJHhSpO!MlmxRo<`qPf_@G^~;;M;Svb!Mp#3xZ8c@QatApV zVu^1qkOs%TrqS3cY%TwKUCqpO@Bsc62H4fNS-Nxys(=kVzz3*5A0MgBT1e~XCsL0P zwDk1!w6vmHK>ggHTBOI0`1THaFVxfTS!iG&Cq;avX$=Q;b>|8aE=EI`K>mZf6^{!( zA!R$T#l`aA@^ADwg&YMQw+&K{$S=SZycAGvdL00ljS6I?J8qr*;{DjpP8MRpQC#!@ za;u`yBB1kk3|r;jEy6#?Kslw4daIVPrDPy#z%5-MyfPNeK0Plo1rH*4xEAdEbI zOVo{|)I@@{Zd*7sHpfzgl;d*&j!nRe@i}sPetPk!OY(P6Zr(wGIsY`%8Rrrol5G zCX{m^PT}p*xRx%xd#Nmy3q_6?9uWqM5!vD#Z!Ml&`VKySo?q+UY;g>*XJ6j~pb`P) z9-PeZw>egYr5<0tHC~(g?^rh8c9>jm{v`L4Vdc@E@M2%$G-2je0x%&w4co2*C7vke z1&o`aCv4WeVS6shgbuV*abQ-gDJicB7AWiWdU{4k7WTBjXU*GIT*&PX zZ-tqwJa4_3G}9m6hj`o>#;*60%$mvmYi!Q&xic;5`NIsOhzz>8yI_y<2X84@Hu2|3 zYeFRvnRNGQPr|IA8eCL{#FZJ)AdKHzL;^T}p{1kCf^R=%Om4MW6r~o{mC01rVQbOa z<@j>h(`ZM><0?5CNf3Hk@*v?8NZV=?Zgz@zFB*+Nbn9m}^s(Ni*9y`WWx{=&yTgO% z1Jf{Q3jN(Key?}Df?=jYZnJbmrKSc8IMp!3f1S%XRq~`7u|dZ@7gC!StOUwrI8fB5 z$CliPO1~qN9DA7#sj7q2*(s}+r8q&fy=k@jgRi)i@G&#}-aj?(d#J+PqKMj`yrhA^ z4WJrI1jr{`Tt<7U-rjZ0t%f&ko(A;&$8|e7UW=t=wc9dEd_ykQcnwZs1gqyh41<3W|;*YWc z3(jnV*?Pik_<2z|e&!YEOJT(3D&7o6lQv zTN*o>8u8NA@$#gq>;8bl#`StF1~l!-$`rXsVJ;9s60wWZ{_AKq!ww&Au1>?Vzla}S z^p)TfX3nD`2WO#VICyA)7T7F)+E7Q8dGxqFr`x`Wf{nyWB!nr{;Xw)6`d3L9G+k1J zwx%9;geyaozcuokARcQ*azcKzXD}9mB2EFFYUDr>#SPz8;--DqRazEam=Vk{5ON89 z$Z0P^7Uldd2Ub8SVPa&))x}lu6)SgJ?Rn6uOHrL;8J2O$G7_{#I^*Q%YdqDT-3-A()2cAAd`Xe_lFBfyk%Ib))O-r=JW>)cvZ`CNkqO_CQ;`{T?;WTji zCPE8q2ct#k?0M0)J|IRQ?GlcVC<5ywZF{OH0^4De%=9?5SfG%gu>;WYU~U{?C&l0a zz?FEZZQwKYma>L3wz?2FMOe6aRCQRN2QOB7 zko}adCCUe3!!ktA6^jutX!!^i`f8nM*3=}|KrPm9puY#nly~sJe5v}KE7Q0N`z67V zmF(2I&_#Ioeh+&@)k3uw$DtQ%bY`l7n_RP2*>#Bb)x3$ND@JSTTSdT!^fC#W;)$)^ zy!r7BcbJ^z@O2chZ*#3(gf|-CMFz?+Cj1~^aoM@9g2Orm0Xj~mjwE2&g0c-)J$>Wf zpo+;PY1wSEiPq)QHv=3-`^o*;l0@z;9&d70)b7Qy3j#ux++AEfwT92F+e^~P9F3#o zgyga-vEf&p{uA+qKafLJlq#^7Et)Kg7zEFlVM7N*20KLAI|@fMEkDe#Be>I{7~(Hk z%}!4$xv2PJx{_M81MXVOxbrp~Ye-d!BtZ;hd?zpYQ`dabT4_SAT!-z}lhu)yy4SSH3Ku&6;zf zZobaAI3`z`l*f33u$S9+fNqFCg9{m>#6l6T3-zvD?;`Q2t#C32BfAAZeQaS}R)l<| zVCvgJH2nrWO+fR1@i8L`fb4by(2ci z-L(`uA(j0L%9S20k9nB2g6}|t2?$eQOMM0F6#<0!gXk34Oe)A4DgPspbeBUFuIn`& zA1^@N`YB~l(*k-IL@Jnh2o{4Bi@h)_XK)g7L!ArW+n9IxXxtq#)0Eg%bFOgcp4Jch zh4+D%O$oYqCsh8F;2U1?Sp7Dv5y-fbMyZ%VEJGhL27ts;B4DS40Csu)^X5hC1b9@0 zHr7S8KSw**dt z%}|Vq<>C_te$zwmQ8Rs2LKa0bpkXk~Er$RXkx!%F^{61~pB|YKw_!IeTeLb7JL|Biih_G{3i>bW!YE(6%DiDfq#vjO17*I^h(Afw zK*BFiiP;ciP%DSIpsBe@IGK|CfEVTtW50FE`74qEl;D3MTuzPueMkv-I&@eC7X0=# z(}6MrjnxJfE~=TVbqKvP0)oPHdlgpgYJ)O}^)r#mlPd{__B7G0AC|`GFqRaQrpm8_ET%?fbPkT(@pkRxRw$HA) zpl$qza|^1;ER&ju+cD@+!j|6hxgwX7^spDbV4Lp=8Xti%CHc05t01ebO`QtU9rU2B z`OcyMaoyakvG~ls8Rp-P57y0C^?zVCcnr)V;(sL`KkfwZG!XN}OH?~0daL#{CcCif z>Ol#)up)tK)vlceIVVViEO9oInxMSfxE2M#@9b0j2 zgtY>6dTi`j)*m0wLQF7=ICBy8mnq*0mG=K>y>`9koVr~L30No&fhvA)NY^1aOW1)X zc17a7%Y=ZDVceWZr{2n(;O~72OzC%y*SF%w)h8PJ8dm>abY0ALlxz7!rF@@##3Nl(hxCh>6;Pdw_0gH&2(?=!f;`?6Vz^4I*Fj$i>7>GgevL<&e%qLIFSD55(0}!=$ES00BIfcJ)D?&cWoQ=-m6Fduul*#@kG$}iCdHPw zK<CU|*`t885XVGL8{*4xSIXO9bdHj?_2zkjQUylNOtO`FG zG5G7QUgFMgJLB^o2-wY>kq=nFMEwI8-B50$Vimo+!)46fmBHG`iaTh0P4b7>5GapC zgk{Etw!Lz$bW$02F{{CsJQ6CfF^~pu3MF;2S6;YJ1`jNrmk?*dpW-LWvQ z3%D}nzetGc6{?vNU;wMViO$ASgGCcvWnx80Q#+x=1u`#SN`O{{ zstlnTEYWb-E9g8F`Dz5Aspy}}BQc3vF13a7lP?b!Vo}zIAzDyU6NXVx|E5^MTM$kp z6eS4I3NR5s67`hSJm zhz98^Qn^W8wh%I$D4U#_^lT_d zQ=2j4k$-xpitpo~EAi~DNuvtOi1fbe-yf|A#)}UA>>(d))ld>OYwIatDd=NEa$tDR zIMmY9q0Ob3W&rM2@NTFU%`$TN&!u>2IGsOuaZ#0F)#9Z< za76mGyBPwMYVOlJs9jnWjX1a}yx<_)`8Z-WTR%TMynGS<{CNY+#m$X3SG&rruI%$Y za5Z484Adbnv`^H9on>xH(j6ir%5oZh@eG@3av==r7JEzKvBT;Bfd%q@WNlTI4m1-~ zi|H*~XIfAy3C1Zgf&{c0pRtgzCygWG9=}rjnA>kP%S(yEtf8}3XLchEBn+%T(Bhz0 zgPvMhS+%BE^+MG!?S(xiCT!>hU}u4V2?;}ONG9C-R@W04UGth;$;{jw%lg?f90KxkSa6iB&e3lzCb#9hh4w=GYT+pbAkiwwEgzdOqO3vZ zXa8WoyyK7Y9C|h3gSXTBVqW154I2ZVmL=*DSuFE`JB zPlOz_Gtuequ4!)JbbEtF38)7lPjqyEt~&$Dj-)Kn$F!wpm>@2vZw|4;X)5_N*CV^( zsGOclNsQ^<$k-SkAfltdMO3-zf)o&D0X-K>v|P9ZhywMKj85^2?1tT2HFNRsNQXhW zN=r(#^W`j!jn(e@rFPF^IyEPoz={f)rn&K%YyN=$f~?FdwgpV~L1!M;G zq9}zd;=4}5wAzy-AO(c|4;u>6GvVVaHJgKJWjAHSL~2#6sXmIbA|fJA6|{f`vIBg1 zxp7qqoG#W|t3D$0#B%7sbvrnLr(3}N%(WuirQ)*B5}eL&CRmPD%Wt}_<(iBp&9}~D z$eI{flqK#~Go3E8weAY|7gl0{h&+0Fpf;|Z0C5 z4A{Z~h=&ByF<^6q-NOefQwg(n{!lw;%M1#Wbb`O}ZPU{f;iZ$-^UT~ld$S3c2F^mm zQJR>|GynI0H&3EoQd-K_2Lp*r5D2VSFtrF!?K;M$8~?(;hmmM9CUnir%z*eo0EGe3 zOlLQy$xe~v^`9mOSGoDW=Z%VHN_+$bnUE|U;Z~peOCPqHJfJXb3l9V7idI&OOH0g1 z1ELiYuaD@a@d+>tl{5N+53wH;OawzSAk=~Fad~-}r!@d|0>SKUIq;ojVj$Ek z-3T4SdQX|d%vq^wFg;jP6T}^CY;Uti17%zEt<8)kUGP$eNO3@JY@uX~O^^M|-$_?E zHjt7A1O)t8*#{ELo*rT z2$vOt27orn!_AF>mR2%g%LHZ6!DxlKscuYe?GTkZSNV;p>@#)?a5h__S}4lFoxdxQ z1V3V8)Hqm6G??&Mr+pAwOV;#)*ztTMYlMDn1P~h_?FQ%-gtTg^s(^iEiNyN94_#7q zc4v;3%d)NhP*+I;CA^RWO52os@|nCtu-&<@(JJn4Za^s7i^c?V*obAbN6$$N)BX4M z_Y$aC{BZ9*0}8P|_8(tj-DZKy+`P`kP%K;4`}cS71Iqw^gRE&_86^J|N|Gz_GbVc9 z3J4{7F3Fe-M6PQGXMq2N2x*&R- zQR+LGH7dDonzRy~4DVY52=W=4v`B~{hHPYfdQBn;wgI%uEiEmIWOGhu?BGHCUBP5| z{`KL{6#2xPpj%XD1m*L`lJ|g?1F5=sy1l89r|0g})Vql$6J8KGh*=Jsr2iGGMr(3J zik-C0TNZ}zpU3PkO&$Rp3TW|wa07^_f$S5IL5bD>o6r1Oz>EkkJ0tONhhOhnzHbYK zk}ec%Z(rXU8yc{^LDgU8PWkU)#Qtx8GZYlTUrVI=G?IY8BrtC$^%AV4+1Il6irQqA zV8Bc%Y3a!-D}iX>W0MKL{6hAEtZ4O(#B*WHqo?`*jc49LqYqH{P7m$n#1?V4;v8>> ztUnmf&BjLPqCiI9L!G>@g+iK`7j4`5-xVT$G}zE60mHLI_fdv}GHWW}=0Lgvat!Dk z*GEhHJc9nWQ!O8U`W8i&bx#S{GC_H3`!>o&6>9?$j~XkbMMdqi{`90h)Ea%3I9=Z| zrh*AQPM2*RY9%6e&$709wP_C0W6h-HGeY#A^5hPpLCkTXu=MYyN*fwYDkSgtSwK%N zrowp69+w%G|2qTnl+W(X&CS_lBQrB*ror}v30K3%#D#QGX!J+=K%ZmLD3txHWYl@^09Y1*e#+)-@hxVvr;~voS%P00>zOfV&)T2Q@;bbw zm54e80rdmSc^qlZ&d#(nG(Zw%(Grg8dj%E?pE1B~z){5@>zU<%9L?Ch*bfXv^%;UE z0$SNv8`Fti1P=mcwh2TXfQAK3c6vWievqP)L$wp+O7G^iTsZXQ`Kvf(tPePqjEt~G|J~gM(TT%Opyv_*-O=c};7DiiV=vS~ z8fN{s1cnKIdhlz=cgOrAKivgc4Cz$|;f(Tm|7$+_y ziZa2HglfU&A<<7na_ae)oo?f{zO&uH{fkMX*rlut9S~OmLNHKp1B&+(|K9FO7js`B z`2#}>T}0B1%+(lJ3|s*OMV58fj@RP#Qw>~*&jbLhsGTu-(gzgIo}NtkIYVX*wYA5G zexRs=t^;BR<9ZoJf=iekX5d;8Kfw$Mgf%a4VlI0R(4OM|hb z2@8i~bf2%sraQuW$9l&mI7J{dhP&qXKa3f@wc_gfz};yuI8?YGj4v&1ROQ(+YNG{N z{asWxy_Ed2NEpB40fe9PHbj^AgYZM`-JSaqGLo`y1*n^i<_@PU+Yn;2g!ToLkYWWO zmTym1$R!FKQ#YjhTZxJB!6#&nCn2FGeK~kG{tg?$49SXZ!-Kq5R}CYgY#&otDsoc0 zg#)2`fTf5O%EWE2O}BX>+MSwr6=QKsh`ufxxQTfh?CQneDs^L6=U>kpO??YfC(nxD zzZDW1qfxhdIU(*YE@s0`wxOYbmj(@2-ukXQkVvle8m8MjBYDal`LU4-G0%S^VPm?E zCCw4Xz4C|0yYo#u>A%W%tAsZdu*lB$pFOHgkM`11(7TzHKfTfTeRXYoy|Q6|9r;V_ zvw$|NCsS|c<eG*+}2oAw8<%38j4r35IrNc>!tiNM#?9i8oFH?kq zjg6Ce5n6bqW#T26>x!pkjNZe2KrCJ5Dw7sVRz`?v0V@;V(z(b@$PqTVZf7mM2k8pi zTN_)FpcJrk=9IbU%1qTK;l3ZM4{Ya`*IiX+#@a(A`K>N0O

V{qeeXf_pav?n2Mgc*%SzMfb?Q;rGmdG^RqA_f9J9}bLnet0e zeUp0)&MyfpEJbEnwusfv6Kq*U2LG=o^I>sTA%{Bw-<}X(e%ogoUEq=?op(H8Twkz^ za4ps1u8AHLRYs0brSOo^5fO!wwqieGe3Elta&$Dr%PEjKBcHnH@A=(IUK}&7A_7Df zr_w8jEDCJM(`kGS4q?YQe2$%&(`W3G#R}_XiND;&M?d9HGpM{cJD!)iQFp-&Y``g` zC;P8^;zG&{*6=0_Wp*imqgRBkWrNfTErEp7gBc#=bP1R008Pett|hRpo$M|~9B7HN zQxFo7P_r8qQbPRR=|#yT-pRQdIw~pFo}UpFH($a*pUe@nrF@Dq%|&x%^d&y(6y$$r zc=om3e<#KI*xAe#H=0Y=tnV3j7T0P?O)(|B(xYp@O_hfoD{`d#+~}^Rs=Rb)M?6KgD93NoLm#UwCB?FEx>qFoEV=S+aHerOoEPa`Z{ zt)eJjw9$8KZ1}<>1g6NSt~@yYcM^hwy(s(yUjafH_ZmO0B#B9`{v>dBc;7AEt3v&o zH}4=qsR>T}i_)&%99BYJ{l$J3lwnechyxv-s9VPyW)@~aW(X9y7goWwg7XfW5W+x% zs~!2XY!uaGl4n)_gg(-BuWX>J{^P50qv}<=OU)FKKJ$6T<4q z`?B@sFY#yGoXxhQN+4!&cgMMhAcnXE^ij*d&h|KYQ2akH`%Bvmzmeen!s3jK&OG9` zXSTT0YiK2dt)RYkmoAp`4^^yd{_cxPrD(Drw700KA9Jd3+A!5=X5!h?`H76mBkl*Z z%%xYHs8)zH&pjY;X%e|yLAQqaY`NR(I6~RxUMVNWX4mh`O@z<>KuBlh)Zyxw3i}d5 zA}55`Ik0hF-S$ybbtRw+UIVcod?fJTwZcK(foO@L78(yh3XLl4Hgmg)`l7X3@l2VX}1B>EzYiSm238cH@Kzi4N&~t($kWWwW z$RMX?uFQ!!eYm-^F@jJQ9Ns>>6evEVo#xKtcgX>qwQrHw$Ag2`4s&rRHMFiJXOe|=%{0T%KX|x(j_i!g9wt3 zO?n77@6A&E{>L3BM>E>%9;vOK;T_`+D%|CtUVvhBEW?v+^S(1aSa!)zxf(LSkU5Yf z_+!G)JwC&zP*txWn4K(6>AlGwVU*Om%~2zBDR^y z)T~JyLOGKS85%MS!TUTI@&*Ro3$ZNSoogTfx@p?qko=DsW(nnAd+zpG4Xg%DS zdsy~asR}ROD{wu+vJR*nZ0sg^~vu>>XC|&SSxU^1wGZ9ZN(`nfaq@Am-qR1m^wUcQ&Mzi;Ru;tA_}2XARmGV zHXjpLWZ6MuDndyM+6{!Q?35laDe|MWnk(twe0LwMR>z5S{zT}7t56ww%nd0}po~sU z@5Qrw`52Rb_4w-bhiP?YymmC*K?T-FlaGZ#Djr*I%$8OkEEZ$~ilnO>M>WQ#t;w15 z1fHZBb{W6Mb}AwuAXTh>cZ4i`fB1~dpI1D8-uEzfTY6`GNB%*4w*>fVURc(LvY<#V z3l3xlD}?_R+n@JuQ|oiOH{Tw)Hi^jwPQE0a=q0EPwubCE$F4NoDfPBdXh^kD=PP>>^*dH z{CTZk*R}&TM07qkb=6pDfH@6`|2UUu|wa z`{B>J!D!O<^Geq_ycD8#EN7KWOX}8dEk#>$-oYo?A&b0CqcDV|I6eK0S@z`I*X;%P zFk|sANc}{sL5JW6gD6Obgd!ma?a<@Wr_zC!Z${bT!tQ@z9AD!Ux6Qvf6wqf*b8&L} z+J8eqnQWRU1POv)N~@ykEv6eQKf*4bFUJAY9$Y@pw{QE50!qb#aR!GXnp{(#(e zVqY{r{IB@+G1EoZea@e^-{fb#yDUv}yzOcAywB2n*ImR|+TlJ#kg%1qL?cv}C=yay z5W07pX_l=3{xP9hrSHf+3jA;n7Hj8aF&MQ#|9PTcfZd|T#bbTW^-IkN{f3~&sdv{Yx;-MB$FiO*~LcS4b(K_M-5Mz=Uq zh@i9$jwj0za$Z~8sq68O7f$_2H2x=Zm(%#3=h&RqA|))$*4S8N?0rA){JC@e7r9H4 znA>}g*ep$1WcuA!=;?yh^jx@j%)1GaS6IR;jswjcb54tG;A?1HZvc2l-F(@%76`tf z1wdi%^ctb*Hubgjl&@oo)sj7?Kih-2+fto;#{2i*bgzI&cGyqcH@}l)N|gW#d*0B_cZ7v^X0?E)L_=PpJHCP-Mq7+ zjt_KM+!r^ln(;fipZvVuE%_6bzRS$=zwbZu?mfN!vvm3>nVKC3u=Bh3WvCPGhpzuZ zd(H%&+-i_*{cLlKJ<+!UF37u4ei`4|GK~rgISv&>$n!Hh{-Cij3hQio;5uLHoQ8X7 zu>8~NaywseG~wTQ)zwJbuG=d?b%l41g=R`Y&&tqCal6Vz>%fnF!DTQnNGBhwT# zpX)t>Z5=C_BI&@rLQ=%nLmF{vev^N1@pO4qb2)2|Vi_jF{y^P^3FY+LQ5% ztp3xdkbz(oIE>gmt2~?KghY-|vgUs|(}!}g;1Ck%5-AHIA(4B6huDRYSjO#ydzouH zJhKmo9DtF9f>K+X;e2?o&G29BE8BhLUhCU}M38EUO?rv*$;HbWo%s`i-MQ<)eh;Gd z`sf_i?UG&Nas+@hdw7NwG#ZAnDjKrY8Jzf|u-W2*k zaXqti6SBW5e1&nG9QrmsWJ-M=AY10>Fri8?Pxc=okpX*6_F`rI-uHV^J#S1zCWeQ- zPusr%_VW$BI7s3J;n-18SeVQxoJc{V#tmIvE?xLLip$h^;G*rMt~Rc7OQJtowj#{k zUKC;5g0Vy9tQf+s^YboBto7%33AVM+^U5E2JVX^<-%h?gr5CXnZ;by?M=taxV|U}+ zUU7c6 zMz_Ff#P#_d3&q!K77r?4^=*6u&V~q@g7& z+6pf{vCsAn1s0~RZn5_p=4Klq#nJvZQFa1tHRw|Q%rec{HCQ|8p6sNTL4DxH*<{;( z4}R#kZ)58kJtRj%WNKSZ`!w9|@^xj$M#FCb{XUiRCC5OC{$UQHYx`PHK zE)$Ir)%b(WxeiXJUh?&4ZEQ5MS!E?;W0u*oTiJX;Q6C_&5H|3}kVhDG_kU3+LGhY*HN zVL+5dy6cB@Nq0+kcc%zQw}60vfPi#&NlSN^fJno${lCXM-}J*A!@l>uV_oYySMaCX zO1n&ty6LJo8CB47&glI&oA*YMdWf~{F=66Mo!_E_gsElQUD6Z3u-`|L$-wDwjfTEM zRiA{Giey8=?&gGc=Zo1pvSYf4xi&W_C8ovW*iz)N*JE1GpJXCzGzSy1W`>? zRG=g5^yo;?b?+wt$gvRB+GQe@#2UnmCw^SRbi}3f8)y4f79K~&Z(n3I@OGs0tyt20 zFVp=)#ES98@5EPc#R(>2Kl?;0LK=po1t)SBMo8ejQ7ugI0sKoHPWEJ$D2p z;cdP8{XSU!r3qF-j=ka=rRUzb*}hXDU-$P$wzQ_p4Ch;{A1E^ap7<`-;f$i_#1&BHr?%5RXwc*MKE{W%$3m7XXvQK zVcD-i=-cUUvVL!2*bo--(kN=-onotV%~kif$o1Xg?D8R07fA&Qhg0l(rDpiAMa>qw zVNZ-vWU8Tw__F>qmqMCy^nEAtyFG=z8QA%4Z?qa?(%r}iU+MU`E_4C?UzyycrvfPaV<0NCRHZQbG#q%5qavx z#qcKovG8rb*wFia&O|drY_#Dn$LBV^{Y~t-x__S5z;yqD=sL0y`Z64>S5(Vjsx<1^ z+u3nrZ3dG$r`p;A9XUB!xzCAh>-@v01&A7Z!I7Sm6E9r|OEgd^(k`iUmw)@R2x8lP zbZ`Kg(?L-hIQ6pS0J-1a$SAL*#1f=0jhM8Zat?8&&*{8KrB6Zs&Y13#cU3VZu}oxX zw?8vF#>~M<^%@dbjBH`QN$}J7lobLU!HCqhus8bBgl2zt*=+CEGP9CL4TUZ>`)}(X zQr!;@uX-P^UAgoQzFYC#IU5%gV`1igiN>F3dowb&I^T8{ggUDo98AGl2hbBE781Gk4_5F#hn-Z)|B zsJ0Y&p@1Gz1sEjgBknYYoQaKzw`VH(NIC!b$U{-1AywudP z3CU^pqUYzYeb*`PKt4|96zf~&_Wn)I+7W>A%qOhSp@@K}sIpUsest_=xUk~S6jp^4 zY{hS5NTeTGnMy1cUJ2Vv=s@sv_&;~%mr|7LdVKZILWB>r``zbC*T^8H?R;u1d#}Kf z@>k3vItkD%nTVo`t>u{1N|RAbzZj}_v$j8USIak$m||bPDxdxOs$Z5>5IPo6dq9V|cz#j!AbOs4 zn);;^sJ_^2#TebeL?~x4^PF-j&n>mlF;udQMbi3!^9j~~)AT2D_|;Z=VQ$-anu~S+ zJO*;$7m_44q2&ETD*pUP*?_Bp8w^rEAs2D zc1pk(dG{X7nsBMZk4y!$vC_P0I!s+wKiV} zZT%5P!hy3REl5tVmx1r}Z`twre6`SzO}9JYw?q)RjuJKjDlsU|TDpc=7|q{34G<&;7}KqxP? zlX!Q?fP+V1(%|5^H--Dqw-B$i)N+=uwEXlQLGqout+E_G^WE4VJ*K^;doS5%#-Y$W zNu(DYbvSdm#=uI2m^@v(JOnS*d7g|2e<%M^e?w3Alf>uFj&&n8YGh=D+(1m_fniR@ zJ7HdzdVzFB75oZ^J@{KUi?65C^C2jFl62;IAHFRHT!I*<9R<9h7`E!Ct;Ow0F1(-f zO*}x8Jlt^1cUTfhbWy_7SCpCbu=o2j`zirJ5@q7R#+{Pp`&UqJeWe9O2Ikp909w?a z!}cN8Yjf~3&+q8$6#E}RNrPvga?)JR=PY;Zu+=Z5ZsMiNi+HgnhOQTcz#c%4PeMwH z0qlJgF9Bd0B;fI9^I&b-9LyW_m`FQZvEywypO{H}&1O^TQgB4!6dwscL&TN}p=Y%ET_Mcm4j~ry^7SEdD{sugIpT zTuihvZDel>lRrfiH(!za(j|>zywcVDs>}?dpgdo4e*aK5^HMXLzq;hZC^k41-HcNC zKCY}Lf1vL9+>@oPA+eZeH}y?1OW1dK^r0sYnHSC5X6qq!<+jt)=~uipH5!-`ibkHzTT@$r} zOoqcnNnE|m_P$zYOxJTxnZ|mm6`j3Xxvl0HvR7e$zSwf|nk*x_>xB^{VX5`3di1bf zMjO9HlF;-0vJFLA>gm<2O zSsoHo?pP*~)PUTJ`H_vS`vqBP_&mwh=Rq@%Rh2G5M%9M*yStj-zdjnB-Vqc^6mEvd z-Tjl8>Jc*eOA{g&*a|H$Yz3P*x5`LAsy%cC( zXgFCqYed!B4dHp#4@b!-Lqlbri=svkg@Rn%>T0KI)4*>8F0WIEcQ1e`&0?%z`0qYO zW@Ebi4&bRLPlA;PWInw5<5zyC+ih!7az2N0wL`4RY@K28gXtdvt?e`-LGO2QY8L*k zwAGJywDJ{P%5b8XMA@y3$~&j*W=~qy;hJQ1;Bet{-WvG*+Yy%ay`dpht_TFVdpZUwLefazhB3_drY9nJTaK5i7rbo_nUZ?5?wU%$F=ms6u*^z zb+KtL57NMp8=3bCuPU=b8+eooaW8eJ$Y`O37Gp>bM(5q{44(cod{UV*lPnRg-c;M) zJNWzjQ0a<_rE%Ue$M1G3L5h?v2>~emKk#^n2sPZ=rUc-2KIT zIzDMWQIh)TbJe;y7r<4;@FK19E&5at4JwD9{(fa?SLmTn)(5Q~WSwiHQ<2OL`xW;H zR*d#v56+crCNfD`PBDfXU7jn{-VlgUkC{>GuoQ;ewZF&J3KeQV(?w;!>LBV*9ynTL z#%nAhxg@p5b=7LZYqP2FMKaaH{)eD~oS*c~r#-{njY$r34cy#y(=SNt!krD^`-<{e zB%~*UDU6YJvev#EWBTMh0?h%lLZ0gUZTr)8B@MnC#Y!C6T%NpSX^6^kBM5mt94~%6 ztF7a!<4<~Wv^_tHkce>5uAW3uU}A^7E>Jh5fItEv|4E9e&l4Zu@5=uwe<}YBK@F0j z4EYeGB!=u?ua%XzZrii|w)@gu*8IIBx(L*~_xoqL|JVW%AxIoaXmkwF)ps=?#y_=) ztI{8uWb2w-IW(P~m@u(^Pt#^mOCLQ`gk-8p1i_!57m1ItO)qFO;_*!>C>Tx<$A7*PBiNbr5W@B05;#es%5WRBK^L z1Ox2t>Qy;X*$r10a83SDA&FFC!ah|pTzX?IyltFSJ<;hKNw64hk|E;PewqExLU=nX@AjNp_vFr{IARMSn8k8h9V z5=4f6?CnceZ2ayDc!&vjIyej9a+3S-YcMopS?nJ7??Z?A-+?tJiMOWR z`@C@ttO3W%FRKtI@%G3GLlFpb2~y8qo5!LathU@-?&jW10`7m}Aod%btVE-_A6lhV zt6Du-7e6uCr__UaZP41vjb36^xRm4u{O#Fg`Yl({$A|0NBXNgwuV2!cK@99-O0OC|PG%jAlRikU+QeuJxZva!5fO1$mA%yo6>pueV(XjS|6X}mZ2f6?+r8ON`;&D>@}#~Xoo?;m zhsNT3F8<4|TMqMlbLLoYazPBr!2ZPD2z`v#{lZWYlhW7;n?~slUz3Ac@~_(|nPmw2 zB^vKvu@bN9YBwK+DWJwq3^a1)KCPs-Ip6+bH2ol!R~A0=wYg{m)zmn$CG4_1Y(s-l z3-?QfHEChug{APta_)lwafAp3q*d6bRL?-u>3;q~grEI9_SyVd)L@*mv7H|u5htEB zsp@#HzrEm<*_?D4DHS;%QQYUZGtq}U65|Y_;t`B=n*hY7FY>3TJ##$Xu>CKnIm8_z z0k!@*^KAeThe##%({JMT76U6}99*P+C!Mz4(r7%->wV1Jr$^TNWtMcwy!Gb&Q20A0MtdOMHXyB>HIgYO&`_Bgkt0ORXI|`%3A>$axfJtj zv2a-jveUNh{$`tgCXRKJGY*H*2WcNf1pF?~^LgF-uZs| zW#Oj8fA2*RV$e`a(l|%pCHY8e%sq^2mERpI)prCG8z<}=5%_)XnJ{>fKZit#iHSm> z(pVjZEd(6IA;F?Id!_A^5gylK((zC0;+n;&zVr*n?oebC=h{`f`S_oJzuND_yW3QG z*i;EMfA_}EVV4?RaJ<=dUzI~Sa>GFyS3i@rn^IQtuYyju=fx~xEly(If zNwWR4*`GOOQ2${~U^%bTaeyRv^oMomv-Q}pWR5=96xT*#@7d|SQW%#1IH!pB;}S?1 zHf(k}{z&CY_-dL94!Be+QrTOCETU(f8a`fpTRwQ1MAeeJ+` zpnSl%(Ws`Tq|bX*5QT~*i?|2d^9b1K=Z~}D&kk7MAa<2C&a-}hko&mz^rud({pq|s zU>_|hr z-HkOE3a~pRBF_F$IsBbQ%9s7rv$YzqUiEZmSV>s9hdSv-5Olj0wwud)`QLL?ee_WZ zUJMjMb#ju}?3*a6TgE5{nYlyG6q~5D zS1c)Gac*bcs+_I+-7r>≶`3%^+E7mH*wy;4b?V@6SiG(KIA#d9k}+Nx3cBo~_5P zsvJh{*K;4KM6SL3owqj2GEkHu;*bVAV%<2o3eB&L=8>J(J%g&|E76H_t9MOhhvR-n z7jzS95DD@hxb6e|$M%g=6f?&XP^2;u=5BDKmN;@Nj60c-1QRLB{-{Ci_j01hlbZ(#FW%!q41uo2axPiv26c7<8S)7ue8PtE_MQ5n{F(p zc;v1ZXb)2@1$O%%l`-tIT(8g(W6wZyX&5H4tGb^*6`kS_p`v|RON(ER|L9=LLM$Futz zXzg(IsBnodz5OgKz)zSeQ3BbVHgz_23qwOgfK5Aa`|6y;*UG?Qj0plUiA5pjM5Phk z6D>cU7978@3vkgKa^fQDMyDCU6F*ku#1ao4)wAa6%o4qocO4~Wm}xcU9{w+Q=0BMXe_v509+lY>Q3ek15WVAR-o2%)V3nB`Q{2f&23 zF|s#B$O6HJz}PT^1BW|M(AkZ!Fc1?Va}Xo&zoH+U^KV~1PZ z?l^IOU>@10Hup=B`^Vg;`by@hM#e`6k&CCF!Cdf$=$`>Uw(Zn@A|gql9w$nafYwP!QVB@HOmM|<5ZnY2i-H-&5wL??-&j5M^;e--J2=mq^Yj=lTIrmhW$qa z=yCgu8x1B4oF(&;F@*BqxEC3!auofI{tw=;aL_+MNXg{;a#Oe|+lNAOq&oBkviFej zmG)=JWLa2v&aTs#6iwG%328{hQ4?0$a{U}=bko!X&7{`^rXyUVvh)S;=)vSsS4(d%sZh8o48+R=P5|)M=+C8(jj6qAM zF>JyLdhK2@_iNq-!I zwm#1ohjdkx_w`ty;7@f^Q4MF$5Ml=^ueDUw#G;p$2rB<~KRA`U8~AkZ@JOJ!{PSX^ z@$t?5&`Z(e51eDA;^^jBgrVsiX{1!~x%cN+Pu=fUuZQ1rIxLD+Xr(i?)_f^{wkEfC z7e6HxDN~qe_F77hGb1v_tz3&`Huv;#uhj>zod}s_kXNsy@WSo>D;K5>H*UGv$*i}V zI|SKWx&F7-#x}rgtfH<_lw4MJoA1@f3-Xkn&9VN%P+GM^v9C_x#eo2D#Bn11)vwhH zkX2g*dY0`Q$f6|SMNO45g`i;_m?AmSV~t@ZkmpkCSuZ6!VN)M&+gO=(>KBhwQc}Pz za*7KfkYmBxLvY+fKf>v6PZ;CcmjIDyc%i~^M1r>i6*y{qDa#n%0SD{U)Ks-nbpS-o z$`$m&hk!ey)BQClM77ll`opd@De2?EVg*rl>u9oIHzJ3C2}4q%cZ8A$K@5ZEIaqiu zFh6t#E;kJ{j0RPVE>P=o`ru?CnX{~AaY`fLg`zVYRG>R5*GK1(1;QL#8n4?^{G*`- zAJHMmBaB)>6@07+Sb|eb64qS)>u7|0f8lfK$!%riwYW7yhW?2`Z zP*hDs-M}|8#+XG8yQ2?{{dg7`q-c&LyRgK^VrOJ@8W8;CjhO5GcqGf>e$rabIzHYi zXN>~I^uwi7m{Vnr!p4fj-R=9hvt}uL_Q0$~nnmKZKVm5J=xKcXP;@z5Ifqf&4OdM} ziHnn>+xKJ7nN=-OalpJaL7HX^WitS)LuddrxC(Fe+eS&M0YxM3E5GY#> zK-bKrcdOoO%&3`t*N=l{;J`^qsaP`ADi88Yz=XmxX1&jVgY*lZ97}B?2zp(3Agq`{ zT6B&(xO(?SznLs5En$P>l)xm5B$dgXl0lr(Wh_b@^q{l#y{pvoS`Q=U(kM(;R8%wo zU6%It|0vE)l>+f}lILUVBQod+_)0H!GGDId(MqWU%9gK7@Cs-8pA|~a;WMk@KBq$! zwcDj2$w+Q#EhGqa+SRDlL~*-vmDrZ%Uhk*tb@HwcF*8mYGqcRhDpCkSUiey{(Jh+cwNKHSMPuW)77l=Vgns)0!Y`!hEG*dG zScnak$VPm=_U2Gf(}aGKjpy-st^RjGe+%J!^2GY?>hL;#(^zzt?&2+kYZ49U*D3BdcHP0I*QrYoypk zV4sonIt3xMcc-P+8Fq_r99E{e6sEZ3tbxvIQA@j?rd3p|%gMk=n%{vyH^wV23i(#> zo!<_13G+)?S#)vL*UIe+FNl_ofelwC46jdYj(hcrP7@hY&+sN8zSX}iW7Hx*?#-AN z7wL0ujlW@ZI|~!F0i@yVf`Uu7JOC%X3_#CvbFP50r1!%SBmvyq++gmq?lt)bIQ$n1 zMT-xsMvO62j6_1eK*48fY;fPvHKzi5RgD(`O+0VYc0Ii+-{nzG7o^7Pv2AUCF~Gdx z;hEhn^s;K%1#z$5RkbWwU21;B|7AuOULY?gqB7pMiW^Qm0fawhxCM|9q?@A2%TKdk z1-6sDFbKQ$cGCXD;UX9ZHpZk*YYHIP;ZSZsB)4uiv+$_{(SPd_ksv~RFF2XS!`Z$g zG9oW;Pl$Rv=Gs1n`}B94kRjI0j5Z6JrkUcnPO+irsi7~O;a~suK-|qF{^%0VQx#G( zU!yQZddQc5_DX7&ILxG%E)cl>vR1qE<5iHg#5#gr27Y8qi>Hs*_p}-Q64oFH2}8^7 z;zP|tR*!~>3YG|`59jb@wr(@bJ>g{=jg`VW>~MPx+Np3Be2p5&zq1shy^`L@(_am7 zzh6H8c4BF*!bgiNMtu^My&JdupQa(rGtesyHj7=a=mOs_&?dEeTY^^*BO%!g(zUDx zomX}LN^QwFr1$unY(`{~U4%%+Ly-6Wh-7r#89UYw} z|IOR)daJyli>qE&sU!T=VI7Mt8wpm|e9aI9u%hu(~aSEbtex zx8Wi7r0F3(2+t%bpvMhy>AfSXIpPUH!Qv~zMMaNHt=3h~4TXe;iVvT&zsGJ;D87Y$ zbfAygsum@36QYfb)D<64GyxokR<=JmZrj2FhfO(yW?++anP1|GSX*Xx292onlnEjo z9$0-pR7;V?@(#UUr!GcqK2A?34`hCZ%R!Nchll4X^;rys`KF9liO^?d?2EMO%nEl?Nph zLK)&9?YkQEW(1CpTb+qSDRkmcCZ z*~zJT!Yb5Qt3p$^VixqVg1%)6icyQ&drGHuSe|ADa~UZr0*%i2mztIqa#cJWdfy8? zLDb8IA}RAK6^#`3;?f0>crsJ|?ro7O7wL=8y(VFP{(~m_B2~QSho>y(E@|pR5@9sd z5G+p~z5?T^B4a1NB0X)TEk@{`7PAeY9q`?Q&63cnPA5*fn@eH!ioGjyW)`qHCB!2u z_l1IJcHTI&oJrT|Jb%&0YS8lI*eYjNztOp9+5R6_eb8(Ya3iMFHJ&3>wUX-6UJ{7| zJnkPVUsKb~5L{tsw0MtzCUc6*?iijpTVKv@TzveB$k@8ULbu!U<>e(HfbP2rQbyp` zg4!iJ=_FWG%{Z6@ZS>}zQ)Fdkg4nX$w6w^Qk0D4%O=a^r%?jUU$l?^66(T<>mH^CZ z^^Y+=Lw$W>v7532MBDKB?>gD)9BzMhq>kNn>HD5f?b$ap$Mjkc#_tbeg(jM7gRB71 z9sQ1Qd#TX{638wLI;53ZK_VMgw*Y~KU^sCR@#_K!S85)wjbBr#l`}R)vpP#U!>G20U#W7Oo5R24Dy4oM#7+wh~@p?pwNEJ>1Mu063S)8;m z()vU7^~DB}J`8V@_iO*}1Dt!iS*Ed|xp{YY_n(al9AqVF&pb1f>uFRG=#-yF=4Rhs zA50cFM5wrkM*J8-dI7FhGDuA7@#wMaiG$2o5^|*jLnr{808j*=)!tLk|MC6?gcZ)Y zg-}E9N>ob)a9>z#31vQm@Wx9p9lP=`+rN|lXfHLAG%6^8CDy#0VW{(U+D5Zy$c&>v z7zF9%?9SNKsg_Ej&(9!Z(j=B8aSriXz70zWoJcFiNr*qPX_6;I#Mp`5m64fhJi1-b z`KRl>2d&*Um%U*q{q;a$0SQY|Bxn$LWz`!}x3P3%7yvUS%3@(*v9OGU6eyD-nuLUO zhaiPqTBLB=FJ{xV6tun)KopmqRhd`gQUDleCvr<`YbKE9Z0a;*MVkoy^l5v08&Fnr zNG{kJA;m!jqB~KWtR(QDL>{4=;L*gO57ScuTVpJk&U^nRj2M_O!Nt~!Opg6Q^i`M>esB-lg;JfWyuSh^LOT=>90 zF2G)bURAqNg$(VEWG{wtuse<5-doga!wJ=b@!$1dj0n@eb}619!or}a1C8o-R#eO8@1{&NECbh`gM-5pAK>k$mvlm zsr~j18Yf+(jOQpvqJjs2aYr|5A&_6Fp(Gzc#8#A~GuH?4Z8%7L|9v^>=I86{>+P+h zp%KfbGB7X@Syl#K1Q2Yg_gbJ5tO=oVXMzs8>fwh?;0>6u(IAYE?FCLgx==zX6+$VP zjnT{S8Fvb3x-Z{u2gTyPJAcrruBMrdvkhQP|J8f0t}w-%0%R#*0UNa-fB=sTIBkPy zs#Xj)yO+1cMᴞUk0SI_#ulJGDEU5{$3FreSn>*?^!L4)m=5h+Laz>Enel_$Qh z2X%G^bq=SpR#jF+cU|nuGo%h9h}O>Q=^9RP#c>K}jmx}3g1|7xvPE1r58|v?GDgj& zO1U6YTpM2byI%3{bwBm;Mbevk9M92YpT&Hjhk&IP9jp~To_Ee{cyvnvSBRoL(IDBK z06fN+m>2-kH)%EUwas#A`~DpS`hmPB*^39yrl6>nCJb6gR-kxLGzJ4Iv}2Ni`rq4! z4BkFO43&)0V(W~o3rSiUxq8mNPz_@vqqUvU%y472KAj4HtPppGM}tgcaEMK#c@`2C zsYnwF3&)F+mHI4;dUY^YTU-fBHd~NSoi0l@56gaaL z-;iRDc>zqjl&QcOH9mi6c=9BaX6vraTDu^b18qf(Ws$)PB1#ChD!o#5bqOlaz#wG` zGsWsaps7ip3gUbF@PnWQMv zo5mjld(7$;i8)vyu>jTCuZA*dL(gwocD(?O0zWEJ z#$We%x*u&M$e6wl1Hv=Red19L_HM~@Xd;+;qJ;Pf{b84dGd5m1j2vnRft$Q_Ituh_ z_y+P|ASn}4hs{HO4h>=S@#06p!d1#=UK_R@@#26mJCc5Yn0EhVO2MI z0lE@|EtP@ileosHpJeQrhdpR3a5;_6n;yB1o3TF*y?aoieSN)bXZg0$vomus}AU<1XcXv1Kw;U*~t#(GSf`USr zF{r#FfZPJdh^QCw^bcpx&nuH<=|2`m07WJggmVBIl63~4LhRo83sSyONP?=u37Jzc zAd$f*>GG%$fD{Vb0%_gi9p4v^6-(%r-hwS{p;)H^Y-*l@ujTk1#{{x*OpgrSf66Ee zGyZk#j#(b|#dlA=@l8#jGA+EJG;Bs-Fe1vc{ut9IavsFQh8aGnpRb6?$U~j*}u*%lS3{L z-Dlc=6^AOSc|nPCoEwbY+$BH@fdSB9P63SFMbC|%DDbc67%2aX!Zv-c{!e5aoR9{K zz4+@?uxn`R>UP6`69!OO>l+sB0B)NuQPy{dGZ(Kmoq$mhoMPpQOcwu6PO1h(d33V@ zaMuIS{-15eZt|XalA9jg`8O|9l9PW*qR1R)o!jrgx( z;(}tb-6HM%f42MX&5kmnVO#l*RcF&&uQEr0+%Yyfs{M(Zmv;b0FkdYrE&b`!CrMks zQ;z={|0%W~3z2XBh0Z?Mcgt+#DWv^v61w7VGltP#f1cX^EesVewpNsv|F>>FxNU%# zN%^Tif=bh2-&x*+>(Y`OkkOJ+9A&FtaEn1hk0mkZW?KDgdYTYX6k-7VE^us9 zR|dCXAp$Ye^>)%sg@kmHB;Bz08D;yo`)Vc5jpQ_$v#3%kpd-UN1H37%J$en7<)@ZI zFzQhDS=UjZ5-7`-Zt~g~8ivy2yDm_uWJ4j~DS~7G`ImsZ1K3_{G?IjxG?HfMY_ASW z8FK~QZn=u`i@!t{)Gub+H*g9G2?+~R*3m+Kid-I2`AGq|ZJe~kudgh_z|FoGeRg?u zn73Sh6uis}dOrBw5VOeD`95GgF z$1h6JI+4v4W`v;1A639lAx9^Yd6A|3nGKa$+xn~%b2If_V&IM zz#n#k)j7OuO@du0~SG}wgQ?j;2&b8~YWn)Um%K=33vsa%Qb*`7t5EX`Z+pg=Na z-K+Iqe&~pXDl_>@qCGV_6`IY<)yMX*$rYUHMYTGfOohH*Ap05?-nB7tvZGr;WUEoj z;D^uqxQqJu2MW50c-pxqzc;Y2v0Ln2I8Qi@%cE3B{9##^%Oco~``ciCNYQ~&9J`M!J zQQ&069nR_nlbM#!r#D*SOMiOof+xhLda;c{<<1N!i4qF{TkHssF_5PR6-AlscD5Vh* zap5@3S|7K6r7_wvdh>edc#h4$x0C8r7mnna19 z5-IEa0xA~*XNG(U{aK-NLdc8Q10j@>LN~=hchDrk)e!a%)<%mc z#k(2Ht}x^1@aMz8f8OsA6$1b4iqQtjD-iQr9Gdx^Xu+aH_A^#;L$HvH3cy|K#6Vcl z(`&!!@hl<0*U3rcvl=JXrivo;>g177#v=BT-caPh?;d_)*Gy5qd=lWfDpF9=CECw7 z$7KiFLp0050Tz64uyN9hnf?MutSh;tmlt2|xJ==M!OUFi-kliP(=ug1)ne`3rpwQh z-H_Rw?y=3kkAhZLVryAf5obr={zD_l2O`bk4RoRYm{*Y@ zDRIRxA^b7$`Vr0=sw|xQPQ%T@0{AZun&R<@+;k$s!-J{8od`YOl%T)IseU;fw_4-h z8<=>+?c7C^=&O#?%+e?#(I z*94F&v4htRX224i@;61=4_@-g-|*kJc$tB3nJ4%ubRF4`43Q?WRY*HAYrpfGhQ3pX zt=Qx(aPO}_gjc-uL2JT=fll9SVZU4OHQ>mbtb(7bssThmeyncra98r(&s}YO;>r8@ zj+l=A_a=Tx6qjeG7;ut9atQZc1{BMYfRlmsMU}qFhfI^s(#1Khp`L&OXgSyn{ z&*l^A{(yje>y(S#@dlj=5Ifs$C`%m*h>PHJaCv(^w|U=1 zf+J&`euekRGXA~+IUjm(dL}1GB0E{SU!tqz`%xj#YM8JkcY${}h9_;6qRzoBBy@b| zpRc@!`xX3dS642g7{Cy8<)^52iXcvfcENNF4C?K4S8)Fg>pbgcWxn7)2ZdMTY*wkSVd!207fvWk! z0K5A)*fan_vh9>WzOTQ(o9&1RY>(iNrW8{BLeAZzCn+z}#RiyIli-%U1Yfyc1Ia=_ zc53(E3?XYfjd^ypGFtIYQxjY+1mC>5`gV>JW(+@9#uKM3qJ+NK6&gQu<)>AYwiWc= zFeA&f`TbY*kIawu2*SQJbj|+&z5C*#@({c-gFDSalnDv3u|F#BKzMT`WAzip-=`7S z6c<3-DQt1`{9}en7j~w%Ed6iDsKAib7{xxL$yT&RL81m(n~ z81pQg(9+ijB-A*I&3tlBbi_UwjpWpK8NPRYFwVZ6ot=MKk#TX@)$CZ^g%l+;S%=;Q z?__)EmR7rCv|J-4X+irZsLSiUp8617LI&a81z0<8Z=k~(Hu>%g_p3yHlq-UYgJ^`$ z`PLcX#?v-+4=1_*8#%+|2m3%Xr9D@Epd`%OzGh_wSJ+pyDROuKi3~MabZ1UqXhsd? z;=<~#VTnBhIw?ou{hm}!Hyh#m`{)9}p+@KN0&tjKTGIPB`|yonpfo(N@Eg7#XBR;6 zd@$$*@VKZvm%>YL`;?l>-dCzVq{?#kE*3<@0#c$La77vzfOh-VL#Ie_P~Zd7Vx3O5 zqRbSRhK8Fyae#R;-gcr6$Nv_F`8wtKW8xVDC086BpT99HywssQJUkYbmM>GHD-TU{ z64*h5BIq+|s(cwO&O`x&L8)VErW3FY{e4xzRE+*_cNdid+A_xJmW>?%3(BO~C4S{=M1Vi-)7ju2HdO zgS$HA7q387K2-UCvK7FjbUjBlm1n{Zrp_G)iPA+=)+!YVE|lRkg;)eFi=T_G90GZH z3juG=dlY_(1WQ8+N_;(qKN$- zP;6!WGpERB#VR-WFu`>Uwl`1>xLXUs9n0b?w9fbujGoVi1tdy}er*GT_+As>6IpDC zM}N`+~_O3YQ@E$tq3X$r#S%08@1&m`A>w~Aix;rqW>;S-;4_1jnduC7q74Y zK1ApKaG-E0d0uXS`m0bV>U6%}yWJ!gJ5wFMub zM;#rpK+J0A1uBA@DuEa(p08Y?2`)Yb6IQkJAX9rS;h>4>fA>gxpjSjvTa2UUD>f6v zN%aE(p|5Y_>_@S+30mlf(>sVLo0#X^*-uY21LIUyL!uz^uA^_GXtC@J^896G#j2%X zLLAj6(E3xCG*B#0(X(w@1g0boDAqGmsPUXXDVA+1idc zWdXN@EWHUVJn|a{79D!9-s=eVHs2fPeC0^#!d56D0yfjGAozhvgU*dvz;tzXeSXfd zOaAhQhsghS+jA=*@kLU{Vu?ZFWnHjxb!M2UhQ>37WEBmKFbgPN^#)xmA|%pH^~|Mt znTM94FBBSzK+gKpt&UNJ7j3Nb>S1PPMzLi1c%kjIzYr7-qNNMb^Di&mAfo8|O|N|w z;YG6gjZWqrV}W5Z9mM1BQ~u z%0>GIDc70;c(}1qD1s>YXT8@hW3}@~*#u&VMA|BZ!!?u&*Zu)3VAjO{p6A3uI{Eo( za>UsYEQ7H%)L8LUBEkX!x-3Mmh9}*CK(C?Uw@Pe~3YIrzcx*z#)hq`DV*DBZIRqfX zEiEj-UkcPampM14BH9JJ_GUO38%gV}-J2h!HZ^CYF8XKw6NeW>} z-gPbCzZ1qu|6`GZN~b*h%u|mGiH1v9SeTdhdSZ9pB+gJJ`SF>b!(groQFwj8$uuu;K2H1-{7Vn@Yi%4ap_a>-6Lwc!99By937E;5uAl$ z+54Tq36dw18>?_mZTzA~_|iLD6^K@_2!K;wzZ&N=)Y8(@)BEEW0K`gg?`XJ8tpRFy zT^)CuNrUwG&x}WqAv8ST zQVIC7AX(EoLn1^%lBVER?zASFv1#5G?80Y~;D-FU?%!5JpMdpmP|rL8wL2l;eiKw+ zww>pJyU0ywJTpBVltBi!o%`b{)HG(eeLH&h6J*GCL`TSZW#Qrpxdf3gq`t+8pH!dZ`eEfoUx7a%B`caFexXeb0ux|0Vv60V- z51l3%2nUPD?g)@_IeN-k*ATibPy5;_i{$l1PUKKyIJu|fD@>Km4Ll#?Wozw)me+pW zvm(K&ZRAk4&QdcQ&}#1Q{|Exn!;K?{BiMJc$o`oS^s44kKYfCVgObUjFePB=4)!b@ z*l=V|Dikgs13FOcsKc;<$U~2hx8D-lPVlGE+U{-ey^(8^zJ`Xl95FG`z;R47_%W0Tx}S;;bfC`t z?&y8-$Dneeqbb?>W3>Pu8G)wf-&LZ%9My45VYun66I(?g!KDutBUza4%AYz2Sgx-8 zaZvm-aqvTw;lM@|g9l>J!Je^r%#%3^DvWLQpA?Jw`WZt_wP47a^c-xwKLhy1Wt*2P zHOqlm&d%`t3;cDP;nG`%WBSnl&Le3sz!U?gFk|D+ny)!3Qdr|&x zI(G;F+{V{SF5}Bye+BY8@Wcc0kSK43ZC^NVlYSd$j{Hu z%Zr4DFsAqX=KkkTz|OC+yIa0m_3%q@u@{IBcA7ewK5zp*8T=?I)A-ZUXy%0xxW1kq zA&x^xNC*&7?Z;E@JAj&F9NXXDC#4f+7fw3HE2Ee7ZY!;r#*Qb`Wq(kWrDQOH4X#r^|NkKwVMM^}v1%x3)y6+x;_g?=xkDf=F zneV*s-fMr>+L1d~Gnb>3<5}V+I%BSx`Yc}U-_G>g6Wt=;+PPrG z92~mY<5!A*o;xaUgP`|ZX^#eb^fawWHYu~pm0{&tDAld3@(fBn#cnB%f$juEfENA; z;ftt_KY#vgEdYZ(gnXzKrNW7W*fegKbgoP5U0PdnE&mKbw^S1a@;wVVc3m* z^B{P-8tduQ9((Pk@%!EiYc@;Bpor-N4gBT>gi2%?rZ|U8VWoPaRF}xoh|{UQYy5;WIWv*FsorKDig`?QVv+TY-*>Dj2Qt@{OPCB-3|pC>uP zq8}5}($Rghu7X3fc3rpRx`m(rOgA#Tm5&`Cc5tz?&?Ba~J1!Ltul&Gxw4)s-VAC5Z z+QJK7=c^v8nSJQ6bl$ZEy!3T)JH28ViMUz(6``v)0{zg=ih5vl2jvAz<6#aZkN1xv zj|=xpKJROAQ+XI?rKLgf>gf2+OIG3LsH5=l&XBq(Y#RPL%mv$ZS;EY8WP>T?5KI|r zrKO_SNukk5l)=wHFU^04KD4)V7vz5kIp#fEb-KnLW5feTDotG1VvipEh=;H+oJaAh z>QQLCc$lftNe8kx+Zpp}N3yVJi7ekxOW%UMFQ*@vB^x$-R8W@ZR;C80&# z%GJyRS;wk_g$&ipAu=N#G~b^^oUTM{jMX$}Rpw~S+`+$FdSDyKmI77N&35#7#@^rW zeLiz3rF#_e+T)=5xq-|hoxDdP2ASEPpou4MamX=Hr^fB&; z5`sHssq7aY+r8P7R2F;-kPJW$#Bxn0*Orj&XehAn!erGM))z&%^obU<-hBL3bkRsJ z5seIf7VmV8@WMnbq~h}=-zykO&7UedVhHw{3D6tykW(rdhFSaLyN~^Y_EyiLHm&yr zWKeI`8>*_RhLx|U_5Pi>qsw-zuqk%bym{PQhQWXCEfLShLlpx*0RjzxO^?G3PXD@Af+Xn`uG4iRX62sq*osYrB;I5JH z9g3)87W+;Yx)BcufX4#qEU{fTx}H|y1#5g(2(juyS;bzFN#rB^s8!0%!xN2f0onZ^ zALAmRj;L~K)GC5mPj6slBi)1Q6_XX)c68xzF?|88`t{OC1u|=d+HdJZDrE)~{4gRw z{+N`<&@O@B6z0GmjLHUWBY#n)Z<#3H{sj0ttjmE@ck?(_6TuXW%1w1fX>uWP!zz}2 z5_@FWuquRv7WDa(kprvY;o2MR9_p@E(26iJVGZeJ0hYOiG^V z@Zi<5PtGqoM12{oI9~P1M2jFdC|E*{&N^AijWzG|lNzaO+nOi3^j(crD1p)R)XMK- z;Y8YMcRZZLXxbWOh5Ool2XUlQzsGA|zlF+sM=d~a!|}h8;rZD6(kNQ$-h6%<%{#n> zx}3C~IUyBO>)+826Pab=7tmd8==9gGy;qA0Is`R;IjH4nfiYLwZ*T3#FKr|pUPo!6 z9qPGK^_U|SVR*2X@kny=E6`p@5auZMmtO`4HO=WJ{u7r@gndeYdH1{r`-*X=dXUhO zDi@(52zt?U^Xkz#u3bV3^gRh=Ige+FMVPp zl-2^8^o2P;nI{Y(3vMx(|4%2PcC~PcB=>+a zhsBf163Qto)2YI4iHcCVaur{;V8~982HJDYM|^!ZmtmnD{-;Vyijq#D{%xp1wTj%? zEJWjP=x$(k72&inGBS&GNQKbShvz<${X`shky9-Pw9N9FiD+WlxQmy8PXt@sgp;`m zna%^V+etDY(?PR&^H!2Fm?O;0TG5xgJyXjB#KpN{Z<#BVMuwBKM7N082By3Z9*sPC>+CgGf5a=P9?2$mHfYlbKpxq>UWqh&L;fRG?e7pcU6HoEK8Zk8s5+!yGl6mW(*;iXq&c z#HG^{IxUKG2or89+m#Go`R`TE;P27{oH^N%+wM>~!0`y(y}7MdL)$Ki zMnHsbN7K>M1HKANY&19ynx&OlpI^(#Au30!(0m3k)aWBmP~%9^bq99Fph6t^DA)-U^tBmX|N90N7EG*xb3yVIr1~CSqO9 zP)A#(!W@;FFB;EEIR22Iy664T`&)BP^tXvEE#_Zi8aUZ&e1WnGfA&o4!kn!jO_Y3X zo@=0)^PxMRGpY-|SM&2M2;`vm^5viRdA%-H{#dQ?Y|Q0QZc+YZr0YLv)1gDLq%5Z- z!2s(VWyy?M0-H2AV0;TE8t1y6#rMlTkcD>WZz4Yt(;pufCu9Jstwo*|H9Z44Av;yv zrJ0u2nONB?DKmSG6w?2kB+whqCbGDX~zCC>cpS1s)B_l-* zJhvOxNCL|YweE2XM@N;lZzxvOU6lY=e+#CU8ChBT68$c8*)kwfDHWutu>%r1T*%k@ zqcY03rmf9vbyU9N$EImssvA6|urtOqU{`0*8oPPxLNN5u@Ps_rSM%^K$9o!UBQVw< zxL4IV?`!UpS7)3^O${88VlEvk2^~VE=i{Fa3NRhQvzvHD25_ln44`k>f}Vj^Fak}S zdm?*hI6uF@FWK8-UY|Y9*a)jI#2P$@IrDGWZ(?0Frp*akqEQigWv1eOdyXa}I81M| ziwlv=7ZyD0hMc_|`c)(*lr#zAs|ro3 z9@)lNakGD6E}&)Cm-GG0eMZ@lUBLH&b@?X;hMQWz8|^M8f4?^}&$5>kPgRp#?uL3s zHOtRlQfqxBfe%LC(5mgPGt(P9*9~C3C5j#|mFtMY|11MIjh%_UgBN`X6;(i1mEl_K*C*|BVTx6 zDDyNp+{H7XWSGXYvOoGW@Hls{UQ{4=#rGRpNT=uRP?UVzhNqp<;H%EvH>xlGP3Lz0 z386Ww$LqJ`nO{FD^K$E;wwhRKjK#rgMWGR0Z!oFmwiZ+j{BIIjV&y%VMx&oXsuK+z z-Q*X1g>TD3s*pnt>;&Z#!7crC4*c3z>wJN&Z{3jDz5MF|i&E&W;$lK+HgoMysI=5R z7`4bzOT@5Yo#6D*BNTp~)+NhywSrP(-H{EQdIChqr8})C1v#8) z=IV-P0frm5c|ZPXYx}EJq-|d}fnx6V)YcgkjN^zd<^WMAv7eNb^d{1+53T9`ZmbsW zJ$VgfL7MO;!CFpvU`b%!UR(?%CqNSqS$rI->#k7j2U3O35(MUI5+IC6hUpX666rYP zhjXC@9TJ3XQr}mr`Pktm8d|~jn=k*vLo7(VRL+oanO}{=z#u32%N3awm9A}xiCWg@ z$gcY28g*>@^rGnwA@*BLsr&jqnXI=1Pmmtpz=YgJ`pULp8XEDYQRR8v;_wu+r1`d4 zeypJ?ZvEAF0o`jEpe=qVx%;)tm)!E5-T1%&+2qDqH3c=(MbB-UJ{4zHI__ftfIuLS(pj`IzS^oS9c?=c3Je4#H_zC`p_1<(J~(?{+nyj`JknujEw7|DQ1QAjQP>;iYym zOVD`q^z@8JuKoqIVP9nEl^jjS5@v^pJu>>{e?laID4I zjm>)ch8GYK!OXKr)cCZ7`89sk7)_dl?j!;dffs9BZIi_uU&jP&a6mAGr%u2|t>Au< zViOpPc4g2M?`r`+|vP~?=w*qB6g|yb_P{r zbbZSXHNLr1SAv=xDi7zo6nT=}=+fMrH~7czy1pSZP|!)bk?H?_$l28Q{PfxPFGVoJ zQs*oN+OW*H(bH`ypYeb-H#bSyD!{2X8WykpW9Ltuo3U>o0}*t+EN`Sd-}kGHZc0gp z@m13MnVaL(+Lt0)@qVz44Ik`iSvz$RBQ`tF9?w#Wu%jr55ssLxZT|#>T(EQBq&8*e z_D3V0K$mBcYvdGG$st55KQG8}iMayAb#px_l>Cl$NSG^xl7FW^_~oS#+Bw zH1tkGevC<^)ZV`Kd(*YmLj|N=Gz?;(C2g*)4TPV%xNuBmFN)|Inc|ch*a)_IZ%_aB z6ocA6M}-XmYKeV~@*Q7%6Y_s`*ev$hea{r;I6)Q8TQ&AhDS}+4yMdat*HJ|C(*=Mg~uf*QWWxnh(^RU{a#vyz-#2oeI;VQ6*uv0j?7zU5S!!`D$*5c5@u@c|u(r+@UqxWl}1%Cwa zm;E@Y)&H@PX6Tl5q88F^s>U))En8?kz!Y6Y4})q5SV1LxD^#Go&X54ggjNl_?$nH0 zq#V<04*5A04pV$E9_X_})pQfGNSp7}s+yvYsWk%%w{RJuj>EPK*IxoiK})M}VZm1L z?=K2^P_#eOPZI+uYM9; z{$LbO_Lt+9w7llK$g6RRhH+g7pKWue$tFaXQOqQh+aC>@BFN$uEWsHSmtd~qOMX*a z{ilEH3U9=nJFCZ_xCy>(iW0e8#No@q5{U@LCFR&{XL^&!Y*umj=mu2$0G7a@*Y84E*_=t7gKt#l*%}+@T%7eBfkG(YA*TYqpP{nE|AutpLk0wJ0wj z8m2Y?nEaf5Bm92?HS)n-1=ok!%pgN$ou2?DJ*rFl3-LmYjn~L~e*UF;)Zd$eZxi^_ zy97%e*&9E;cU}GQpU%+gt{Ch_WgnmQ$t*60b8y(cbHT-UgnU}?Egf|dZovLUxx(8A zIicW?)TES)luJB262>QE#t(Lkz?hBtm{a%pG?{>p3*0JDUZ7R)>u7%ku=IHT4T@j* z1>;<-X^Tq+2b*-d9=hwbinudW@1RtE76gI&tv8CNbaN3e}@4944~*9 zM9PIQp@|omjstcn_hI}t*%)HRXJ%^AjhSQ;mDSu}pZhar#B@?9|}C@Yl4BiN}fJ^8!BRqIY#J|^*> zoFIGvb{rVkJhbxhbZ-Kn1T!aL>DHVYKT5%VniCaybe#9?lSaoPC2x!)=ml3dI_tMSXfGj>;4$5)b z^_t;feQxUq&mDy+61pX6EM!n~hO)c?%*P05%%EVfy;G_SB?zLTzFwa*3C6IBii&91 z<{|w@b;$prB|kXLo$Fx83(dSa>NH51?Fy*TZP%Q$%G0qAXS^8Df^1h12$jd&RB`&` z(A^n75>7*AmpNS1;9m-h5JciC z+mYetl;u4>!VS69g=iH?P|T@V#7$OV=aZ}PZ89izcBF)FWMoc9PNfk$ybf3ht60B0 zK62-Ck`a&116{EUo+RbA%Hp9FsJ0;E<9~kNsxk>0P`bb(`ThHg1re_^A*#5kF)%t| z%Y^^;bK@(abfVWiVKV{ORYajqN#6I1%SOZ1+MG_WBQPIcIs5;rqGy7ER4iF4fWhhM zI{`Db>p(lKdx9}Gq;*^3d940{gG(O{E6lsX%v2&nE(wFbVD}*lxfV@A=gTj@0F9rj z;Tl$M(&eH|KcIrPYa=v415RwkT$qt0_w;`I2BIvy+cV|5zug@;#p{veOb< z0~3bVUhei#X-_ox$5ZTo&v)r2&a&$0tM+fiZru^x z)SR#AE~Km8>)Ag%a~;?q__7cMplEuiX8`H=we}Omsv2cb2!5^4dJ_Vh- ze_4kOgkJU;ee~8Wq^u1}vS9pq8cSv`oCY-4^{{japYJZ|zD>=|&1H55>@Q)}a-}{u zM0Rw)eB7rXW*twCKr52Br3+-wqveL}d$&K9P3=mS4?13njmqH&E0n*~uAy%n-I;R$ z?A(o)|3M&gyIJh6S{Ru{tVw(WCGL=Snp&^OVV1jY6hUxF@9@mA0Hf2+6teKSaIV!zQZ4AH?#f~jhG{xka$ylK5) zvtuF_(h=w9yny&8TyH?gy`XBN>ms0z%l%nk@S7q|CitC{RlsbOzPp2vQz9QLt93OQ zY_yUd@-V&+&C?TvImlBm0>U)F#4whxOTT0oo_`je>ltdW-BUV=0rj{MV*wKGLH*~O_09QsT3q+z)-5!^8ts-zWVj5u3v+HXdHb@Ib$xh!cF7h(x)Suu> z5l*|6+ynCye(TPhQZ+xPToDODL1BK7agSIPJwFHRwInAg8a%?bqklO7(`I=-dGUf2 zKmU8p$pJy9Ty5p+FPk@CwwnrqU4)#=v`8CH;D#yZQr(#D1qup^ro;}v^_OwjZWItv z0V!eOr3UAMAs4!W2^hBz%Vj@PE2u1nzg?TZ$lgGL>S<}ku99Fe##e#?s;C-l;OFIb zBO?QQp;WjrLFT^_wPvKW6RotdZv}-tynUJf;CAOwO<^>eEJi`H|NhnM$vwc9Mf#j% zN4S}kkAMMO;YlZ)s`}jEEe8%HT>K|%iCHAyMPekY@dy?k16PT29mEd6=UG<%h%zSK z4PZH|CG1?GfV0>fr7LK!kSl+mb{GQUlqFz9%vcZz{i|}`cW|5K3>HR)9S&rR*xCtW z-zSz26j1P7`QjAm9jh(YRNEl-x;HgHob3I(&w+IqoRzW5C@WBV8_EFH2DV9c2sHW< zJDSTk5fWdEnFQY^4hGW3=i9*?nISsfrr5BQTOqdIzJ3+uLjapwsoC-xg~f^L2Nmg2vKZz-qFJ?*k@@qyizgdz9l zagBkKtE(1eR8WTtjkutd{H>n!^lyu)ONtk`**g2Upd-;4cw6X*J5EW2Y9JeVADh*!XP^3xNg&H zhynlki+Up-0$E5dfY3N_!1|x*-t_XXI08h>tK%7{kDwPTHkp)Q2UEGv3yV3MH{JT2 zy5T8{&UxH-GcSJOpgDmIz%j7(!T#FY$m1!2f5)I_-1~{buTu?VYaTzAOzO_x)Yz1F zZ@-D_mvuNd+4m}+XxYy5xM0~Tg2XLwU=8dh*c|*eanu-EwprWKIAuDr3wqJNN~FLJ zJZ-f9<`zvXQ4=Kf*GBGnC%g0k1gh|x+A3B+N5?Pd>fA2Np*|`CjILrpp)|FULTT}l ziCD6q6>NxID5u*_%66PdG_=&v8Oc3GcS&Fo#Y!hXXf2YIi;6)wuyH|sVQ?9pk@lOK z>2{KOJSqxhX48@jtjWe4zM#5_b-tk6fXip;n?nO258YlF1_VxLXZW~aaH6BrNwh!c zueyhO6uwEFt77}Yx(Y&#HjjI)hiaW>_9gqb16qbwY*VdK5yi#wI+xp3@&IiI>F60` z0&`e;Hda+u0x2CZLja7rd@66)vqGW|5Rx6!2@MLF+?4KZs+jftJnY9A^$;t~fv4sY zJET%X8!$eFs20xX0oZ|1G7NAx>&I*cYB{2)@H{4@(MXL`zPNZwHeyoM22tTm2SN2I-SofN9t~ z=HlXlO907?y|;#{=jP1GallH~sn~Z22v3*3CM9e)>El>Tzu)Cph4tDP zrd;b2wRD`=qP8r~0k%S(g*>Jc=$|6(2A2iJe->5mhqbK#zE}S%ykF!hlHn&QcX}9` zD?kwpchi@LFZjAd?hmjAY+kC&wxh9JgY2`Wb)Y66F(<~7aZX6}h+d;Tq_>L+k08(6 zP!K}t=qzjHrM;N5d$sMjHNAY-(aJy7sK(rFX^={Uw zH3gV78=n3}+8oF={&!6iUWjP)zXv#;63P7h&pt3Z|I$fp1tIbw{EGLErh1b=wEm&r zdKU`wD6OLMvAdzj*HhXVfr`jezdlRSpAMcA3B-7K8l&;r1dYapjvdMZ z1X6@=7SZ}-Uyr6U7|}{95yHp(b;+h<^v|tQ*SmMeO1u1R7d9Pht8?L-^d*o5do_t_ z)t3Qbj%^7uAbQLk6`znm)Y#Gd%UZH#c)}TYU$v{NmcWr!`oA`7{qMb~^PxxfGKOEL| z@1q-X)lcGiCUaY`*Y%j`Kg#!Rv_<9b<1?^Pl&!#aF?1}L8c&^nWLDh0f_zkU#Dr)~ zJ10XB%+}@qCB}JUcoeqlbkPh&hM3SILk2%3dyGW*BW9jHyA$~YCrnbZy)Nu2;n!El zPbd|9LIir2ISe5qqYx~+D{3~2LU5QNV(aA)6!oeCQ!f}^JMG%q*uV99*3KpjmEo`7 z|88s1ogJHQPy^ClzlQe-)9KN_4QNR1VxO3=#5dpgw7V8*ecE!5TVYEgpG9!+lyjX~ zD*{Duu@x0oQ6#C)<+vj?WnbgiIALESrq)9$W?VjIFC?Z`HC2pr zZ4D2qY=#R>QQ9rtC$%3-_P^TVGx{ug3shZcJUuw5^$w>B^m!X&v3)nBut|vah$5?Z#I2Pf4SFG;E3h4J;DNkNW@)uV}`&D zMrq)`Ri#CG5qFT!lGR^$4l3!N)-E!4@7(D!Pw~>SsjU;WTuq1C;xdOutkVJ8^gJVz z)4o9QmL|HaME6^?m5!v{V>MBZ^daYweZ2ndbTpDPlOXca&^P5R*YXU#7-9ta6Fyh` z0>*&NqSIY6hr@tBvv={LrLF~sklB?U!JKFdb_>x94>)Z(<%J?2%b260x%&y3!wM0| zU_L6>KGZTkMXW-%s8!$$N>Yg?g21oepOU04fXAm}Cv%3ig@e_H7{12feoZJGt=;Mi z@O;1cggl)4?ZP3sqPFYW_>}91cXkl<=~)7 zc*sAt^TuVZgRdw0CgGMuIYtEfWSU#N#0^urxt~zE>26&j2z1v?EAv8)uVj!E%xacy zFFz3-Wz$f0A83WV??MrwCg0CXgRIsp4C;5|yrsOj z>HA48UuI|h@2WN-H~v=yf-;Q!VhF-~Kom_XLC;M{w^jP+zdkd8d8-kLuks{bnTxhF zCkeM=`IK)eg2>K2#1xB7)%wC;{vtPwlJmvWP^G-D2YWJ;PCC`cE&Q7;f9oLimUL>n zZ>2M|)P3U@mno0!I#J?x<-B>HX%K?UQ6Q#Htg~Oa^gKOTlihV!dB8gLU8#!2{O!5< za0CKJPU^C%9Lk-xLafE*FoU-Iqt}AR$YunOd?`w730aX3t$O@@blDQaDUQ?_mPTxP zjEtoKsa{vqzMf;3S`6Nui{^lhU%ZlQ7ck!TpJr|s zkw3yLeVV2{gDHK4AxiuA7v|VMQq~F2BxpLqN$h?fGUS0qL-^tQz5NlcPi)WW?BSS` zphI_=%s21+5+V3Yfgf^Cn?6sInM{D@#cemUZnfhRLpJZ;T=D}YLWXKzR zZU~|?;_1d(ppe287w8ZPBSy(6Rabn%+)gfQKL1prw>(2QtcwuJ znrOH>!i{2JfNH(C_-agN+HIS6$f)AM*11l(vF(q(H8l7*2oCw4%a>CXH}nI)qTjuy z))cyRrP94O7R4t=Y#dXlUso+p_GiG#9L6Ge-LV45V1%OW&;;in$cYXXwMloA6GXf* z|6I>mYoD&D&6-pBRyCygYw5hm&Dr3@0Zi#Wz0!Zg~9t9`9*W%l>{?VZ<$I+?7l0qPTCSJ6sAF zM;BFFMtFa>pOI~N`tOyrc$NMl!^gp*<^Ijbo7ZFcuo^t#_oib+3XAjIYT*@h;V9YtRbs|Z2o%1~cuMUsCL z-i~4pdt%vqdiGHWvs!fPM{;KguPim@{E7L+)_Srlk(ciAE+=jW<6h_xDL}l_<|%u} zU8a|8Gs1f~Kv;OMsAP3u{ioo)Oa2i)A&28K&TGGZ9NP3Xlj*o}MpG+C6X?!{57uw^ z(kqa=$w8OfElylAk$d}tx8>xnM=7QuhXZ2u5bWNDk+Y#0YJCI@ET=hb`vDqY-$J!1vMr%j;e#J zZTn*&dE;H<(cv?Kbzl1G>!qQ>QjLq|aRro|$KTBTD?*uoX$OV)re&;Sc5(5jEQ*RF zcis?8Rc8qla`JqQO|}!cXcHcZkO`j`vknKIl6(pKp=IYkWxd zz2sJyv%!1NIj1@a}ZhD3=TamJ}NLz<5|| zE3C{OWXI=9aq%ieY!<7U6beXvy2nc7q~tb8<`B`x@f?u-*&%$_TfmjrnJeUrCt9MA zmrJpQh-#xG{`>NmS`Vufq>wct){QXIyp0SuO!T^QuVGR3E4`6x#k)0&ZM(`fvjT=S zg8ig-@8)Y?S97{j{rO82Q9(YhN_<9M8AHwY%;io?@);t9*O%o8AFF_8KBjX`+%0Tp zM$d1Df5y5aqWLOIr8k?dBtX=6-i>4uc$a=TY@G&8Ob*6B;D&%e%OId-Z%=zKKvrS1 z?=I)EzpJb3KgTd0PzInf4%n{A%*_ha9A^XbFYIqedga`=~J#>9+2Fm`u+wM9Gbm7Eg(85DUssW{Dd>=R{{|ES~p$}sj%sKR6Tpcv+S803_6hGCt<>8|PdF5}53M$mg< z!B3Gk7^Fx~T1kIx9|H3n8y7eA(PumAMkbcxYlrbsbMu*A(5@ah%$8u*BS<3&ymJWFGxF+1L@>2k zpXk%z&~DAA^TO?W4vzvCmgX00?<+stsR26nPNbOzxxj06t?+~_5tsRcU)r1*(WqC~!V+BxjSlXUchs%+;w>3^+!N9Lvq$|smlE#gTl;85 zq#~@P5M5x-6U6WNtU;r-!S5_Ey%_y8%4TS9ZQhTOdlFo#IKe|V64ce=OUrw~aHsoq z{bXRr?V9v$Vb`vK_ox)Y?`^@j3i!^LrCW_Zr-xb?sy%2Y2}#y`1fs3j!=8C75YhI8 zez_{j=v^-oZ<(6E>;st)=a`l=jP%)pbl~ak2#%Sp?QZ}ov(9!3f_^^@#Qrxn$N+}; zCIP#{1zCY&$vwMLLJ^`o${E(@=er+hdOSm z-pE&-H+*cmCY^8)OWJhWPzHHOCvChZa*b1p$p6Khf^D`F5JGkU%oQQ@JVuh*#(OBn zvir+LlGhrC9N=?RE6Bwh&0~MT?gbIIrC3ZulJX*?5kl6IHv7v%jzM_>W}7}|dXfjG z5?;K8q0H~dhCPi{>dh3pSO2*`?t) zi)_(M>Ai%rIq4$DYSY7?UP~Zn=|(qDZBMrT*nwTDTa}g9OuXmD5+utV^6jZJ?}^Se zsG)X4S`{PlfIGUiU3jt${BU=36Q)0<_pccr@26rfU-0S@IQGjf$ViUP*IY1^WzTTF>;EW<7d^+u z)O?{c*{J_R$$t1#=jGXiWs|B2S+Kd6>SF4Gj_ZOp_XA;)=pjb0qhmZGOI3r zw`-_Z8`v7lN`)@K3>&q0u8h|$y_)fA`@XYPsn7D^{d+C$S9f@SB>ldTdFsgd>Nxa# zXL%;@kcOzHW&35&a@JXH(9(tKiua!DUt#~;?$_S5m{Q$tXD@eScv$g21J)?LeYhzW zH1}mqMlIo)bBTkZpt7+tUf94Z4RZwQ=oa(7t5$>4w(*9JJ82SblSI$_%K*VAG`0Ww_RG;XbRnk%Xy{vEO>l z?I(7o2W2fnQm*!BpV)&-S7`PdWsdhl8iwj`rF;9;c1-&P4qYdvIH(ZZfagJDHXV9W z?EQN?aN#x#pdkfNIPG+DS!zG2++TrB;G}Hxh^HM5frd`TLpZa&b<-;BF99Z#%QFiR zg{a)!)BTzAjvG>ghAQ~5Bx4$kMOsBW*i#1LRq5#HPV~-KO`CSU->cOB=0X?tOyqC- z>gTdOY16rx74BLIkET9%6nC`h)+wR6JqR%^@5P~Er8^OTMF=*=!p|B&naAj3j!|WqWy|!Lq z|EO5fR|K~n)6)F5y~&B#PsQQq`iGv|&096e=WmSC7^(Y-FE1Aluhy9!dd}un-S2PS z`I2?ARwW(i_{Wm*quyziv~RDrYs8C_4?#z~FSinlFyYQKb>82jU|Jo~? zTr^RrE6}k zpvLsL2F?@H^o$G`3mxdS91UZC=eUybn6GQJ#$lu{L;N-x&E!c%_zUO(A=Pd0paWH& zbS1*PZXy6vTE}DZ5mdCJmiTMet^t(Vg2~%Fn%+E~h9Ib*?X;Pb(wU=DwSsLpUtmUS zE;+$Vvasu$_<4&Q;O)<&m@|mLnM@{^tJHsYL$eJ{+&%zfr+tOz`1Q9uNOyMr3nEIG z9v;ImkcWRf`Px(7bS8$ovsT4;{Pg?}W58ZjmTI2Xu5Yf^O@S_QIUvu$Q`hXW5K(Nu z<^55Q=Ftr&ab-D$5s$eEvx&VgFK+-Ny$k>+Bvb)N6(x`EB5(>{%XXQ_>pC4emO{1T zTsJPOo~hrdX*oFB?B{6~7fxaL+nx#~@?IxJe@$ez}pkSXcs41AD}9VD+-Ji{9j{Ai?~nsPXM zdT+DVa9u@=KCX*WA@t1e#f(A9%;ab3ch2*YVan>SCaJiB>f-8%uJXxACWQ!}$Dk1v zy`HWSq69LnB6uCLpKmE%LT5a)Ex7vI`tN9myHr*I<+RW8*YU5bpFg*LdAUFoll*A1 zK2dpT{)>pFxj+U#?i~3&5her)3TIv^{aWoe>57C$U(S;9Gv!yTUTzj$L;Sw_d~$E= z)Zuulv1&$jR0u2^(&yXKunucG2-1qA_uZuTV_FVkOi$*o-7iYSGz9G?2YKvGj=_>I zh%R+W@t+9-_mpJ6INaq{4^BB3p^Ae7s{{q7&6X3 z9?Zu;5SLfrvKI(o>KS5?^jUvPX7MciH6zc{Eni9HUk6CQb|0=O1lcKt^8)OcqXFSM zz|(H3E2czhlbh<0^ zaZ9?CpD>lW(Z|PgjV(KQ0?U1(nGWKfxr6q#265JyBt6q6|CLkUrfd$#gia5&Gbk>G zF%}2DWysbZIy;w5TmCGfv$Ip1Q%F!SjE|)KabRGko0Om+1f}ekP4?n#ll|=6D&6&p zkE*zc&_a;Qba)u`ME@?f7I%90q^Cqr&*U)le6Phc%g5m%x7%Esc4a+UI;eEm_P4!- z?^;pIDeJj~;Bj^KlKTsd%H6-WP0p5Xw?M?Um_*|7AmgcFE~}66efM6OV4+S#SVc|; zMJ2rV=L4kYOuCVn4{^FxmR@{Jvjc@fWu}c=aV>{klan9q4Dzq?UB0I&*p4>hWs8oE z3PH}NvzmF;up*Wud&B*xo}L9+)HMbi@X*#d;l`55%8-$Tq2*9G7sH+(T#6bE3d5mj zMUjVHf=Hnd`Oa%JS1vHiL>8s|K9E0atE{AG9a!t7%}`rk8>3J%pYdHT|5;p>;wWK3 z6K#Qx;8;eA%%|hc!x~2fP6xFdos(3;@mUU~T;8>OmX4?j#kN-#-N+ks*mJ%meb&tzuyyYP#MLzPT5lW99Q_1N45AfqfbHvc z5mfbC&hUcJt}NeG*zGIX?@(hHTL~#akAjXTV9QckFJw3>vXIa93W1~qL1+@kYgI55 z(_Ul=1C|E7j8G1i2c2SC4%1phg~dFfXYA6i8vEEnI%*QT&fH*f974zVBvy0Gh9pG& z#`2^K-PZIE4|=E0PXb@%X!dY5H-;k`*@?%>F|Mf1+ww?=Sq7_WNqO^M>C>;}?&It5 z_XnNcYmpH0`L+v7J;)G?1MBY7Ejwq6_w-t};vR8xbHiwi>L`q(J6;uFrB%>FXc@CU zFF(bRc`Ml9rXQ~r8(mxsCmi^b#vJ!XD=kqAo{a(9OAwU>p*##=4xFkZc#;r!zH`?F z9d@imYKgEGzZHh3N-!+HeUH&7D_}W;YbLhf%U0FPr19~|-X6k>b+oVT=ZCURnr(Re zpB#sxuAnS0EDUU8>$T(ZvlCvU37?~tg7eOngKJPeAZ9N}b>UNR8`T`V`srcRqRrW~ zyOg%sbUxkqsM)YcTcRJxltjK*4M8d}&8@*Y)s*F|AgL6k`!p6RBFn)j8)BTsyPM8V zEy*A9g761IjZIu$^*UOGADBP-4!N({TC>+}fri9sd;DuSgF1ZEfoO*w+o-K%gK=R=}`Ov7Uu1j z*!~7ud2un4+2#9(4$1%kDLH`=*w$OF#{?c?e1x>(TI6e=)s44#_DU-JmR4&JLw%c6RU3TmpVS&R?j*kJwH9SJ){jgE`5;^ zrxGwyaQ3svQk{c?JVKL-li4P5khIMBeCBNA+-Ye^^^Q)^lFiu$w(KwzzKB2{IUo(ifAa^7$3%zH*KAbh!+Za z9F6)`{&WssPLMudNeDU*7FrbcPJD~h5y6#-wH$2d9Sx9KLr5j1yUpx}YvvW`nDwx6 zgemhj5w0C{NyTn1Hvx(PX=p)*yyuH~_mTq-lY{(72$h;eQC8m?d zsQwRxtf)g$w;BUP>(=sSPV{m=-Pw{wy7W2JjKdx*;32_^FQiHoGTRkL3&;0LK^9e@ zsX2r4cGzrkTPb(`W+Wx)vC&x|Etx4U8+H8mB&+G{tfy4# z*s?(CaB^&ux>uzt*cJj#lx03`NlFu+Z*QKTF`gw$6WbNqjh~+e9eOSIWf1vG)5L|F z53G37?)nx?(0?V}H?TmFBO=ox)(Hr;09?)1@P>P%dAC##2(^NVDE(Ko-f$dz0`OJp z{6l|+h|m1N)=eE9o%ht5xKEV}C#Q*t5y5hOc*ACkUzI0J&o|4@2Q60cxGYnje?GEb z{8dTBD1JOfaYsE>_LYX^HYFWvtnj)h%Qb2`k!)PUo3Y`#zgDanmC$wV zZ<}cFBnN>cvzp;G{V&QKdn{c*^l!^&%s_TLxSHTGJlWk#MpSJ3Mv3WsHQl1wyhr)G63#kFQBw;7f+=+0(Xb&3|4>A42TN zOK1VH(yjNF1y6Dn&?dynD===^h62wGSeBM8+Lo=;ne)>L3rwEIm?_El!P={*LYc=Y ze`MdcAQQ^f3LxD;zQ*8U>{AJ3h|}NQih*kW&j>)#|C5*1mcB*s?Ci@cI|E{z!Nf<+ zC;X6RZkH9bvc2N2h%=xwyevaji2M-Cg`(wL5i{xd!P*cad;Wwsupicl{@C~5Pd=Ln zI<6Wq3CNpRha@OauBPf;AjQq(qxaVRtfODEv+$0=v#xVhPN_e6v&_z)L?qp9?oj3? zgpvXzo!qn1fZb0aL=$H#Sp0vO`tEox_c!j_Od@3TwG#=Eojo(M64_*r?CiZ&NMx_< z?5*rQ;#eVBNtC^_H_zqw{P8@U{y5`x&iT5(pZjxN*ZX>pE1HtX;mM2XrfbI73;oXtKA;R)%4&vR4d*F9)lH#{8N-1{GWRTV?X1gr+bmIM2x0dr&zJ|2RBsEImVbZ2&7)u?H4aaca_EujVmBTh0jj! zg|2Ro>E}T)Ky(I04(DGx#hX_RZBpG8at{`#l`^s)F26>z@hc=xl-@n;9l~T!y z3dzlnH!Cb`7YN2vL#Ogb`q@~U&vqV^H68o}{YK3cC*Ful>JtXA#cz%$IbN~WuNuWaz6h5a2A$RW-fkCY3d8OTK$`2_cmwpB|5#1 zG0h_XQj6$~L(}L9!R6BB@yvyvpN>6bwe+(C_Ue7`^|sd%ZeWIf+2w-4#lqjeSsQ{c zx-EH9s;4)csRnmZIbm6y*`c%-e`qhxmPGkJ(qbYrdGNZ!FTeBsGT6`6X%tx}|AeRv zmg=iaruCKE2Hx90R$}5Ub7Pn%c6jfumQ;OC)so)4r)F2AF<-n!}&^@ zL8vQ5)CN^cX8fj@qcipE&(Q>f%g>WJZtAJVx10-qFQuOkLoU2e>)+gH$!0V?2^`xv zSEE-PN-SKVi((dGv2#>1{alc{4-~Ze2~iGG`<~67AHIS zzdhAjKGo0xRs4lu#9V#pKMT?k4H@zrHTO!p&pTq`lo6{g9xu;Vg~E+=V&52?{=8~h zcmsRhUHTnd2sM_@3Eu7+?_YOi@ppPP1Pwqmrci;rd=>*AX?Jr(YG*{u1Ye!9@67u- z9-%rpU(4zi{)4!CLX{NEh0oThF(m?*p#cgA8gWfJzL>a!sro)vOB{C#V`NBx(yOe= z{9t|mn-wLW-9&lYj4`Rew}pKNs>34q?x40kU|}gN)3bz~!Rq=;Yikf0>uPD0pB&RG zlBjo3-TZf`tfVyX?HevG?jpaPvoi?v`kGf(lECRKEe%W|sb{xjafVIs#w3^aQ(n^~ zWi&j8Y7R0Vm#OikUGO5O3pO@3*6<43^vDWix^oa9*pP^3j>8m1zKu924<4op93B|@ zogRFjo(iM=Oku(+wfGC8yKko0zOe5b>+i(VC*Ds)_Z=9ApQv{dnul;=Qkdya@(J+z ztS=8=H2W5fCMPGK{Oce%)n$8ddt_h;LZH5X|E`qM7p5SOi+cmO-qZmS;HiOQvJ~xP z>|F_ccW!-snf)D(p+8GYw{Mxg3ki`|P>u`>bL2;dwR3#D9*SOlltelWMQd+sySBFB z@{sLVyVz6YuDW{NsMNxnz1<#KzcUZJa>;+ZRp~?>eN~Nw=#Eu)lW?Ex_pr!Haa-UY z5LBUxv+08rc8ZGy-(vra*V}ba>5&RH^HJRkZ15XBRQY-&8G#6vujKZy25CTbB^W={jT``nh-rD=2(}l?hmws7B){ zi=8|B+NJg8Z|=Nm5Mj)=bZ}@}aD+3le{kUH=EBGGV|<*C2h6B=9tos=`}gmaeA}H@ z{^f0kPn4D6SYGA2XlW%TCGC5PJYD*}ysYKq*3gwm5LqFbbDv# zAI23@($XvJ>)<#9wElNrE5DO(jJzNJ`Mv8@LIO@rD_#FpVkRDrVI0x@d+%J7Zb#7i zrOXYP5Qs!Ysf84@t;NW8T{~T0ue9B5%?LzkKoxqd4@sLtWi5IILx68VL`wu?bwKvrBNYva&KZZfs~c zHJ1JgDKo%4jOj7tY4khf`;cW#KqI-p7nW!KOVXuuhUpq2rN#ReIVk_zUBM?G(TM2 zSN{QUFxcJMp}P5&wNH2u1R{fjgDqe>u3K#r8Wt8<c0oq$h|q0OXqtard!2V zWS?&=zOeMIX?IKro!!*TblvOk^`g-*BJ^GkN%Xh{2=)4a?FC-Bj zTS^Kg1yz1YfvE5La{u0!JMyprY}d1J_oM&yC{lcITr}87@);>Tn=14ZsQ|J(0KFV^bP@#PGC2Et zd$oF{t*zoB5!E-uI|EgRgno;;Puy@^|?!xomM+S{l<)4#6gq_9@E zyHH|lTUm^ag_SZurmew2NBdK5A-R+#yyQ#0-t_KHwvOTIexc^2p8CnDOTw^=Y_VWO z+bL=C{$q|;X3et>7(UWC)+CRLnQ%EhR_0Qdl+vqNy{qMB8 z>ApF|wf3Gh(>`z)M#>v;?iz+Ohh>4Ax@8gdiS@kxSRh`F)`dFla@2Nm*2QAWrI)es zN6z}gitZgpCl84`{>z2Z@k;{P!zAS7M7M6KLNzw(=$e>7NsxGpZk?S4j;yS2Zo*+o zOGI^}x;v#DP-ut@q2cFy_wJFBk-&4S29O-*(Il8mPogv2<9p;jdT- zs9Y2-5@ASjGUB**G5zab?|xuo$u)S%C4VAhvqge zV+=E|0CP*m9WE~UsD(`cWlKxJoDZZbPhN3IhRg|s)A92&i17*CZj0 zq;%;TW@ctc8V1}NC-bUn)h|mh%#wZa4KBZL-vp^5G!1;J%FCZ^DOQbjIh6D`mCfl7 zFG0oqs?~?l(yq!N_gzR_1E!#T_v%X7*aX8^^5=hbm*7y-M#y_ z$l%;c>lF@0>-a6wbmcab6Dl;i&5#529^Bv(0clxWy1k6nEt(ERyfi0gVP~OmanT$* z!|I1|Rbe5c9M}X}T3W*4exji<*58jO`6S*fp-<$|BhRCMe?VRMRXT6@B_6KhkvCVD zRt?c`-cJ564Ii#9h{c3|`gd;b{OoLxL?4ztk5b?_Q`$+7CSm9&Gc)nz^#*27Uo{v6q{s;+Z-W_6#15O$pF+ZC4N z+F%DK=qKv2W2#_n{qkl0G|hhwS|dCWhCPN)104W6=KAW0+JjM1teq67u z3i;M@2)ma?9PxUnG_&ln%`wb{74J zks(h0ZwE8{z2Zv_e2!d#fz|gVqr+t3oR}}A54?QvFIlyGwx*}2fjH$yib5U19Gj5E zF^eTuHL^R~%}Z}aqKb z=al3R{$O|07?Y|Eeek;Q?UyxXrse<{y?Ol_KFIQZ?&*nVnO;F!YHCg_ZXhAyEc?#K zLX{Yao!!HkDr-$|?~kKaU=ISuqJU(=E5EU^F)*M6qb`IF)TbCvlGVxW&unf-o7I(- z4O&%!UHFZ_jKj0x280|BJA9jho!}L>6c+M9hFp?_4mYG;ME^5w%RIH^MxpoVkiDwv zNOk;rD9%%?E+e!;N|tYN>4?%~lZmotFJs^t4iSHZERMkK1WXC<%d?^od0pL8SOs-= z%Lw#u!D8gJ+IAy#8v0Pz6eSXA`6|i+Bc=V;``9Qn^jBW$pLMx-@3&$G&2KB zqF=w(_dW5Whiy1-2~c!$sspx4LM4laR8S zop(lp2mvZIP$JGUgN><^=*t&=J~6S1W$*w@U%b~tMDnKmKZHA&(-0Fc06V18ETVo< zqh)m1U|c|511_2rtiuzE3vGW5bz4#Qf3xhq+wZo#mxNOK98S_fp+JuEW^Ef3rkC1 zySfw=73-%TiUdQT91kB~w;jQc_XOeNA7#xP93a~I8YbqlnTQKtM%qTnIA_DVW3?JZ zF--y$-uG5&w0_bmO1^*hybtT%om#Ko(P=Uu1Gp5_XDP@Q7^6p;JpX(ow@hiEDU(7mz$ zq|l$*cI|VYhDBj<1eyLMh3+JmR#4<0>gUN9Wfy7Z5OXK|=+VIys0KY9PA2R&dwY99 z#~c z2L)6ZCr8JA^WFab@WK0$k&!O0t`HP(g%!oudw z8yOKf%frp-M=8_@hyFVOmMkXRKtphS;Bm4j^(uNr@0hi4iPnW8%vLiOmktH<`Wc}e zD=&+VuJ=>&?`S{NBKosk_q`%NdY3fh@rnqQ6+810+>}ykJyO4CM}(kDX@tL08kLeD zk*>(jo|}luelb0bJX6)Fci9~iZ}9Vlw*c^Un^# zMkISujGI;_*IYOH^-=>LKCpW7fu5fGfvBjRt7~aP!&=kfO>(^4#0mb0r9CkUDhk0@ zvjdQNB0>0lWCTK|@v(y@ZN_-il@oM&ve*_RbM^{ z?y;drLpdLloyd5-C*vHVflL+70OW248f(qI6yMnJ&P@|u&11vd-rgP~-{McIpEG^}c?FsM+Gzy}%gadEHTzMZi>pRV*W=)UIg z{|eT4Xjt0o_7|@=4TrURllOC$wfZW37DF+ z4|Z}CsHr#D*w~_oh@u$+L_-w)7W~$$5dOdt-wPt5C~fT|9HTs^p0$0!R0XI*U|Sr{ zDG}RG_CO{>2q>pJY8sZi?1nj5odB z@mpRhDpIQ`(tFK`?;b~twkxWg;=4P2Nkz5ly%%8N$XzYx?XRi*(V<%H2Mr|^`NQe* z1`g}I0DcZq&N)JkPMmCP!@oOqL#HW3(WXN@1NUJ~XCUaZ^C_2^F=pk@Tbgfu1AnAh zaa2v!q^um}<*npRolOn=W{#bfl2cNk!MqPD#5@c7tSpp2QzPxvF6VPhVkzbrOi`wZ zsOV(Ye;>Q&A|fK>6%}Ja;Pdg}ArP9tLar3-rmh^#N=gQnc4cI{2?L2Fi-KA<1+pr= zRr;CG4e>Z-P@G}DbP_570dzi(rxkv>559W0nmtUkow$Sk4}~)JFBM($ti*i`7kkoO zH=ok7a-yawLGOu9{m_m^&9s3j%-|2XR+krBm-;VWoCC?wtm6&7@F3*VQQ*A^3=AAM z@H@Yv8G}Ol3VgoD!SN4U`A9n$4GDJu=n;4jKY#r?n=pyAohXMlN#xgUBU294w{$TB zsC25T(Q$DuHT!OTn?BUEo&Vq65;-|^BgZ6M^B1#znWhEvAsAXxbjcEtHsx*C^+y4;?=D~5oj zmT_>uL9eBx6ldn)>1pNc35*jG2o?u1!kV?SigX+R$*_p4iDcOJ6lK0Th!s5O$5#S&^;?6EVGS;8t zVRy8)wnD9F5NbdLefjcbDb4HAomO;&q}A|4yJS%I-E5~FN4Krtaq-EJ)-pXZ@*IZw z6~|G4Ul`=RoM377=PulMA*R^ZLWqeXyz|M5q6=a$7Ut(SCn`kfINM(G5Qlq*2?`54 z&b9_*X3~i`4zB0{`R}TSF08)=96m_Z_Of^Wz9T{Xn^?-ME|(KslE_#qE{Z{$A&8lS z;boV=BT-&aZtfx$M`tzGE=&wb|6$x0(y?^4KlK)fH#eXDt#6DakkR=Y=l#!XezK8z zXg3xNvSDL!mJZtxzoWN)kUHl({~r-J-mg!z?VcSrA2va3_SJrqg^g{r?&ST)-OvAF zAjyT~N?p(Z0UqUZdSGU1y8NIpiwQLcMj`2gZaedGetwrCUI)cbotoCP=#78z)oSk5 zJyo;g9Q)b&{5@EuK!>}f`rfk%oJwvW>8KkGWaVRf$CLBjud2|3ndG>+KbN%}{Omm9 z@7{)?=q054{o!wDVWVf`hAqG9BYut_@PJ3A8_LW7b7*X7X~|VlAI*QG7H1qxgzjnk z-+&`77b0;7*Zb9zm#(fF0A`L33IZ1g;zAwS0DOJ>CYZIZNqEo+RaIR@%m1o%MUN{f zTu!d%=G!2`thbZJRa9|tQon_~r*(aA6lH!U>F38p1+){!;YD}PF$!y}L^Ci~l`KW3 zdUPA|ZcJ>e8GF+pj)L7qt2YfwMQ`6u-ETc8)52E?N%oM!kwkDfhhnHWvQR3)%#x^* zF#20KOU@7X4xH14J=$0F{(4+M^h<9~4_yccwx=6uSdty3NjN?I-AA#u@=xXdNCT*A z<>^`2+?<)0_jc}e#w?+>3bPryETO;5{?9SR!7?37@vNz$`aiul0-3_7WGV}h@^^8h zvXN9K;aJ$%@Dh~5??0;??%$zpO04ItYg(t`cSw&~ChZK6-u|4j0J`Q+J zva8ZYSmgIni~B5|+YJB?e>oPd7^-7wng?H|XZw6D>)w&595R1lT zwoVG7x#l*wj;eIKov3RtEAOtQ44wEDCKnd66z}wv&6OHL3_(eWi>Ya>Dj6+p42Z^X z&?PcB;m>c=(jK;A5IDQJ(TX}A0kR4k-+}&KNEsS{uqq|FZBJ5_c}_H~Dlh!;6I~_* zdzJ(9C%Ws#PxWWdYJH}vAxHTz zjanwV$B?5}e{fo3bO{3wt~9w)*mW?D5Erw^+uClN{Sv0*fB5jx1EICK z2Sk*Z75N!=*w{7-?zcRHacfNX8q_NY0SEtV_|)V?WQA;bQOl1WnCw|~97ekKaCU(e z6uX8VIIDA@@VuZwt8=nYP}qIP-V7^&`Y-cg@85H$42Y&H$B)~Ye5AxQ?z6mq^5ITv z#x~VG&gzf%-BNxxAR^f%3z|1i)wXg!%C@g>+sI+P$I!YcvhI61l+t{*(NI>@!%`kr z(K=db7^@o2QLkOI?t>Z5Il4SMo6$WdCnvYEvEjl`brql-?P+aY#M%dzkmjG02(ub^ z?HvJJsag<|5kQ?e85 z$MI&8A!#9!WyUUY@h-neIl3O|fMUcm_TRFsdd-j6RQ1NB3+@mA+gg)CU9W!-e zesl!V-(f<9@|RZz%dUqBrxs$TlM8(bm;VS_h)GF(YWAJQf-(%W+90VWnRY3T;Po}G!i>W~|u(eQP|Vr2C7jPoJd z@z=1gt*ueQcSaX#kz*T{NahP;o!yK%#3IrA+*8V<1jh8^iR z=(l|ThWfxWC3`0w%^)51UZ$gP*E@DYT1p>~+9vl3bmZ>cyVW+6w@66^czB>$EMo$o z-2BOt^5zicu*&FKKQT0n?YV(Ys`H8JGTJil&(#}0;_43(LyKWi%9}E3hW=>fqIjrb zUTGC)ZxR(WII)zJl)T(rE-tP;d)r4-5aK>NYp9_?Hp)~ybz`m8OY1(% zA=}4L%29-_6VZ@y(HF4>7aBu9OpihLHvggk4-LtAXk z{^qjBM1MAIWrap=L&eJhgvMn6ZF)O=l2q-b<{x&cgr+S=xh;4~X?dMn=Z*@$0#6<(J8Ye9o%s11ruu z!f&_FEmw@+G(D7_eSedAwe7vK;e4@ONxiCH*MF2W{T^Y@Zsv=lVHkEUS3RucHGSmU z`202J!S)M@P5WgYu+kh_+wbb^Bq1SLwvl3&g#PO`A>p%U&+g!$FKZK(-nF>^SRKsx}Cy?eq zp106VQfpa3{a-h@!_9=jcWe z(zyHo2;YnS@dd#1&lZD^e(=G)etGD3d6)_L$Y&MZ)teI_Toxz%510N|)t}P`0^10> zEECd=2E`yFmYD)p?)mw^^cJ(I&`NV(?V+BkvEW@en`OUf&-*Vd_v<}I6rPOIw$HZ_ zqb-CfONKs=DLKm!pBXo`z`o(t&M%?RZGp%wZ?OM;>Qb8dr#Ikj_2y&L@NaS)fWOK6`pO^}GaUzAOpI*4+J;Q@EO3FzFFNd;=GE&e*ub z{00&p5gD5Gk}F9{#S|&cxfe6pj%vz1Do#rqM3DB!kEsYm6XW7ufJ|&}c^K#n`|Fe3 zaCH#~m-qd7h4_Cp#YS*r*}Ni$Mn7<1qee%0^$H5R5)%h}r)suL@p$>FK8pG26Odck z*c^d|V0R#;??B|()P3S-VU(vrMvYrHDhRvLyn467z6d4n1u>DElh0LEjbXvk2qqokBPfKhvKRbTkS_7eaKcVvDKwyi zg3mUI^uT}rg+RWmyYTJaKDdzZWK6)nK0}01!*-{{DQ*h$4MONnI=SXzA|WYdxmK&E z?{Qb~k=>Swa!`UrhF-1vnY^u;juqHxq-uzUWozK8A$TF^817YCgEN1f0UJ0 zk!M#^b8pg@i&kXJ&dLwVq7rhoadq8nm^pu(di={G1BasnL8){V$%I~~tfcwCdl;Ae z`Tn;}w>QRkhmkPk3wF zorQ%(n1*0*w_a%BTaYSz*QCGCMmZ>q`YAY^K7aDv_2||sBApt!-o$^aXk)N>Mk0` zQBC;;*nuHfD)RD*A@ANlU}K}Ep%Dd5%fIFS0?to`eooB+7XWv6!!>2prXAA~kLmTA z_dY79n?42aE>>c?fiT#uc6J6&+OV*);$FXLXJuDf?YQptgmo2FZ|v}kQzD9^Xft(z|pBQccV8lxdLBEBDJ;E9LQbt;Wp z3kln#YM62?Ax%_dgsLC!2!JFM%-gs5cz8@7C2fTXqfD}gpD@QUFtf1WmU=#I)VG(F z?FJGiQy4&=o^w?$@s$w`A@GvZ8dph#5{*qWpPW8cE@+uT_kAkCe2*$AJ*BLmz~P|) z?6aUb`Mb6B_WDavd<2uonwBh9mT)(A%wuFd?2xe%iZl4cDD7?GKk#}!rr@O5Zn+YqdF-1l7v%j45#??(u`uqFeOIY(h zK5o9a0KlUAP?+L2agkG;OV&c`^Za>C78BgzuqL^q zR}wDE44_R8GMX>AQwUPDS)00`_4&A9w>dBvAfS}tm4&t?~M z@xN6)W6X8%VyY)NtUv6w5V+ikzZ7jhC1dDHhtG(HWRQj+uAS4`xl*o&PX=H|&majV>b|#x3V>K`3t~ z6>T|lLmbRBClBGmT;4}~kr5H6-j1fmigtF*PrM8}CPtM=vMvtv@babt)U6kGzS_ZZ*EG+a?I)c}<1Yy6qlfM3M z`TqXYZi^(bbp8awY%0|Nv<@83!0UgBjKl>*S=l#23$7a|XfFKWZt++p67zeAu7@oq z)*%1Fp9ygr2%$ew`5h-DaH(0jDuz0Dq4&YF4?VLc4E_M+DX6Wyw)VeWZv%rFsAO_- zp}Cg8gZCWE_x;2m=cX!l5}(s!iVHi#V{fWINCx=Q)YQ1Gjg?>rNvNy8?WmA6w7a&s zzkey3n0|;kCc<)ff6tjDVh(ZuO>JEsm+mW`G++fs{TG_|G(8!WUarLoLlIpYn71m8 zKxL1h76QjxdV2c94X?ZR?{}~5Yn5n7;z)>>zHwAR!qH5`5y2M{NHoKzygW!q%=^Ru zpv`J;Tp@S|qzbyMg6_l2Eaj0MC^nkdgNm*04!XD=g5R4Xl|{{ztIf3gr_Tr@WyAA1B1=jVrgwc$(}C{%8qVrjG&58%M(kxA@rdw?Nkh)EDvAxRmi_Sjv# zm$xe^{S}HRYab`)Rp=lzFkKLRN2dq)j=cQNG)?gojx(>r+f|ZyqfQR&>~xX_uc^=B z<2t&9%Bhnr7Kcv0@l502?{ePKD%EZbODax6m5)joJSd}-UvVkH6BC1N&C;$>IPgQk z;T|}m0BYdgxB-iEG-x_pU0i_EC@mxN+apF3BJVGJp%@Vo-U+*#^93p(z=GxFL#A9{ z=PN5Eb##1e^hLvbTu`*;s^Pd#B0a1t$kqExzeaz>x~Xw?`gQk4Qff*`SrP0Z!y`f; zvasUbbYKc&{5KD(L|C%g0+eNK4RmZjpM&+uYc%e6xVigGxo$-~LjwKp8safwVfu62 z!4sDlMoHqB{Hsac-Mw*_o*vpdIAdc&Ls}{hZG(gC@axX3k1~#ljYE&Rx{C*JkW|@( z?%!u(`fq7Bm0gSgDJ%pQC>sOMk2weShYyqcbF?weM|TQ#7GUoH zL((%nJt|UCGp;0$w3ctpsys3UGR#i=R0JY>AEug2d6NIBlM{BszX`-rASoD!!wZ2T zBpd>a_WSp&0v5?-c!m_@q|G=A}aTKE5BBZh`OPAGH&r*HV`V*MY-wLHz1NYbbukGtJc@nKEZ)jgNq{J z#jniF%&R7gmlptQPxp;fZ&_t!WpF5m34Ub_U_^1CQ|BtBWMyXy2?lxtHB8OTO)Zv!-@aX&X5hyoL3=iIXhe=7 z&l`Rn8xt|lHjN04fW(L=PoV4j0jfVD$?@03A;NsHHkg?BWC>+D&X$E6g%2Iu;ocq- zumxx#Nx|*pM*(axgP$2mTeBXT%FuN?da z^Kj2uQ$IfhCvviBdnKclKV3J6gs|Q2@D|d#s^#ZQoXaw}t6z55o5K=0{=m@~FgY8W zkDWgA%>6QQCRcaHty|IQ2Cx1|dH|pZxa0Ejf1$*?!0A(zm-pT2@u{eA7>Flg1DH%BYa(Q`uePx9c^&UGYGb3XbcGi$wrC#;@ z`xy|ArB#Sd8p~rz-Q&HnV+{NQyxdO{W`qRX7UXjVs8UkxEJV?rm0GICQ(1}vI3Fb` z-hBI*M7m~bW8ZV1l|)(vht6~45IA`<9WE)P`BD*`B z8ZWR{H#bFHPGb8uU_6+tw$=4#6Bm=&Y7ODMB~2&Raz{j@sl_h?phKtzbKQ{%nriU1 zlg%#xhk_h0`adWC0F{hbqMHth;JGo(p-hP6!{Y}%iYJ{3^5K)U>wr2`wfDei2!?Lv zS8S|lS@+bBiCT^pATyVKDR&!$#9;)99&a<8T(rxp8yZ0`dv^`N7+Z3+m>6g2R*674VOsh$A zK?&fU(@Yd48rGJU1(P<8eH6<7eUE;p3eLc7cMu>_sN&NVC)6_iJDM=eX zr)H_z6$3Lo{KwcBMCPVzO#}(ZgXRmkT+f(y2l2Y@^5U zVn(au%rqS*-H4NADO8p>XJ-7OHfw_!qF?JU>8-0zFHe03Ee(_B#*%d`wkAUN-r+Zg6KQo zN&}7|6B82~n+s$>nK}ns@xU&Not=QL*op-)3>K~Xb0jE=M)Ge|nD*1jm*djk$=}ir z-ly>8=H7F4srtw)7KKV_uG^E475|lW%~JGhrvu_#_vN-REzVJ$N(aoHnN8V`4;RfK zGFeIrN4N_q#D>p&GiNXJEse3_Uw(bUqXn{x2LdXHKOGYjr0FRDL;<=vm;JcDgd(7W zLSclB1MCpNIt)%>|T|F8JTlmjVH-)(~T)XNE*mqk8G`;(+$4KL6f)M<(N6 z;_^|OFU(>3s*gtrny1jq<{oFD;op@u#IfJk=OyB6IwM;2Y3ed3My!KPk#kE zC_x)Iw%iHcBeO=$!T4I!dOc8v*-)5E)cI`wdd;A<8^8P`JHR2Z5Z97Pj~5S`CMFW| z^Ou!U_|k>kHfIAcu9*0l9b5PAFvv} zb$PXo@{HAlxnDv8&14UF7TG4*m8{?^4sB2q7;}Vk$y@qLG^eMFnw|Ih+oHeQUf-76A5dSg7+m-iA5S9slH-jRum1K?zYKAHuWZ_e z`K&8DNSGQb*Q+LNpzQwpSy}t|3Cfu}Rf(+* zCgOOA${U_egdV{bew6f{PS5KV?#C# zgeEYBB^SP^ef8(RD0;YnU)tJ^z9 z2k-qfSbP6`^RASNo}M0v6sW1Meh0SRXJ8;9B`q!eB*{Ed^PU_-TpZ0*dzeRQHbhz7 zam5cg_-mVZ|APAvuSfqDHZ-I>0ElPM>fbX|l9)$O0jtDPW6aQEcnRBfz^yydGWAaVk| zNm;#W)mD!WRX0e99tsN!!}=6bb&r7rJarp<=U^Sbv%C9Qi?v3bCT+<~TL!1;UpokK z*r7>($phvSQJlGlYQt(J1qGIHr0(B;tMcZ*CC{U=3Yqe!YMYObyhWS#!qVz%;4e5d z#*KG8K_u@BlfYJ!+Y{OZu+{UsgeGa{9qGct?fgP`ilS#j9WpWV+sstmN)G<*}6=r?$LxyjCzx^4m=y)3m zTTxg(V?gnhM>F}LUgDS00=KCO$=ePU9UY2WBw8vSb!B8`IF%Ca0m%P*ZD!Z8Rj?UY(@d(kg`Rf&S1@Xy{Pxfc=ephlOjvp5!S=SqUhfV%;_r`SimJ~C zI>l5+MDD5asB1|Q<9>HLCiA8utzh7KM%3{jW!vps;6+(b3%Vpu4i7i?h$&Z3rJk-X zIv21FFl8G>xV;NaLh@XoL1p9VN|5`@3KcL!jVJv1}~ zKY>!pqoe_Ih^}v#wv+r1Mlbkm2bCR+NLgbsf{3b{o4kN zhb#>Z36XVI*sx8NXgt6IgUe{MoTet*e82L#y2-VDyXpEkV$2A7s%|H$8dWrz*%0j3 zT|G%jxPkWo34=Sk=%Im~r6u|DniyEZP}(=Dz?i3{1vnxA;{~7{TS~tyWbKoY8!8$= zN4h@c)|(q<7^uv{#e8&hbg$+R7?KWDTaI;ZHZ#rT<*ydkwYB5n0ccMyHk8zPQNw-6 zfpB0pGMp?+ZW$^J8hWjC4TQBn{V149l3LgFvh4MQ&qD^BMA;B4ipPk1oY0=RmG{cf zV{w?xKyc~iR?}vPcm4YG7t0a(pSMiJ2~qEJ+00TZKVJ#;;0eM=Hw!eo!lELyJJ5#O z)hsP6ws-6s5fDAszv>$YJ%QRUhtD`*V7ej6w=sFPEe-191e*Ii-o{)U>oFW@h}h z|9%C+6ci4SNuK+|_YR^Ekg@l*oGq}|*i1%$_>i&d)P(ts?V98=P)fv3h9gf-Pg{0? z(g#v&ZDy!|GTHDSc~{nQ;VB^zpdag)YFlc;W(azNw?lkFq|(~fWu=33WbyIHtm^1O z(y_KaiR=2$75kc%>%A8mPL`I;IX|sid|ZI`2X4=$Wo0#3*p5`)yHs>~uC8M+8fysV zAPCkSVW65(($Iis)A+>DGnX7-(0<8o@^Lm^-&^6v4uS&$Z7nq!Sz(I_i8QH41UmR! z0N?N-D{JuU*YG7kGY5P2FI80ydJ0BHnal+RsV`m-6VDaqE-oyC`qiiGNDu`Ro;jnj;h25;v^&O;ey4qQAG zqF=iFTGZ@782PVK5ez`8z*179F^pAg|3|aoTWW;I(BEGLSDZ&FhvZmyKwA1WM1sL3 zs9psp8eR-ws3H7rn|j)lxxh5K$H-Vf!jET~RK#k*25A+Eri4^Cy_}^!`MSiZh9|WW zAzFHl;MWN%?I(no05%$RV5^Tp&(PH+OaAiDANb*V;9-1r8ZrM|S6<#~{@JgqYK>l~ z|5u7@Li2Vf3Ee~3E|+XMcjzvaQI*>}1?!C98LIxm+qHHmFmPL3BgFgzmMgw1ISKA* ze&E0D_b-SRQVP!>G)!LrDKnMdAuBJ>0{Upbo%??6L^q9`Q@(o%(+pmD&Q@1d2~b5` z`DlUM^JJ-#b?xS5g1)CExW`6bEJtd{@BFe|C^4a4lUnfYl+R(eYkE90jXmg4Vp{@Q zx-6!X<6~HG>fql(YK28(Bsx*{)AhWAfh`y61lK^uG-M(Hn1%6Bna?naEC@L~wVjx|Z2r6iCF&D1; z^uavMd-_^h?psYAj~@qRF@Y$ala*)hzSuE=x(bX`e0IN>!v4ed^O50+S+32$ahru+@-MB&zZ;L7{%a599x{~;*>(R_)5|~Wp&$JT zZrtQ#iY_f@=l;b3vj;O=y*P9v;`0fb*WdLdLYw#DLq1bjkLmi_+IYhW>=$SS9KXP2 z3Xmx<&^({U(p<-edm|eUA-W?ooDs0NdmNq>(19{(n|m%5MN6Ch(#}ppjnw30a(sMT z(u$rAJSi|ouhwEE5IN&OV$;R8baX1U&R}x4teG;^W~`}0|K5ZZ&iM@O_~`FPY7D3M zsX=TOrH$vajGT#)SW=~y<)0sA#1u8KeVvB#Mut`K=@SDz1Z8K>fk&VF-mvWF&z1YD zy2*trPy1j?bOAP4leG>eP%?i#((D@ayV!-l8ml6*f3?R>rw&`Uf z7#JBhe^yziW)+)k7*2m^BFL;!R8dza4BprLntF}yV{QTDOlE{@=;bFGXtm-jz$)3? ztPr8{X9{D1^aeCQIri6}RKr?ob!`pK3yJo&yGU4Z|3H6uWCT!mz#EX%e&Rz?RMdwL zt=qzHiLeMc9s9r!LkoV($S@+a1dTt{>Z#b9YnK7;+WYmZxMiT)vd=uIXMWpr@S{*4 zNA)#p{cnGFR5(~w1tk5xDO&(W`95Y!r7ng!ckEMv|%rUKW%_#o1h5hC385Uj#s{4J8 zFJ4ST=+j;IjxK!8iVDwDX&LhN5Hu4K&M*CfC=5w{Nq&&wjT#$gb+k)=S#T1rhj7}wsdH2?xXeJfI6Q5`DyzhA*@qoG*0+bQI z%lkBbdeNW8suw{95&le4Qu3+mQw4>-*4YdhXc=KAM`jYf-E zOB$=C>kE2Ic&gS5905$^<-oQ{p?0B@yH!dn_dy=S zF1H3v)hA(pH|}>xiOfDXjab6}h=u^%>jgY*m~^&$4;wsxjmr~PyA;oD8ef$eEz6HP z9=;;K*f_epD+x>#In2ZH@gvn<6ztSalfAnqBs4W6hM;+KoQT4}z{Rx_^_Dit;7Z<| zj!I8VO=)`OYvgR>?CDw!3#^iCdM++Um<#}QIA|3QtYE^WXv$<9CCPv$#>z@Qexdlw zmyi&gccY$gk|!s%5tyzOl$1PHUf;$FICTF7SNn;<3pS~0z5Syudp|?NjFuKLn5zE| zOVIGvkxmjv#IXs5?7jCWg^s;4Ls4X9k2saBkZd932qAk@6d}pVEM#Tx{lCxi z{(Il+xvsbCt*6fU{l4G({!B^9VVlz1ntb+=9ZUOZ13Upu0m$Ei@t3}FGkgA4-v0gF zcxmtFyMKR}hie6RL~p!I{%0*4wA64mK^LUUYg0-H!RtTTFEB)9f2^JnMMp(pJ_%l7 zvFN+NHYKUd9d`N00)97fa$15(4W`MU?LykoQ5k0MsV6U9Slx(IwLtkOWeMBf8uZ6L zx_4FZ_u{NvJH860(z8sX&99KB``1zsv>|0MySawqSt8O$OL_f@JVxsXL(W1KT z+To>Ya{Zh)ai7-GhNlwpo@}HfZr-p?E_j)npouNw%MKA5^!rir@!yfG+{sX(G71Zj zOLcWMoWb~b6tDw5va$IC94QFF0G?LG9JMJNwPhava{x&QNTY&Ib6AeKq@@qa%`pz# z*HW$~G0HvOl9}A^Y5r~nzmf&M_>;iEH?Paj!bbpJEIK+mxKuj7eDU5Hap<-eQNDgv zYRS0c&e?sKUb<2bLVK4@%heyTAME-z_!`*6Ogj&T`qy>rHfAay+0-z^84gi=d^O}O zQo62LAp~!8^W?X;^s=5jCVNx1t0jd+u(J|JW?NMzfx-W`_3u(w=o-AQBiA4xBot0d zY?26ie~6Fjn3l4%kQbFi)6?JpAr~$#zIA{}Nl#53Z8}wScR#N@f79|r*A>&(dUT`{ zw9^l2v-q!DKl^ohd%iI%JRV!xR@LY@`kLtnJG?WP5p4wIw`$x!n4-uv?9x`Dx~uGy6I^zP11$A=H` zLH3+CE~*Iuld-f7f=>{J0HQSe_bNkn6*Nccb zQ2-ahzu`ueDQ^U85*H4L`nu{5k4}#EAx0JNXoBgjjCYbwVYorQakutpmlSp)xn~R< zJa`q7PavF5KB$=KGc8N&ARYY|G~GPEm(;x3D|c$Ht+Yha=z*OjlDxs#2wm8?If`nioX30lT!w`qsI6Har*hBwk+-Sh! z)&kJd1A{-ewzj_Yc6W4K{ZE)n98CF$*p}Eib_nXmKYw0`Y1!YefxUEhdmA_+K3-m{ zD3rQ=&N+g?NP>BXP2UEWMxPI*rRQwZ5|Z{{3Qg|*tFqc?HhH_l)f{X&uCD85E)4?% zBb+z>B=W)PJy9zye;UwQudR;|v`@gjaU>Ks<;~-K*y#19e>VNER2aEd+bz}LM5!x8 z#7MSh-=Y7_j2v2kuvQz~zltYyoX)UuNJ{QCKOy3OU6_*-I=2t?1nrI$7nMZ~s8p?C zO$Qkh6B85Q!{9Z8&k(4-%$UM?(YCowAa_skunpdk3!Lw#PoJ6&*CF|BMvs1Ai}pp= z^nK_1#l;28^o&Okp&tvg_JLQ>t`~UXsw9U1-4{4*2pI)uikDAag*Zn3j8HZA*cU0! zSW;*v^eUI)af*KTUM$;_UGVxv_3JzP+L_Xx2j5Iy(q3+DV;MVEyz4C~*iQKk&5HHZ zZGpP2c9yd3a6wcECnhW=0;1)}*EXFVx-if&TjLhDy?G!HV`Y(ajRLv08Bd(yNRPPe zTTjCp^#pRsWi2LMECX^&YhJF64K9ze_)k4id%d(=HQ~HE|G|_OPW2CccYR?f1U^N% zt*vn@S73yQXnnx>6OxS>on^pq#In=28G>^pc8}N7J4?>|@86<$G9Q8X;m@s)qtgs% z#>IIE<5j*NKb?r)Oa(a=G`eu*m7*{nV7SI5LtZL=a+1u#XFR z6O4xTR_dOEEI<|Gztz`}o6At^S34Ovmk#2J@{yxg4vjy?$APUkiS4kh8qM@yEC>q| zL_9+wY|uoALu_U&dLov7yRJ#vf6M=HCX{YkECOxSN}!ZQ`LfM6pshceN{{G9>4<$O z6Z=HP%q1U6g7zpOMKW|P?kz*@r=mcX1qxO+EXwUH?$YGS_Nho?Y)EJ*q-Z`pPYXL5 zK+aH;qNz!0zT7u9Ik~#J3Sul69N=1p>;PVxo2shB2xN$2MeMcR_XR0=_kF9=JEnj8 z1lrsGh5Xvt%+vi$78zH6M>ifx-mNBZq61pF!T4MdqyLn^(Zxl^ZRQL7MlpG@5isWY!f z#bjmu=f7vIWLT7s;P0LBIBvWhZ< z$d;&Us8v>Z;e~n$r7xzYmPf1Laxj1YPSA{3?dek(L?Pzv>AU7=y2tV*v8X?r`_jK; zCWf0>5TH!S&+z?mSGcQ8Rzzg_zw`z_;OV5L|IW=tt5bEX2L>9#s)9n)t=~#Y<>Zlw zG!qFz5vdZONyx5G6N}F=%V73nMyozl+%;1AWk|)caOV@F12N(gwIZ2N!=wK0{gPs$ z_9QFq*n>9-jay5v4}1>=JLGnTC%`xm6-64;+pn5)vA8 z=g(c}K%%Zlv)^U;E=fp?M6%h(ONsM~i}HT6F9&hT1#pQ34msETK>gf!Zrh1 zpDCc=dGHa31A|M`u3r-*1L=Tu<7m(IwNt^{x7Dl)tDnNDn3k8LN{h^@lY=VVmd1*f(RUXM$(Qk}khNa_7ie4Mn~&FLJ6}Z z=n-pcrD25xoeQLts9``UlkX%AkPjYXwoX0HDGC+o8@1En(4 znUK*8K2v#=U`5| zNOke}_-H%(Zf4cGX-9```a6Hk_hLSSAx9P^UY}k3%IBJf!atcNQQR!%jB*7JaouWT zA%=#Ma(yOVr~Q*EbPxfQ+yt4afF5wzS_zxoB6%tNOvsvSxQue|~=@Pb!8ioSa|g3TwE+QG@kb_aA1)k*P*wFvUS|*c25JOw6L|agUC9f;rgSh z-9htW8Cr%{AN;2K4Y@Vr&!`bRp$f(*`o5tiz$U7A6}wuGj~#E+zrkEUKv}A#?>Z+S z9=En=!8BGko=0^l=24B04^Xd4N=gvZX6-z&v|Vdn{Pk-%^lpLiDTOZ)93le*SBDBz zB|-?EC|0k1|9}^g&&UB3P6yxXL0o5i}3w zb8(G<@Dl8)(0iYJD}^MdxxlmNI3r*{_z72Qc)Zy=T*D8 znQz@aB)=X)@%O#brw5(IJp;Q zLOOJp-R=wP#dc9d!&#}u0__+UB9M$fxA^=t0z{wjBlLHR1pS=hqJt+J^`Tpvf5isLC5aSAKx5Cw{ zS9y6K%wn-GV1@<9QfaA>au{zHk}b3aVlhA~#}c^r8zN%T>%7jMIb$6q6#Sltup>4) z*5FR#5d=&vZhYn_5hDvdeM&|K4GJX)ViX~v zCqf_yfkaGYv=9lJo$S%Z&jlzEIk+n&#~08M!=yysQL{M*F>v$draZh{VG$9yYU94i zNdaUle0sDDRP13P@3|gb5Zb#NfWt{c@aBZyYQ@Ko2xT~n{2uhO|8ir$@Pb1K#D3D! zkX+d_IjMuu*XjpfB*^g~`E+!26h2>&5dc8JmIjbh@7%Pc_Nrb1c+?$-REbIwm&K#D z9i*ZhrY{zSbjwHlnL^ekKSw@UJ z5)8HBm(-Gz(!7@|hWzDO)WIizY}CVP=!fo#;xm7UiK3!n zz>NN;Dj!mM_#l>Xyw=mkW~Zo%99~o_WzbZj=;?j8<|(!MgIkQCzv0T2XVd$sJq3sJ zy$V|nMZfd@G7woIMX&@!2tzUO44h%+ujbGKS({P{y`U!&UfB*ggzZe=C3euPst0{aAEfhTY z0$Ls`d-P@Sp;;A>D6nz@CkTjPV`C%GtHIYX`O?ep?#4}Q#cLA~LX*1e$9&!>SqBXb-|YhUD_JmN0k z*3UWIUG=IlhX5;JI(&_=jRKUzr%+r(9qadjdpg5!;{0w!SR0OW3H|u z(!=un9rDjgvc2#Uu8!``r~tIzd97q?cNawQ1PGGbbe|Q|_`THdDWUq;jqh1wFdnx& zCZ}#|c}5Q{Rg6CP%*nawzS*tSo$%w`{JUZ33!Fo>rr_dI(-atggUrm&&ri%i#3VXa zc4O^NT-QC(B6+P3ZOrI}N~t%=D9{G@sg z21SfTbx^ho=0^RnvyB{M*Gs)mn!|jd99OINnxW@l$UXz;Z|(%-?s0StxlX=xlAjO|zq=^oYmHIg?7-){}L`!LPcJzfz_y`>f14QfHq ztTo$Tpf8@iB$y%`0{`mHA#D z0B=qr0v09B;`2@>EJI#gsLQU5aKiQQ*f2+dBEp@eZlfz9x)~+uf&4BN? z_v_ay*fum3EWWL-X)RV^px*51;pXB(B9SQKiCH%aa&nk9D=RA@f(tavHc{S6!YQ}V zPjL5-20Al_?uxTxQAonf;Wb~RV0-vfelBY!|z>0;nz}wR$k)ovX<3S+ZhoQ;g$?wBl zg_=MLHwGLM9=(@0`E8hM+ZhpoN+i}!AcMjjfaAHj3uJq{%Ko_VNu|SZeG`tkfdQj~ z1IYA<9e1Ad(MGAZ&pU4Z{Y83rwd+x#`tjd6ia>>UVm;Wou)!DAxp^;ii!saY6xQso ze+m2fBdvp@=b56?8v>i5l~oZ08B6VEKzwDo=E&I5*tL>IVOUAew+=O}E5J0;+0_dl z5Jbg@OGvON1RiZI^uwx^lK1qP1{$&z2N`?m$RKxd9#UF>aR2e+V*FQGA)&sJk-qWq zhgD}uq(eltRKnO4Su(XhBl7a>@p)~q=#-CKq-8jS+2Ovc_3x>cs+3eSWD6Lw!F&jd zQMzT)v`-*@4(Sqi((e!5b8)VB?yH{U8{6}*TMTZaUg%|2zJ(wq>^R&{9`*etnr$;7 zAtlGmkZwIwlte)2tChP{H-oW`3eWa-P<}a-BDaRw_)_*Bhx=>;8g@|py49G&U&Ezg zo2GYDJhceqPGEO+t^zs_ivmUZEPU{Qb-=R?dqnvNBMnU=3=wg0P-kQtE4TB7Jr*+D zMwTDn8E9x|04Y3V8GILwo|C+QzM_g5z&B3dQH5}-tHGrThnA;>A3m&s#tlSj!QY{7 z4&HK*a$W9%w;19w*V`FjlY{jENCn2;@h<`6OVbrGu^qm`$N8U3;~n_E2y~ULyt{Li z$BVqM*bjj2#YIQNbuFy`kKPb1GeBCVYO$pnQZ762zb>1I!y@(U6n` z89&?+V=xF04uWRaUX)%%S)~t`Z8bDfgtf}BYF)n)e`B|n3->qPAcOsMHRz;1KRY}WJ)03U^K;+O|v(??v4%)p7hA9 zf}E0$?Ag_f;nx`%{A+0wT(=bvL{=emM49hzYhJiafz(h_13}^?>KTS;^&1xQk6!&(ZCwGagWOw)8NDtZmYd*(pYn>zvt(S$Y~h z4Z`w6PL)aS!krf{Yn~G$>PKaF^;r*9c^rMnV3W9^A9ZCj_vWj^T^5uY8#nj;$;N=b zJ|Q}~h5a*U%qpGFeLSnJrBipO7{Fq9<_rw9uAIM;rgV`qSjSK`1JMdO4ysr%@a5&^ zE^@*sKo3F<;7iqROi6C6o2waAz{LUYzgePPRo8K@KmfPQ)j9y%zAcCVYE4iX3@a>L zj1dv%(d?<+jebKbK(BRj5qk6+kSa`jPd*&`SDU5@sK^JN-j}TUyu8=S+B|c z`~wabvd>3_0ae5WK|%jW*)?t?amYRHI2oM1qI=bzlbR0 zs~;{IenJy|dPhPg7&z&^p=7(Y*u&rd?itd;gP&RI@fOr-)912fu{G@$xq#ufboRzP zn0`W7`D@Ih*rH0mP(7JLAgIU@yjMcm1qJ!!$^ikWuI6B64W8Qv8KagSq!xi&9}YS? zK-!JnG~=*RA?JkXIK~rZ&nRIGDuw^d2dwrwte1@K9ktvNAS~3F7F3L4>-4@Q6dyjS zbyeG_VRXVMGx?Rc$nvj>*>hCquBKqo1nR~1y-OBsQHZUe?d80 z`<`Xw^x~2M^_kaQpEs33Q(|F}(bN+SB62|NO%Hfd;S z1iCgs{dH1M*EK#`c-sA~YPN%%Eg+BG$~WPPhM-l?ABR*gX@|eh7+WeQGJ4X!@s^|B z_z1Bc(Ulz1m1^tDuOp&LzuxWa={fl|)JVg`G!E&0&;xL^m9KEe)HFNc*8}m_U7oX@ zuK<7f@IkOWDPy)3eN8|>BKekYXB}^sFV!pX)&X<_LfY=T!V4VvB~`odr-de(qb#q9 z^yGB#5cLK%9l2R6Who^P-x#a5&QY&;9dyhP;_9ZW8^)f5`m^l|Drf+0L1(ntEwQw? z2=@q})!S4Ptrjn;JnOk~uN8=;;EEa(ixo?T>X-)}>w!-$^P-bhMg?nA-hF9|$?m%` z`!}zG{N?Eo{$=KnKe71pe#RVtzxc)ug~D98F~DR4KeGQQ!*>Cn-lrLAqPyj11QD4! z>PWN|d1&T9Tx$HCT5H?u*FlHV&>#nqCU8~3OdA*&D651Nc>UZas#yF7N*T*a&i;x} z2W9tY{<|M&tl~J9hfCcd+>Wt>N#4J@xOi}S6O=92AONMPY7!#96Mnb^Zl(&AmZoIC ze_(GpU9tCuI}Cw8olN}bj`+HvRloNlc{t$gJ?&vLCL{!w@aY=zhNP{ zkaty8E_Xqo@W32nfT1SKRxJ5tp`OzRv^{8CsHr)wUKJsMM=~<QLou5?zOYeq^4GNFj1XZmWJ_tVt9E8O26tbLUb7LG?= zOu9V#)>+u<>^+kAoAGuMWQ~j9{>7atvbSKof=)p_4C|!bH91h@Ns3|-OK1DY++sgE(BRav2%(#*kFZWw5zJZ z|4ffsK8^>XBD|#2AukbB%y5a!0hT}Y=8cYmMdpFP5Q(#IgQpkALp~8Wgt~SZTmb1_Db_G#Mu+ zCrDeM-a^OJ&#$g)R#QiZRt^5QTyX2GTe_SdIJpD`1#cS+fO`io(Vje6o`kP^5=Q^g z2bC^?OJ+fbUO}|<^bx4)G3QWLoc_%lHz*|oK{^N7VUP&`IY@J|B%y44eBHBda9%%n z@PLSfKXiklefafXuemZjoI6y&ne#1;1v!5{tqXP%P#DHhWA zfory}*U<+&|7sWf-e)JIhx^Bgf_MJ;c#Q581={P`YirvZCbEAKk`o&r88I+2f+V5g zi3x}zisKl0@VH?zXX39iMlAj-1kD?mnu7FGP)ce9@QWL|kmXI z*7a$TpN0}j z@2xH^Il<@R>jlarBsAEk?&p~Oh*5F6(pw5Aj|JH$~Fx1U3TLl zWzx-LX`^+1y9|I42F;zrqQ7W{UsFvBB{`@)#5O1;3(Ez`c8NY`k1-uJjqRHkOUVdU9 zk8#2123E2FD)dV(Xgd)Su)u9?q~KTNcIB}Yg?V92q8kX2Jm%^((dx1E1d z@Bse#li|C;tq^nyZVa%ER41BSTmLqza+de|3pxW)TV8GVV>4 zHsd=2+NUUPp3N${!)FX>#W$YV^5-+srP&J)2UAA8IFUf(Ux)VY&YS|%+mCL;@0b%S z?r!{yT9T1tlde zczx(G#OjrU$Or1LK<0zY`t#=x^k$xp?f%pwPrZYoeo2dJr_W7M2c;$H#-{r9UNkVG#=R};wu7DL0NZ~GlvsFVBoKYz z+lh|80OuhrQZOLE=Xrb_05?cPHn2i~n}^ zcf99_14(Fdx(VK2oLNv*1UZANU4S=jzgjHrDTWd#m~gvO zJ7KHp4m7-@MW{io<|+A)fZzm{#PmCQQU53l3@Z8HgmS^+FJGvMTi5HA;@%q;;_hd&c1AE-Z4)6_jWL%DFtA*h7jr6C*VoqrbPiwV`Ptb7 z-BwpPNa4`{V;ux{I#Y}__*L~;8C&Mb%OuhyKFB)x>|-`5ccOgPDMrzJuy}-A) z+g50H)uC7Z0>O_|^~5sfL_~!Dt3kU77&gAq0YtgLHZ};ZpBJc~+T8cfptx`W@3N0Z z38!|Gggy~cM=t#Pmj(?n5Z*|vlxU}f0#XHo>05==+trcs4x3Uztf;I4*449Aw z+n1zxs_0hz^pTgNer!ZU00^mtAv`+P)zb}Oe3CRLiA5mTsqa;B=I5yc3$Ci(86hh# z)Tbc%AH&kOf+Phauo~i;SmRxTX&-)$FD;ukwzeu7ei~k0MleV{Jr8>AOSHu{Zf@{* zxnLWGVkL$k+*|m*xY*vgoP?Fp-Mesz0Hy_)l(4KU81f&l3>OqEwn$8JByn(Fk*!(1 zq46fvTtUasPhb8hZyWjX{9IQdzmZRaJY*Y!X$T+%?4*Em6y;x^2`fQJ<-2QV2SyZX z8XB0r&JYtrDn@Kad3E)7V1{64myvNQHHLyCjweR?T2siDBtqk&!M#1~*0(5Nx?`rP z4RZB2_v>L=f{z*;7GMUgiIyh9Ts)&>^)HX=yTbU9Nrg-|n)O+*1nN@FTpBx+?3jE5 z1p;##nq}eMa(DOegNsDbai1GzexIwY&*r|MMlBj0!VU&nc1;L7ByxkB*U%5Y;ZEFM7N73qfo}OC!`x{T5 z1jcso-FOCu0d_^Sdm4)7;tZ1PqcLv?D?J&>S5{Zqs}b22|Q?L+8)B=RV-Z~w!62v}xcHrL# zn@>!1G?+7>Wgnd9!x_og0c6VkUydTvd2-mQBHBUFx!~U) zBYk~17iCykr=+FmY@shLXp`b}jRMvI2x77?_t~ZsHRu;zF8^k%b*C{cn=3)rDtOJ! zeqph+adtPbu9W#;`~NrdCDk)`KHqsg7btz|X=Jvqfx>=PbAbEMLF24OH82bSZ~*%o zLrZ5B|2YI@g~&iXA7{=?s)jb!pC5mMeJj5oZk52&4V%+uR!ZXX5l6x6xV&?Wt~CwV zX=N3axcGPg-}McamzFY<(2%J3NIKo3Lz znPRD0V5`rNboCDmWRs9^03Rd1dvb3NT;(uFg8bml9o0P^`Q&$N^?0WbR02MGMndqr zer(JkBfyXX8K_`8vv{9t0Bsw`0eQUY&z{rF%+1nL1IKG`XXn>20rw%CPEVdF0TC#e z!b@|WsJW}pTv0Lv86F-p@$tSM7%v8F2#Dcz;(zS~nb<`|4YjqtpvkqSNKnP(B%s_J z!bx+35lEGvd;)}XGM4KTd!N3CQpj?XCJt8mTD?fTeJ}AJBjvk1N{@gqpZGX<&{VI4 zW2RvN6#3B)paq)oML^lmoTpSeT!L`cz%vyrWY8Z)5|6_r0kqu9hPg<1#JfO8_xUp? zJPO_2(E##r3ka~#Gcz%;?Cw&ahzYDiC{eeFGdhq2tb|CS);F}mq5GddOHkC0eE)t= z5Dm7FpgFI`MRT{~ovJ4I3UU%?sHvqHpHx(c_4k7VYpl%fCODhA?crf*o13Wc+AXaL z%*mX54Rs{m--2(Kl`x|!c}H@kQ5?sY5K=VVd3?ehlX#Z<3ko-@4EFzP!i z61IhPQ9{8H3e!1@?|c@%=^N${-~;%0cp_qA$p8BR90w%bMn^_=!P%#yGYQ^Vcw6?T zfnF6j`6B={c%wrv^YgYW4Havwg7+mR{@0PZ^eEH|NYF<~`|klp1y3Zr8IWuSTjK9W z4}N`wqP=xc6F&?J3IdTo3PNUagE)@l?$@Tgz=ZF;!jM{Uh>7 z$M*hLJMq-AvT}HK=oc5w`Hg_FgAqaNP5=P6FmHgBAt)I9%!GtF&2@I1|3y~hIH>4A zxQnfx9(*_e+q+t0Kx}VgqkBsqJHqu9*Y0W#bSmV2gL9v|YBaaQ1fnPf(a^*e85z&f zH?7B&8^-|z5@a9!SlW)rHa0Z{I|4EU00`|{w@i5xv$Ox;r@tL??2B>dYrMQagFIeA)PXgw4{B>tu zfOnwJflm&<=ZlG{p@@|w$%#ll&~vcZ5vmLI^;uh64(nG!nnlHtzmJM7(ZD@8D}@efOm)QNVY+FosC-}yw)XJ&*Zf4H#2nWt@C9tZVbS3gQXw2S zgf-J9a$h>~^749g>@457v5pT`!qvL1O{J1Q&nYhE7ZHIHR}oR(Pxl1F>Zf*m&R7_S z>$jl~seRrAy(uh=pO#GJvl=Q+PsxK}A7s0N?e7hNthKSUZ0fK1v}vvw90cH2xBXQt z(Tmle-s73giaz8uau(!=oL3WmR%B*a8Y9WX^as+lA||E;_PZk9e~sYS@RILb3C&Gh zsjOV#d~aM8C8SvZhn==|46~0K_kid_XJ>MV$+(Jz`n}`5r%%s}Z-7$~ydjBpBdl~> zIZ1ygii97O8iO0YcdmSN8BchnY0uphXn?&D3NBaiA+WEh)Yf}X{`HPI?=FES5!!n# z6F@Pdn=$87x!cEdJ7sjaQP=aSE{Ft}si<*CM7RnKu!N zQ_sG9)u5!%8t}k{3JvOC_&xjJWLsmZ-}*nxiPQOxIf9VpuClneG{`C*9~#%Y6Odbf z)VjAfJQm{q`0);SBH>}Tvtxi|?a!ZSHH;{!EW#=@;nmBGxHP<__}YLGJp;X&sVToX z7sa`Ad=RC<^#wFSoY+TTFtd}T8W@aE=jsU=`Hs@D<1ia|Id&u_d{XZ$(apq_GtGY@kN}0eY_VA%6 zUyWod=UdK)JfojlDwb!9pJodotfGY5k-ZoMR*;RCgX_Doeo}h+08rD=c?Y)}pc^wY zZy_lJgfH{+Tm1Z;6QWXOxeL{Rl*;Ue-1yc|EQINZjCJVT1?-H1v0@b=|; z-V2n5lgjEB<0`2^@;BW2l;|pRLcfO|?=3qnD91)YmN~Q%kdYj#$bB4jE-y?WHIh1) zn}D&eFy+q0;h=u`@Ge&_mc2}4km7J^pFj~sjKgRaj60uNo-PGZDqHw2aB42F9QUz6 z4bqa(eK6Ywo%Ei5-8>p~X}W;GzJSER6%p#TYNoY)e;Br2^INu}S4XRQAGslTocmWP_@p8G z4t@T@g;yM3p)!%0&TJ0ktE2L5gTDq(0uS=M?Kym=!iBLji2eDdNXaCI;$ z(-Xa<5>zyPWDNgRy|?%3=0KUl4LuWe-$a}+!uu45m>0><52Aio%*BI1Sbd8uev*8! z(-RSu)cUB&n{N2qSkh3=vuB?$(HggmZ>TFFF&>^f@6&FaYF4otNnxGu1{TY=TD=%^ z2JZh3Q+j}$9AO6)wpLmilD`fBJ$3>s48ZY4Maz&QnBZIJv`C*~dO0TtI!oa|%+9v? zCoUtf9`T;EIr)7eQ=QkV2V4LO{440smzI(CgQa|QR3}}^p=zY8yqxOdMP+3b$o+V5 z5=i}~u&yerU}Q-TWQtX*V@lr6P-X-G57HWv6*M&$85|+>0)Bu%$89r`BPZM%J3--N z=JU3=q0S(4%zKKa!70FUwe=}CVe(}x=aXboYZ?S17;!^YNq>V-=wa97PU^9iVf=V+ zP-$)sWcU8KM`|ec{avRuIYRI@F|qu6kG3LTg<|qg4pCrhe5G9KWuYK(d5%f`6sK;? z61Hy7OA8-G9s#LC?ru+xX4^aci>(E_}@N5_+=$4w_oO^bKT!9+0gQloJ_+9l!D zs|8g`wzmcJznudFXH37JE;q|%=D3wi4=(DhK6;D8HJ+A|bWxRvjg=kVsnA=Ax3ieq zU7eg98hUXO4Hqc~WvMxV1Dj zZK6OU4zj9`m6ZTl!en|t-EC8v)J;M_q-3-iN@*2^CMwZ^#3~<9pTV&V7eS`);sV@$ zFnU6}PXOT9($bEQ0|9^*?CQwhpl%$m^GfZu*Q7HY-#t-rcG{`b@>w8|l8sA1APCyXV;H3j?*!P{0Pn{E)9Z=jCt9V|!ui^JF9yEqXSKv6 z{M>r+Vk#@^cS_14h!AROd@^|p=`SEZ6#cuuaOuUl5HE@bBRxI#t_>mxB6pou4{doA z5PizvsJg-3f=09vZ&FiJq&qK5$w&_GKN@+>s~e_1Km7ByrKN_ped9QNv?wK-usLi& zOSwd{5s1fia!{iM5)m64bJQNpXP|e3Y_Y{x zwBIXI<5&(80{)!!qVu|ctV)e$RYgBnqEf49C^u=teRX7Jp(CY;jq?st`f!_+nC;tE zUN+0^qlcl5u9bgCjn9tN`nbvbO=a%3D>a6(1<3|GC7$|Nc@3GD)9csc5+Ih-tp=Ky zAXgg%_+TNHAK#nEo0}c?nv{bC1CX1u9JR1CKQ~Z>4*;gGQ!!CdU=(VA!u<~oeh!5` zuv)+zuX!I}WY`H{I{`x|MB8kCkw(0*Gl_d!TU*-%?NFys%~A(BSfI=R(WX9xh0A6D z{>q94dF3ho6#;05weubsi-yXnJ~eY))`D4HeOM{pdF1 zfsfEU9(tAnERjX`e?h7o=+QA+Ir%#92dd5H(zLdQgz!#iT?CCEbU1}l$R|Mf7E(4h z;cz%0{nEBzJE}DTcQ0jd?B~(nme45Sl7FL>o=e5)G6C|FsV>a#3rozBEFn4oTp!Yb zN5+fC>jN~i1LA*u{hqf^-TWey%B!GP#xtEUI{uhp*=*VEoRKW2E8 zblLV5q$5Gb>5lWr$2+YAVdt7rwqz*lp~=G{4xY{P+U<(epMw#n_=;lG++TZ$2OD$t z^7AvZwbk?UW3c<+xikpz(x4`}5F6v&FagOU*xMQ!4uBy;{~0X$)=|VtWl2%EJUYpX zQ**S;bgwcpC}|lO092y8L`OwccOSfWsp3F?nqBzuCk~(IW?)o2{IC-II6MLhO~4XD zwH^y2`XaG4FthTRRXAMdteZ(uAgG}AA3v_-ijcOe+aU*|U4Au>T^J#Pge>eLIl-g5rDw1v?{Ji=l)3~M;<0o#9^W%O%3+NKO)+MXqefzp`1%OA8M$)- zc-h?d2gM&?+;I4T78G*YUf=`)Wcs+ zN=k~G`)i9*i%7Xhzg|BvRqT+6q#$sOcPxz{Dzu0|h0#SU_3oRT6CW{|hEZ(JZH zRYrDy`v%`NOlF`S?CB{^PQG8FlV~^4>o_ZQ`Pwfy!d!w5H}DCAYm+A+J85mr&$|Lg zfMrH}Sru3v#ko*9_oB!=j+{K{VO3c;mEmcVgZ3((b|9 zx3tPTb=Qh~SMOBv-c0n9XwTcl-ge#~ym7r%tD5Z`$CCuVeOH%XoKK08p5qS8}#1O*Tc)PN0L$nttd`us@ zndS!iMhj`_oslCgl&|tyYtCZoqA3Zh`umksAM>X0aAs}ldl_e8mLr1xECDgGa@d5| z9_=>_8}kP^%}9UgjuXF}v1^*9_ve?Zd>G6WKxsy!Bj zpSL|hrMT+zf44YCK&%*oM7&^qHhaPRrbAH=lEAQj!b)jJmYdk*nKkPZ#WTT*&4^%r zZS|+ZZ9S4a{ZH<@>2m?pK_3n3G3j*Yu0%dy#Lk^N2e|+!Ws}6^$9_OQfZP(i5zkN% z69C@kIVXRe{_Qe2fM8gF=g6tJ<}XoU)`a(Tba$%6bLFhMm0Blol!#X6fep?IrS_6` z51^|C3!TxW0+Y{W?|bJCTPOlQI1TZGk+WenU=@(}>ixf|Vp^jBgToC4uduW83Zxdn z;+6~la=bGO#BYA^lVD{uEH;OOAvrxAUP4H4=j{^iaLrURb85%zhx}3QzS=-9rWJ`Y z_ZO~R5%{&=ASg|OB2<2tBoPuki|)nNH8zqHqAq?uX*@}3!^^dE9^m7y4GfI(^YXQB z?Tt|}2jAx6ASb`DyR$1HRsl~|OjHa=FUvneu_EdbW9G|b6GL&ptPDyd?xig59zt|D z@CZP4DQ8(J1!hcGz2FFMf5D0cG!<&kA}ywN{snk}O~RtYW$-=zV|zGKRsrykb<(A4 z5R9gOd)fM#5{uWpty4%z;^OIDhwZ1N{wP9WiU^@^WH9cw$%g_bYR5WgH^63*CRaCR zUoJ$(-WAs`1?G&tpFhowjIMU+D%)ICzCMKpqcYaE=em|MQZQNnl>>?xg(iCj2NkSz zK;OJ}s;~Fg4h$r~!v8fFP?!XE6JIgw8Xn|PnuJJP^GHeJdEA|$l4#RIb`BRy?U2%yj z)=%G3#Qfmqx)2-XJi4-iU0RBRtY(AMf=o%!y)&8;6o*b9_UP zQfB+ujVC&N*K5We6);{x5TqnH2DiMpZv5W$WctE$`)>Y)#rf{8jECatM1*u-6yFdc zSgrC0Fv+GrZZKWy?Dsu)WzIcX2NiZ?tSW!2ZYEkCLVHF4*MYYb7+(k;1@hJ>LjyVT zpfWD*;_EKpJdbZ0KM=UgQN zzgI^`e@~*q>3%Q&>n_KNz{AnWnDB6)ll?gg7WuT4lmp;P>7&&X?S$G%LbD)t@N>i^ zNMJx9aPcDJPPWp5)9?J1TnXmlI4V|q0D!{{;?IAlBYwqHS3Gi+$ila1-q)qpL= z_hFOV<@)-C0`UVaoSQPhffP5L_?kiz3nB3%#(dl&p7jClptH;$|K@l4b*Q&#SLX3(zqx1*vm_Zw`;Q= zOF|PE$C|SEISqFN2WCGW#BaFbq^*-g< zToA77HST$}YaM>(SB-7{&=SyI=8gXQFWEJ~($H${`n&ZE{;y(Ui1N@|up?;JvpI4& zNzY(GB0ylb{1}b}9qtHHHm+;ejysvW1j%H?s$y7w3g(?l1c?D?O&#p)*s)|J0g}8K zvLl1Pel2=P+uGRt+qsuwnsxGZpo$mMv@>b^ z6LD>&W)1{wZ(+z)Y>q|gL5NIB3JKd?lgFk^myp5DoAg^%*n8X=Url~AS1a`3Jn8;nXc|)SoF5H zEEozxLR*@LhwqJk-HF+${PY$pHS5k($n#MAT04wp8?v#bf42{S?9>51>(8g~+k$Aj ziuRF*2m}Vh7P=TS^lRED9;non=SxZ{FG7h76Y5lA^-G ze>|j#o+yG$uwxobD_&c^0YDGs-Jdi51&N1FWNkn~a-)#d<14_;FK+ufGF_mcP+0#& z1@?RvF)aX112(>1-6oL(K82$I2M&TNj|GVASOgU&98~bQ0h$I8{T)_^lNhz^ z_oT8iqM}F76^{CrAG4c`Hxwi^I6cC2Jf&35Ieu!!)PnA8*uM(kU;eiNfGl2rC} z5=x;2hx6}&W68X50LP_iHu}|T!pFhiUzSfUG%j3-EaYYI+E4O8!G+>?=nu%SxNoIV zxchL`!J$NQaIfa{W6FLi& zFfCcuk)O*REc_o!=N(UV|Ns9()Je*z?8ppBG7BM2QOZcjNcN_TkQEsrq%I_6gp5Rr zB4kA*nJ1*Ol8_`J*}up6e1H9Q-L9^#aNh6N>p32e`=k5iKObY={7urzzqqbe0By0n zW!XUOV1{0eO-mdCbBz<3)wbUVA!eSH`1qLu4#c&8Zdw&Srq**gfeYc4eV#yldV0Ps;o|154qRIXFoQr4ir1EEE2w*Pk|giF zyb_b`p?!4o9Tg%kL5`h)aLOo_DdM>BryrEkIK|yhd``vM z^tDp4JsR0z09#;;GTg6a68$4#HmrNuBb7Jq+x-COvmVSYyzkkVXYOm{xdGYPI8Kgji`sTZkDcKL^+%*( zVDKY9)ph?jw&_)~x}{@!@mdlgci6(xfBot^;)>Kfd*d=p@BG$Zzke^5zMItWZ&i8x zqY*EA;Zd`zt47uJ7O6*E2TN=Z?ArZRJ{G2jT~x1)76o7n}A%eiN!&%94S02ZAYB#H^^q$7*y-s=%JRn>I^!y;ElM|rnl-e#)G57Ajy zuo3y5D7|Tw;n=*!9x(tpE`bo{DAX z7t}qg@~)+HcCzn?U_EFO02u1-5QKGg4i4_%6BQLL%uk|9&H`&&Z1{aD^?z)oW2dQm zEvXZ9k@DoFLv!IP+emg&ebQZZr92Z8?F1IFTZi`@f4`F;8J6@tx?T9rwZW+?B|5=P zzoCvlZ~zzPI~yr?u5s@wTVovQ9I!0U%T6J2|HS? z(ml}Oj~%t+y1Okl`f!Hl$q-(w=#w;a!H?OoeFqCOE3`(se?g&Xq*hWz!sh++prDG1 z@6Yu*uXk(V_C!R*Q{=nCSAdO$zMD71ve@Y_Q~jU81hT_qcTVm4LNTn|an~GvL7n?+6n19o!1g2?%PKym0EF_82O>7E;OQi%fGUlb#CP zA(3o8RH%CHpOg;8IZbc^$?K`%|R>5I!JNs1)x@Dp4{mj_+ zG-96)=RCjnT#kAbgpU5(cZRjtteSyu9>gPzOtaX{Dm{&DZO1S?TRbcB(D6s5*AxQ-L;m*< z35S=NO7HL2*hkzPO5zXeMAx>1OHM>%SJS`BsV7DGNdmuLJ$(D2fRV$Te;8P!FS2K7d^KGSG!r}cS^hJ8uhv@5n zz*(GhKdry>UQtt;=&|0*Fzyj zOu^yCkNlrkR(I4jvQ;}fD6#m!rq*B+Xud|xqD(V|{>*IJnu?%LZeHF~0ur7DdC!Wf-N*iTZ5E>0MME2s>~^Ma zXK}}cl-jphS5#;Ts_cmUy^<~;tTURQntX-MH=T_Ra%Q-pB@Q3fCWYahE^y+V;MubW zds42evADky@7@KIoA$ccJ3>e%cA>x|lBC%ymc+a!dVc0%M-P=E(zNsy0k zq-N#UsNAo{o4srs3t3sBcG14|tMsxF&Z+H^;g*=mU>=&%_)xIWF^ohf5M$4tCCzd7 zk$ghO*R+XldC~ZbO5f%sc*8_VTMuy;9mPI+a9|)K`MkresPX?)4TvG^G@8OI{BojU zq(oca^o=Yf^V~gb_R0eOnIW}8@hE#w#XSTq?y#&ZmN&R~bk85D8ZbkQ_ucpvvvA}fH}Et{|7NXPp7OVGv|B+pktjMwr!>m;S17(0r2SrU9ol9D3} zerekFa$J8`)-X!KxDi5n|DlyZGz~~t1{fD;HY!4>vlAaZiV-}J8<2Y{{dp_!|55iI zpD&ePywWBM54Sn=%oS6|1I|B*{=f1_E+dR&oG8G-Vw`x4rnV8ak+z;XJ^#1Gm90(4 zDsY^1bex>@3*{STg4`RpG*6p{)!Cu8Ngtj+r{Bif*eFLEe4RQ_?wm?yhkLt73Ee4r zW81MWU_(K3(C>1XtRlzt)pu~(n~8}D7#|Ru8Fw&3)4q$0Az2n7!*g2tZ5Ll}Ix470$1Xk6^fxq4jLp8|h}-#a zURvI8I#d3W^oS|=?n}cLNs_kL_C<+|q`!{2Ut5n5>Sh*KR?TiV_w@{LQ^yza@fqfI z5}I;c=a~1Ah`l}BuHYd}PO1@wg!A5~UHA{oowT$|hzOTSzp<;A|x7A4t5{#a+eYnYb{}1;jp!>c5#>NyFWB@gq=~Q*y?|wX#vVZ zDD+`k+qHYQaU%BO(8S|ua5)SOLRP`-& z0q$2>A|+zdjG?5gS}8^W;%hwAFj-YP^%ST@vtprJWLCY5+WFPm-_O)guRni02r4kl zp=?5;`RD0hKYv2Q+=?}!x&QPF*g$HBbchg1Zvb4vbo#iatdV%b+&{*V< z`0{bWyP}(Nj_si7og|WL=Tr!C+d5|!p(Mc@530hgTk@#Y62&hn1O4#m<^^J!qvo2+0qDE7O;Vd=UbFbMIf) z*3d{7Wz;!-I=A?=GVXl z^C7pag|NOUVY&%Vzv{Y^hac{IV3}nZv$c`#tS&v7Nba=h?9ZOg$*~<9E0x`T8-eTd z^T>__{pf7*5kxHRnOEQxPPDyYo7~%57PNk46l)1HvtNca9#z_TLuJ!Nung%Y;N7hG zbN46&8Bk$LG#M5e>*?z&R_Bbr__~=y3UCTw!jB1ao zY&^JXmG!?+<)NLsv)@TYvuAYlQF;vge;Q%a-#2Az3?51O%a=aAg~oVg&%8;kP!Dx` zRpU69crdDf9QLH|aRCFt z!Z^_~nPdOmZPC#cgCOAjWF|wl{kcc(k=;gBf>3`sITtcw2VuT8HKrT|G z+w7KU5^ZsnQOyt-3Pw|P;xJfn?`TH3H}i5dmWmvwN{1nWn@H@ieH8TFWcpCg*}Nig ztIcfObULg7lpRG(%ru%SUp6NhLC#3O^@=l%fZ3b`6zO& zTipwC4cgK#>g%7_IGsV`+*^pmt&bJXhStX$nr>~x+`Red$rIH17=dUa2&!2{Hz8k4 zAZ@#XlKzQp5ZZN4-AGOSf&oF*lP8x)n}PtKL$+iI{1I{OMO_Onf%& zsZp@>;q*t26N&uO`!j$JDCuy6`lcgNRa{)Wi9OU&Hs(QrWrn|%&G{B9tuYQaJ90yo zLsW?hk>k|R!((}mOEAS+x5_#{Ny69V<+T7rtWLTMl59MxE~;lU?}#|5eiFMV z)xbbo0~W#qERJexhp=)B%>=d=wLkVOfLA|%z{^7*@TLBy!(j^3tbu`XnM(yP5u0Dt zAQcid_9H8cKa4$QEA6=8uyp7)!fSPoT~^lJx4nK!JvEtmyPsplxa~DN!FQ_prZF8EDZT7v!Ohz!_S)c?xq{TvGU#5 zERS;W2cJ0s35)1te>Ls7%ysqFmQdA*hMluEb`n}iE%~NarXRZxsTqjSZ)q}$dwAO} zHR;!EfRF@tv;X;>2H)-uItpaAaly&%Ab-e9#PRzQc}{F%JM}Xk0oBX+}764=@m;Qk|%6m z%H87%kKzWpU~;A#2xBO_Azyr4Q~qdn{#lT|Iz3rNPVV`_ts@XsYYFW>F@bLYdb>0~ zH`?Ym&2PrX#yg5q9~byQKiZj@Le*TiKXH?@xmhx~;Ai4VM>sL@Jdv3ET{i zQWH<#=kD$hl1bIyD1fte(dNo$@A`Yg|1Zy%i)LkF%0hdbxab&#lAeA|PdVtmb+*4{ zOo3T#@^L=rTF$%tfX2T&tw3W(r3-P*E%oeNOB|eVV1@!L>gM*excCs)4nu?UI1%zw zq9k|FAw>&QWT-ZG3!9+TpffuuvEF=a1D!wg>xqZV4SX}rFh0b}op`<3$rFKAXm!~Y)-lgm4;(}_ld^K1_gbRsFt3Uc`><<5ENeFF7a$vv0wTEpq z$>YN`rKH}J#qbMVwnX~=S z$&>Wfw=9jc^tRSRkqDX7^H8-s=}vwINv`j{fi^xbpVTOG?O?mdeDdUHa47d*EgPEQ z;o)(;q}28CgJ*tj?&GQpx(R5*QS$#9tai3>NFpiLRxU>!c=DiZ8(~-}9krxXQ7Dbd zf3j&bUQBzLT6ZH?&(}Fimd%1j!anzMWpj*VZpc;X*mv~JIVP@Qq(aMZONVkx%qpAy z{a-j1Y<0o|yR(Nfjrj1{&`e&!b@gn4*mp~@;~qzAGcuQsNq^Tk%0(vR zt-FpBA8ZO8qr|Llof>f)f7FhNU2|Idm(iSrJ~Qka z98B2#pg^=kN}qEV|A`CCMs)ItBR;{(-Nb7mR*DFPS)M8`FDrvn3=cMJ%{?Eus8suX z8YTUO8$(FV!`9x*)05fy6{>Db?4F5*N(SrE(;fOb8BUP48h4}AeRR;he7BA3jDDlk zseMt0WX^wnbwpY~`e0dP&pfe&pl5Y}kk|h_JmK)truAT^)suwji+qfsGPj4{JYQe z+O4{|s;ko$ymcvm5vU`(Uou^EnwL1#AhlRCM@yq{JaXT+Dew7D!G@SA*7`hY?oWM=*&#NI@Ix)8Gz8w1 zpT8bdpkm-6Q1y$rG*DgX>4De+CINOC=wjyQvB%qZ+V6tt5Bn%K$x=Us<-)GKGE;U7 z9QIF3+;()F$R}#KVClbV)9g_*H|J~Os#mYLJdsVQ422)a{P`@JC*`-g$-QZT;orx* z34wKo=2K2O{K?=YVp3LW*Y@uHdjOnws=2#(%8-bROxtCT$Re==V@I!rRRBT1CD;%m zMQ#Sx4go$REBm6X3@K1PYxV8F1`l31@;Hif)D5W5Au2TqS5NB|Q?%?QIcKK5kA~f` zO}?%7Ag_2QpHYvDfdM!TgMQBUDU&wrHWKWCU@Rxy+Po||{<3u}9aMNCEmgut63wSI z3bV?ZHcn1HbMIir(QAoR*JDoy3wt44%W-gi5QVN=KTs!JxA>uy zN_t=Ve(>thz%y%5lwfl+yy$fod((v;t%e2x45^lN;()0g))+eXERd2NTQK`F^dNCqfy?oVxqkt~_+V>f~hNtoU%)lN?u>y7Bn=_U#c3&&$mc8Z9K` zl$C#ZROIg&^;9qqT(VUqfiohXXl?X8p5P4!pX)B3`)dK z7U`T`o2O|=_1L=;m(ucY_Mv)N}Qt4@z4Sr3G%#i)Oi_{)7MKv-7x;K3(JZ#H{=(xv&WG@6_^ z_IE$m-A7aLmPVrSyI5?~WS!4r=5J+1mSCs7o)0m+tt!Ox^A;2Pm|>O*<& zD~g-i4gKcht)0I4V6wc*ckV;;GrWhe@52s>?#Z~s28KynAWD$etfhsC^(E95Fqp)s zkA8VqL3$lMnRC4(x9E>C*PJI+*^(N=<7{+ZNKKzWpxdpxODB4{?R`g1gFGgXgfO^+ zUynTP-(`C%3Y5_{*r_#7NwDA2_ZH=1U0GSFs#JKEJ9%pv-8$C8;g&Mnbu?e9>0)EN ziEv*t_t4P#7-T+u=KdEUetY%mK(Xmmq&IKa;N!GEHI~|5`t0ErfNKeATqv*U>gg%! zo5C3B3G}0>g_?xxg-&9vz(GUT=N%MOJ8&71AvratzmX(q>d&4!Bi@}p(`V|g2KPX-)bs4+_sWo& z=!DHF)kX%Bf}gT`_=Q)Ui@wqFH2puCl^_=ybbYa%=+npm-8AE@1Ur-HHMG35~9^NoQ@~I#5 zz8%Z>BHab@W)4oy-l^@7sEQppfM>@+w^>=yd!fN0#vcs=0QB^bRM>GjS1af2PDOU*$zo_$981c-C zPd1$BoMch!kzpacb&rW17gW1zdt=-vanwtJkB^TkXap=CmcTfplpY~er%Q<5xW?b?-d-Maeg*ymR_Yd}6L`?0)wak>z z=OMq)8IJkx;IoU9<+_6$@x8GXv($UfqbJO(coVi*hh{;+_VU3M{{>a0JFgxq6pgJ?RWTQqy z2&2tJnC9BbB19M7o~eI)lxRCHR$Lx{CD13mp+G%SeNQZjhV=E3Gt;%m*msPBx97~7 z?nJdqND;la^XwjbFpvLTdyZ`KtIsE#xX0dAeXr9KLB)^+x|7meDrA$zR zv0;8;p~6(0m}yp=_nd}OJ=98Lzi{xW*csteTjSD}S8iReRd}Dmkcao@XR3=yWUIR= z+D;7NE?oGGb`&YVjvD)?RSlVM*!Dk4E}u+l>iGI-Ni5b^K?2{ge7%0+@?vdL{TP!; z&21suBfOY2G})D6>{bhcQ*A8_SGhHrmdI=#^R5v7a;+NhrHJvi`+L}UHo!GSx-sbi z_8Kc|>xti4W5-8Fzp@@QeJCn=bvaq9y+|U>B1@HvA!5({loOIJhwsC$TIo4y%mW}? z#c1qNHCaJ5`JFrLxASEhGP=iQdBs5(oc|bRYeF3(HY-~r$v&ecqFdNdZ1}bch@A~-h z16VvpiI4v$ULX({L-QL)=DnZ9OWzEB`-M#0NfW7ABVQeDml8Ww{g6|SCsC|fpz+)P z*e)z=_fT7<%k`s|{;{x}xfuZjNKwxXw%|=lARP%>4{)m9si;=I>FXRvE2FY3(dv%! zr$mdx{ULEjNL=(Hbaij!wk2$~{&6(PVza3QTZs3MIYk||gAEchdC_S*o-(c=vxTZ= zDJXyf<{Hv1M8w5wh%42pnFXTF?n3dC*kS20kG)|tUxfqAyxI(?uI3W4rk4*Nep_Fe z*??RI7~utTzqIw<&zI>5oq=~FAADo9b7Q5R61CyOLnzEIQ|i@11vM0#w<>6p zveF;^KR@^n94zz3`chD^(UElA-*+RDR8|xt-g8u%Ch{ zc=xW{yAfzFVGN;Ocw{o*^YMZ=#)TWo6C{kz%c>58m9s7S&++;PB2h`Csy1z6qOF>1 zCj!(r#|iF_^7!`~0wdP+aV!%`=Kd^4Y`(qs@f*^_LimknHXg+Q9|s$D>(rzqtg#=V zh&BE$_xGIq)}4IcT6_dT*xm@eLM&Kh)?0~4TX|IyI(ZKdk3?~EnB~pP&9Qmv>F?jK z5m(@(pj3-x^TUU9F_KN6R@jF0nf@rLHgr6+zseEH8$x4s_PJ88T{QQ7b~nGxSZV4X zYHH6r8>ls8mM;Y3KjPZ-InpdSkj>-m7cY7;cJC5gw0nqjK>61OZD72(4xOHVOWZzl zo26r)y!oT7V5VNI-MB`mqcYX!GS*hZ_qe|8=rA<6E2IPCL znS|5U1IVRqe{eDx?O~sw=)b7S%R{d4ix&%k3gLx*o@dbK4REuX#5s zawpAN9TR?#WA3)>`O;qEl21q5gi`S%-z_v?BG3{h1e6a63n^_HjF(@JI{*5v*!wwvA&AwD!iRtUG7M;g~SK5?HA?c zKJKgsakSF96yn($w+{X{FcA5luKD@bNcvJUQU9{!3N1rhYvGDwH;&b z2RJoOYuvwJprouE__cAXDv4Mh&34eeLJ=f9gd7Ox@gFU(W45Is5Huyy=i72u{3Mb! zM|W$V^1mu!$^P)`BJ-Gep_->QLgg8=MHp=7&-E;Z^>pmpVymd7t%XNIBKth5BbN^Oo?LigJWD3da3v=$yceAl6_i<0QPr9x=WI<+2Sqfa6i6zn2v3dIX* z0Nt3PPOSJsw|#3Pz6K(lseAjE>Pc0*g~b1~P)3P8lb$jm-gb)f^0H%eRT6nw=Wh8#+kdDrozNbxd z*hi=G5~}3z1^#yjlYwF1>(Dc%5k#ObzrZcapY-nX= zvRvO_pxkxn*Fe=i4Iw&HY;mJm4_X<8vj$2tW_`LgzlTUeBs5Fe#dkTCj*Z!X;`Kho zpvw6N-zQ&w>=*?rTxm_!ERNuf>xW`5LN7c+w{T7`!lgUD1 zCyd3wQXHN5w32W7ZbhdyXS=G<1Z5I|e(r2lMhX z`*W^jhuMU)cU;SpQBjtWfdhi+(#NNnNGO+-{BQ4zf&$ol=fRZ5*^nq^d=AVa*yC|T zR1L(E7;cdD(vF;FWRYhepSpH@Edy&3m`_E{FfZI_U-a3p^egjt1YA7q;T(JS?&abt zeDuh7_6=JQo)(P%+_>yzIJMN{vqV2wFMXSnr>Dvf(MwVjlCZ=O+P4qW8WwOb&_h&t zPefTVtM4m-usvwUAgE7=SdTpsc)?jptr!YM}^2lN=a*NYcW%LPu#q z|9`Ms@rAIwn`Qq>9Ucd;Qjf5f`~YpY^BrFql~2lDv(vfCci@MOvs9~86sl4n9QT2_ z(Q&mM*i|S-59}AmUp(q-X2250?w0N>AQpwbia>~Oad@F_c)p#UV8LkQX{Ykvox}pC zl<=J)M9Js7g+$-ENF{eudb+#ga4w+^twG)bye3nVlYme5q{uq6Z&k&-FKF!r<1jYN0Z$rvdH z&V2mmma?#_t+YKDab;8>=hVXAg}<0zbSdamn*kEOAKMb4-5qmK2`&2h`!jLLQ3)WR z*s7A0m;0fNng3bZ;gs_CWA^PXW2K?v%xaQCLf`F=t@J~yeN0&yCTj?QvE_%IO;*bv z)8aum5unHb3E|q~^F@`{8%==-c<#=HKH_=5%f`&c+A%fV!BqO!n zeg}pv<8;z3ogMPp{`={6lY=05I4he)nVk0v)y_k2ao^zZnM7OU*S&h+$WEx6d%yG~ z1_niV6}DR&Pyk)9YZKiTkux}$9vn<_zb!o}2{8-&Ja+8i_8g4Ya}nw!L&$IDC#Lec zjg`K`whha&LCh*HeZFuA`$kMF`iiRf1*D~=1>)6+L2ltRvs^s<^#T!0{)&6!TY!s* zPfNRo9u^Lc{3xdv#sIN1aVs!0gv%7>eKMWQ3(Cz5*Y-08+fB*%$w{PE&CIdII>c@wk>BT-;j|ZTH*D5b8$U6o1!k zs<-%!N6()9JC+q#zVFx^T2O#9X8@KPq^kTwfxVK3Kr2k4x+in1YbR*6-{kCl%|`pb zha`-I5Fsy(eiKb(y`M9TkD;RgDZ%E;SU^~B|#fCc^SS!;)lKKL(r=niYiHI+q`Xa5 zcUCNX*%|dNTWuzPa?rj=tEo?x(Uc(wQ#Ny+@0w)X^kPuo;970D1oz%Eg4+8vcD*>X zDY%YahRAQfW0OSx?i(%r`kQ|rul3^K_b_Y!!$lxpxk}pR zsd4jBcFpd{qpnj?W!XT^#<)wc$B1}3rU+^5^6H3-i^qHtS%quxTHw^|{2`YhF`vX3CD1noO@P94!t!_T4WH^1Z@QW zdLka*Z+iBu1S~mcWE&8MM$wlP=c3|M1 zvo8+|wM>e9_RsR}ng0Y?2*doRrUFkx4{YJ3AT1PCm|7mopj2rgk?)uS5ZnE>`8pQ0 zt(!x$M7qC3_COt21T~{}9!z<<+ma4{rF{1fw`8x2%z}q}qLEBAFxZaO@L8gDAvT|L zjIF6hDuoxFUSq$()_2q(14Ki0Z?SeDN7l6)X`$DLM73w0Z^Y)y+arFCex6K zUVKEp4|R{LjDMK_?)G173BdcOdv?qtI)@ZaVzE%*NlNTK+>&zH;ZVhZ-=%_U?vdD@&XxQleL zMhR~2=_pY(<2LBX*p})Kvmk6D?xBSUkvr0YK-^4tz+}wGNPa#OVpjBD{)qmI-pL00 zzDqCPKKRfebLKiJDm8a;$XaLopu6C?K;OaLBpP=92a>Fz#(J_E3>mJCDe5)CLP9cH zxCWtL{c49U7KlYiSyopAv?#n`3TY5H;iH(^uJVUSyyPRyF^WC7?V2Sy#mP98aML^Da8o;y!b@ zh+(A=f-m9|%)8VE4%4u3OCqAH_6IFfGJ2m}eEf*>#BT_>YS2eQgv#Qn0Pq0xufU)l zgVeOYUn71q0(WW2fU7h?T-MkKt2}OD;Qk5JP*zkmR38xtkfD_vSUy)y=H7qu@@v%U ztAGNJe35rN>%SwYF$NFQH{I$#w)o`=MB_tv;xFL!=LiWqt0#NAZ#Uke+kKOsW##2; zEG))Fy{F9fF*DrE`iH%O=hdsNB`z;aAWdf#Ga~Kp&DrNvA_kc$By3;5TBT_(q7ps! z_oFq64qWPx9!xpxP}O_Ul-op2v&P$+E8OLBcw>0Wv6qM_LTJ0-!BEM;vTL8IHrJFC zPK`Jc8BvWv>vI`~MJ8T}d4f^=4H6Ip*CH2?`foR;%s09xJ+FRoKZ&c((6A4XVMhse z#J#<@n=^jzLJ9QZeJJt{F`Kt}j_utMaZ#L&z>9`T=zqU(zT%m}uRkLY$bYq_Zu5qv z9v)x&8Nzwso*y$C)BJb%hCY4yA_$+6oV0XlRu;L7xSjR>t6{Zipd*N~Xh$C0@z&J{pYj16&h-s2?Mqvzd+o0auwvC~@=2_2!K44EviFH&f z*W40sm{5F0K>=ud>+6BVea_{d!S;qA1=%;@meYSH-n~1W-4&MSlwq#O8#>7UNKWtu zuOPwV)@cs1W~BG>$H{#+Y3fK)MC~R4$BQ*BHYJ~@pYvSZ%tkZragp-lo(iwZ^v|~| zFmBCk{I)%#rnGRKK)B9JxMSu@LwNhg*IdkV&_Nwce)#kiq}40;9p69j@t6v$7jCv=%jwyWK*aO{SK!)a}K0Z$rJ(QzaW|DymbLp1@pxM&vlT@Yi< zqsT+VGLd~7;q%Z2MQ2g1kUmsD;LCueKbYdd)Q)w4?2#kUSwta@$}W%^HV{e%#fh+> zAjDwz+sXp^|KzkL0=te(UqhG%G3Q~XQ)O05r{Xuh^!EP6kOc$0V^vuA@`a(GMf#MF z2s64|2>Ns}zI|rh4AUaj9f{;VJY!#+UVxQ{eaLu5&^jp75IXd<9T@`l=Ac~_&U~JL z->L{xkfbBv2lfNCP7vwx(#W?@Qb7=#{`;bCvr0NF#p( zzN}d>J{ick_TG+YxAgWttRy?%JuGk5qi^E>7q7@3!c#|zPz<##WQkIBno=dKX_HqCVa-7thv+nF-7Je29nR8d!n7E>WF%J{xki;C@$1~ zp>`?2wV5(~Eq%sEe6+bme#G>9{9b|T$)MhjnWe9rnsqWto`3xQlm8ZzXksYIlEB;0 zxk+FDhP8aS?Fn5OqQdgGZ4nI#`#09+eg{5TT}Kqb!#c;p2!s$`ER7(LEHrxN zv9W%Q+UQSBAvan_PjB>sYjuT1{p+YGc_0-PJ7(=!#C4W57M&!M*$x_=J!e8bHaxph zWOwG&F0YWw0D71GooEb;7+->4AD~@iG|O&b+RD}89sn-}4pTgl2U*Yf`I(re9;euB ze!bs6$>STPLUC)jdZ^YRBXTA|wIElLphAu`b0rY|1?(kAkO|h?X#g&{USRch_Eppi zFo>U|?~Q(=D5prr2kLj`uqHfS#o7T88U@gPa&p zQ*K-cWMtj>MBe$hm66Q-HpPwuQCa!tzFm`3D<4<-X-4s_$^EWAQT^Zf+EZ~Sh+Q(b z&iUX1cSupsew$Uu$+&DGfjoAC759!UgzJ_h6~@gf0%rwCn}{^8369^CSbHmM40D$P z_gLOiCm^~pxKE%lD1=_^W)U749F?eiF+IYxr=Z{md}|OrV_sl~)Xum#vKJFh_8I=L zbreD%e39Q`+D#?BLD})C&11MO7<%9%gFX#Y3*DRt<6Cgx!A1gr85IZrK~Kk5aWA3n zkzSse?-(_q?e^Y1)6$MkPFZ92$WfAef(<$*g>mD!68;FpkQqDhRyHgQgeC$_^VRQjO$zGgH4GKx@76JL1^Sq1t2E zRs+T|=CP9ucp>b&xSl208`)ifF2PhA5upH#GBdbym173^?!Z&PO{q9X43a8P7FL_h z7{VX{=`_w7sLQG2jj7(EY|jp$K~|~xDX?kIU_cv{r8xIq8pkK~)`|!Ze?9UWA-h!H z)*9xn5T{VBeNgoq7Mrv^eLp+ej0R~C7zTVlkPKi}VIlEv6$c$aL@Bvzp9eOYo-$r? z&J16vtE{aBgk~%xb%MbfP)fpD5}#8g_c+=7B7i4>-f6>l=1ua_CM_G z@>4{KuS+|WR}gFpe5L;R^Xtt)t0!T-g%IO6SbkYpAwb+?FE9E$_4aZ(Zeb>+c^rrz zaieQZ{;9#lu-N3q59^Kv;g?QN!zJtVD1Cvo8Ftz*sVz=_zBP^+pW|PrpMP$a9BG-7&agv`FSZei1^kQmO_*R9+%1Hmj&GUvj3+I$wz^$8^ z+5Ym&>5JPpk2Z!?6-__bt_&N%CJNX>U6rCB}!A$;_+AAi>Xh2rc@E7k>J4{qCHi+eZ3{ek% z*aBIAPn|SJ4C`kwre%l(TFfUGRgxFd`~m-CPDBjdP5;=gTA zZr+x``^C+Dc-9B;L2|J`A%HBt>6ieKcDl-zeY0xhp~>LRB&B^fv)rwU`;JOV?qs-u zvPG?He6A}_P-5Q~-qw;V5ri@o_u(9~)MO9udET<1{?;b@&6v;3J|cm9NbZCHH-WEF zQ(dWjOF;o_7tkS}LEMQo)rp>*d!IjgH<_JU15%hLT6=Ffo%d z$#{dT*fHcu(V}v9vCXx!6IuunN-}PIdj^gBWa&_g|yO&i> z5(r}p3zaxQLf!TwUw-4dR|MdJYsVp@r^DD};UH_Ka`m?FIV=W;HU1|dytqG+tYn^B zb(VPya&40;?;_CnA$~yV=MKFBbqI#8J%kqmZ|Z?W&Ea%-+=8%gyxVxm2R?tsi5Xaq zohp|7)WR4qzpXKF6RmjpFxyY}{ho{#3nGl&WKt;9Ut+BZ%tn;wNOGmRrtv2(LJER4 zifhGKA}CCdXnAlg;zLlzbZv8}C6*r;!K1v^K6Q#fc*+Qq$?~y{xns!K7<~!lD0&6m zp`UB^jLDM<8p77sYN2&YCi|1V)r<~3} zT(V^0p!5hr<*0dLf}z^>rTwU+J4F8k)+S4}FwJqubH@pS16qbSYO*DH7{xcjNiOaV zOt&XlISePO4{;C)bk}Y9=rp#@I`^3w>q=e)rV@(;q>y1*hAn+D>;M=?2lSWNZrwV| zwTZ@}Jts=rKsLNyIivX5qn2rJ)kwh5-nEX)$Q~|#{+u`LtCjTu(eH>^6l90`2uB-= zbUM|u`eWxfZjvOBb-l;U`hiGF6L|-%$nWLILs7EZHCwJfV~mz%+!<4%4hTNJ+uK!5 znFT*Q{WRimn}$b(qyHgWSp`rxaz+gwMwaHc-AoKgT@YGk{{CWUrfloVN{25^MBby z{WFWy{!=k<6}ENi^?}bO5PJbA)O&k+e5(g>Yu=u?aGivdTKMQyNj391hYnt>Oz=+9 ztX8AgIV_tbj^WJ%h)=h&>=?GHsPJUlu#pU(R! zP0v0McxF;WLWae^1HGof>>!F46hvipW`e76L`LSs$&(uhhlT&(H9Msy7RM1)1F^C= zgMdZ|U)Vr;83T{9OI)?kveG>r0;o~Ph{cY;zol3w=~60i6{0X?X23qVsj(54G!}0J z0^%ckpdACmUwM9wN`@wpvqXCP`i`b)Yf-@e{sThbwb3b1uO-Ar=aw8OJDNV)m99}U zSY-Q5DSds^AKv$GE%8|*NTXpQzl3$rHDGj4g0x9-U(m+N&_?&YniHRgI1+s^b;28s z>1LJ)b#b==wBe(IPXhzDB-6o{{xjB^XXg1^rjGghGt{xd=|1E*!P5t?jr1vz!|o8T z_M>X(K<|uKvb?J5e(s4Ul|Kp=rutyy+e{k_nIdQhpN^=Vk@&wh8Dpred@$#Nb4+y` zQhA{GUuJ>hHtih*C3?lMb$b04Dq>-E*n@HM;$dP$gUpv4k$3tB+mku&D}MN}I2deCpWg9i-9>iT?ui_(?MF<_jpfT9KSl}z zHy)0LhT{NgbwK}tDa#9tP4NO)DzCp3dFFA!e`WpJ78uG9?}YQ#z~GY8^DcNd zVdn|DI-L)G7!TsegkA_MKI8;Yt5&d_?%lg5%{T|e-z1{4(Pn-~g+c+Tx1+Nk_CCIN zyHkL4wiBduDJAr2vv_ss{FAEVsA-w@HbAua*}&DgcxC!vYX1e^-zi*1q?) zJ#@F?;+LmcoP*Zxn**!>nPiGL^d2Q*K+o-n*w1J5?G>fB1ap$@&Pf(@|7quCw=)*G zzh-AcfUuCzxgF;3SRaT`dF0h*uE_K2n9CA5*KtCZh5Kp#vqms!mOxr*of>iCPw5x-YuyRFPn&f&xu07~Biag}~Q6-o-P}_~?U9^1j;DQ!j zC>K{!;wGp#|DNCvLJyB?7%od+--;}e%jKV^y>G|FoanxtQUF_?Ru6ljZ|W zW}6_odc56KmBQ5-*qN2xhC$DHi75syXYIsWX@lHr25$(dPxhj#C7jTGBJX*EBGC*+ zv<9&D0s_|fllqOkM~+;5P}_HOjGbzlVf_LZ!wex{MB; z779L*fW6cln1RdVa=$1E+s?7sc3-SNK?>8by}Lidw>a~9?yK8`x|DE^&0XjBK`F4c zfmf@6y%po|y|(eT$x-t8DTS2D_jaY)dFY)-+}NF8mX_*~ux=F$A3=XmRI897y;I;X#llVJd zltOdRAn~6~8ZYs~3vY%0N77k9MY(rfd->T@r$b^mpd{@Gh5Y=~Wo!d7ks%XYb$6h{y#b7P!BFlYkX`VHp4j zGzdt->iNU}m>m$=cL9j|Gl-K;znpp|GVO$Cd7K7|Euj32OiU;Vt4Rf%tHu?;8ZpE) zpf!7LCx)G3NYpc5;P&1T9Hbwz!!{e%i^pJ40h6cJ-+73vr_JVk`646^({k`xN zBzKZ__6XUj7tG=jr3{{ZoDo`2o;1THb1Ta`Zc%ij-_puTYk3UHwqiQv7!=|D24X}2 zV-utn;S61x%MJ@H(8#c1eYnUpK)=Uo>+sA37aPnE=tc0g|Al->VLMc1zz7BY1W2!y zynAP3Z$AxsBe{kI2y@;nw{O4peE4U=BNLEEaC(9B5UizuzPAj??on_rH8jE_9k~^R z!62D83glO;?S2IgivgGm5z}dh$R~bS{lUg&8QK~9y1R337-UJmb0t`f zroL}Uki{@aL~i)@=Yc1j|KJ?Co)@WD57qpjM}~WRasqllMoJ0_-&D^4STa4(#5OxT+$NxAZ(lm&aL=>_Zi;=DO`!r{v#wp_!2Z=x7D>q&8$Lnqc7ga`0x2eHY#jNd0YRux7Aq5zHz#)V~A=z8lok98C30y1F`8G@{?GXrEV@<3OuhdOCKq`e5=c^IvvJR20AZ z4XCESUq++GCs+^_&z1lhCBfy(0~@&GIxsOh0&VGH<7FfE0*CP)M)dRleZT`!eFWG( zf8jF?;tY~SLI||gi<1792AwQ22M#0HtrT23f6DJ255=tXXOxVr!x*mn@Szj*j8$Sn!r(h1b>21CySlniMT#BG zVR;W9O5s#DB$kcD%XM}y{n2F>_W={GFD$^oosF$|zIN}_3G|ZX0m$3yQD_UpV@mzk zpTk3##K3Lb+4*%OLdJ7=b7NCp^D&t98By=b%FfSVI|-uT5tbU^>+1_N3lf*b8tf=< zzgYFGrkOR{`!77p;1=fl`IWtWR1ZDqd3@<=iY&L!?y=nS{?UfKB=?}Gy17|B$x?Sj zG%1l)xqO8BH9;VdA`=Tl*2I#ub#8*xvvUYb)u47$!x6P+7WJDj*xVE|wy3VLFMw0w)Lum1V-GRyv{rGEa%e(a~lF=+_*Y4`59`U(B`?oujIWlaNo#6S@nurY5!yU(i~NT_;l_7!CwSHr4g zvJ~VYy^_R5u(ps8EJC*-kcC*xcaKg%vi-HOL4e55FX%6R%f+!L?seFMY2*7gjlpDo zn)UYwrtOlGXqmp(r7e$oS0f_Hn75PmA#~3h?wj{ris#@BK+|_)SW6;e&Hae@@bPg; za6IR}s?1QrLsj^h^C&gkV)T-ygoQ-We@=wepW51X!{JB1BkS9vRul@;B(}_e<9Ad) z8Hlo@u}bOwbhU1Z@@goAQuxpdqI#M-Ku)7rK`YHrDGY(b-HirZ=y9w^XTC9=EXrg- zB=scqpFJ{R{Gb_g_ZLa;%JMO>3JZfw7A$KDcTb{V(gr?m zqY)xp9I+dO()czhUAg&WeNNjP(<9l05di@KaL64kdE^3-Ozwew@L@AC!3IsGjY~Ps zsNA^9q8qLvI8wa0`veqY?4u5A_XCMOm)(#XA|0@7#UgN=mwUiw0>E6rk3=tGBgh?J zJB9|ug9gniu~&pQh;eWU7O8xG#R$E;2aqS4%1_U+6?%H;YhhuLUsS|!&-Z`ZF7QM2 z^!9?!oq42(M;YRZz!I#ReCCQXxI*sqI?#s-2I)Jqol+EOrVudrkI}0!1Sp zKvIKxD|m)(c{Up*Sptp$u&Ebd?WUopHv;Y=EE!-QDz7aKuQt}r0NqHjH8n!`Lo6{8 z%*w;qmj#-iXF-LB<;2d;=ZkaTcrN!QYj;yzc?t)v@y9Oe;G!5D90Y`VB`9XFpEw1n z>=~rJY;8UMPFjQSrG1_qrV~!i2EfV!4Fg^-W7w1cXuIs=-?XqUHcOtu+L4!yH)O+v zqj(%lEtaktXlw7orTq#vnVDYO9x$-kTD;+@fT0Oi!OJr1q3~UMq~T_+JpxAPyc<_Z zZaKy6;0*n&8_{mtd%mn3<6j z746!Wuq+?nJb-r#$GWXmyg|{3Y3cd_fk7;wIVho0R#)@w1MAIhy?|S1*A4XGK4!1? z^zq}zItjX(lt?C~lvQ_tJk8ERfLcelpw@yS2=Rr{kn9Fgh^yy~fqnSGuSbR#ENty; z;ol4z#3eK&@-8mABAW3a=naN5jyQTIkbcoYN)1BtN2)*Nu-b(N}aF z1V`-EzXykhhmpAxKYpmJOWmt1cODZE?;D6j9_c?JeP@g5UB6H2YfUXmYGvKkd1aBb# z9{vOmwl2o(!Du{%Le7(}D(>Cc5 zdwm118npkN_ip|!)BhNNjZq=e^3|F)FeR`hA|@izPkQZY@~VZOtO_nkT8DZjxQao9 zjm3F2u^tv4{tsEqJ}t3-zwpxGvTHh0pglX8>;u__~Uk;Aew00obx3+&2r zzCF|pbO314%E@6;j#GrXg^muea-B0={AR!&nV_Ep^g5P%zF#Egv21^vYg%4jCvf+L zKNQXX32q9Gjazr14v?Ssb@7SkzxhxYl_39dhF!@Bh#(Lh^`JVKn3#Y&F4ndajZJkX zCH<+X$^XK7oykC#&qQ7swrFq=hA2jEm4rkm3&FZ;159QN;En^c1C}DJt4sO9L`v@i zjQ@yTZ9k?*1&T z@?Dqr8I4sv!CylU-qA_u@BbMYsGs2h90?m>KCG&Nv&Q?lKKtD}G?UVe>aKTkIX~g@ zzw^u!XlL;e%)ACwz$i6R_(~Vr&g4{DYDeNy_)3bkCw>XvA$e|RXMOz!EiEwt9-J)z z!`SLfhnG#z<#KTd+tl+XI!mzoD4#ogeR_KO8MeMKsywUJzs_qEmzK5%K71AD6w3*Q zv$gx9y8Gyr#dNwtHy?2E6h7bn+Ve+qM^d2)4i*!`aN8{;(O_M>lnJPy%gZ)Z* zBtGZeJ~CBipNaTQ)3!yi$J%&IU<63i)zv<#3JMfy(?Wik_1>_AZp>`Cd4&XSG$0hDd3{ZZp0t-rmh_$2?oVc4CZ9W6k7jOceCRz{ z^XE|6LUtWd9Bxxx|AR>duB{x17{F1uCr(a?Pc zAMj%JW2{ev_FZ_grM6)45d$d;?9^ci&D|mn-+Fp*!K445v9U2MZU7mh6TBeJUSmBV z4#pTb#(AErJ+s)6z8GS;f1StBLI~}8SF)3@Mi%0vFPxlEOs=rJYj|8XZsdjK2PItD z!5zM?QRgE*bC;8f2-v!+h#yJPNZ8RqrW2MjSiiE`>&0MMLqkxU({tGaAc5}>A`VQ0 zVC4v7{KA(TFYS*KFyuDXw4hehrHB$%j+^Fr1o!Wn83#TZVO?FFV3{zx5;z5eDFkE} zStTO}BORTDQ7cQ794)(T+KHA%%Pko=4Sp6kS92yXgl~v{0GQ%! zxXyFU%33P_33V46IqFxKn99m7U_l34UT))MGjK|SU4jq+2T~%)kBtn8!`*8`6Ag}w z@NgL}fJYw2UgE%*mGxq5qh#Xa^cR8@%&&LDl_bvWaL zHV>SAlNtM9Gb;S0^7pj?geg8wKv4B$Zu4xo$)LH(m9#=Ku$r6!N|#CgCg}qsc|^<4 zQ#(fXscW6j!K#rUAPfmAPD?t`XKH{6g0h+uZz&r0?VX)a6ns%YT4Vy2m8Cv`$u|M8 zpOLgf7muyJE=A@lb&yb0p#Q^HuPy+8&Dvcf`F$%B`dp)8RM|Oh*XnbF=4swf3KwDB zeTe7~CPS<8(pjmB^=k%Mr2-|ypnk|D6#5E~*Pc8W|M@e!cWXiafu4mSgV`P0J}Vah z*Mc-}X`MuA`M+N#n13uS<&IvJpx40aNV`ZTRxz>EW&QL~h!XT>7$=!TVB{ujgnFvz zDCDY2qex;nBba+5Fr4z*q_R+y`TI9v9IGbf15FBz5%R1I%^V}58SWs!-Nl6D+D?BF zod2!n&c133TZi7NnsD!9RPmiJN?+C7Et;;O^vbJ!Jq@oC>FQPsRD*HDQG4DcqPlbm zR?lf7g|otxXypeF3e`Phi_?z}G{i75vmk1NjD&3vfx!S<%Ls3mA){01S^*Ww$jHdh z5X+#25lcy=U+DD`F0+=icdLl%Q6V7{bfi44bGUIczp#6F%ApZNzpYg3%t`P-!WID` zowN+1!QS2;`0jz0HVr-jhWP@uIUSQIunL`l-nX)}gsnFE&XgnZ6;8@<6oO6U%hRVA zMN|+HXtG!v>Pa4qG?7PT%UjZQ9DqM&3}vuz}0y5M@XBWW^IHX)XzKs z(EK#@vl#FRjZ*(XXz=r*zM%9=HI<8tamoi4Qy&Vy-t*DSV!bVSL*^Z<=)S=v3T5)1 zYHCqKHs1*ezAzH-BM?`f*4#P7UU!h`ewQL%RGGFxP0n^fFTk$o3{K_cajo>%c773RZs=Zq ziG#38OvVdpfNU^+9P4eP8gR+uacu}qxiiElbD;zXqnBdmN$cFE>8N;nIq_R;YJw~k zSmcK+t~;9yKep(1qe?4;0BK+=3W*P2w0owbgTseO`)Svt7Y|2lzI_m?7xceOOR&x$AhbR5xQBJ^sq9fJU(n#{l|ioHRd<6}AM#JIhP)IA zNbDftDB*AW5@e_`s6skC@`jAWywj4!s)9k@norf|&snAuL0|CotWG>nrg`cx#RSc! z7Op{yAK{#-VRB9LW_@O+QTcf9=8LM*pmru@O}{%F-4TSQC>WbemG>^4%D3mDS3IOR zIaQKwWh_;DQ*IQPL;(&TR>1K7&KN)v3OrM0prOH9iu243lh@WW*-5fDk|oTyNvpd< zvbVieKTDbUgtX;=rOf-wsWt&1BG(?W%OhTvoR#+?*JKll<&IegOM7*0Xd0A z3Fj845%f#B;gj1DNdHMKR+SGURT#1t080MoxCkRsSoa0%ExZ5;`k2CEpl-o~8z`ii z;&o<#w}sLk8Pp$K>%rfNi7qe&UZa<&_c|<#zcux8dtzw%dz15W1SmRF_J7~86L+8y z^2iWW6}O!7YdoeQi$Tq)nZ?8AlWiQkQ?NBGtt^3d2*<0}(kMzw-N#zfvqMr6LZx-FA|Dd|5Je${glqquoo{T|$)F_2GqJ|{PE~T6?}%J`S%hC%nRd_{C(&Q7clilCK6=!= z4>p}}5~UykN-yIJV}p<@!E%a$&q48wP<>0R`#gv+dHAlT7O!&LV|ZYc6Ud<0`Lkyk zQ`D_O+jDFUq4=Szt73*eBRRQboECvd7;xMucK$S6O8=upbojpmF%TXyRx;Qyg&d6WD4 zWew@_SqxtgYS{#Q(4bVRz#6HiC&EGpyiP2_XQXG)4F0-W`oGH@c@AHOpg;SZ5YbXO zee;%YeUKlDGlHy`3V~3nvM{+d@{uHsJD@7%Hn{u+1+~E!zsTW1BkB{*&+XFc5qgg% zBKH*Jqq&bR)&Sid|MXY5NBrCK$Lth$q5Bdv=4LrX9hsBM$1~gEH&v6INXW0MA`ssy z(h7&|cAHKmBzw8i2;eK1#RurqKo!v2IS_s(aamcffMqHP7FKNY4LC38u7-vHT#Gmg zk{7rvun4+(pT;q)>E3~Cfn~Xl3-B{&PMW^sUA&bZ5duGTxrVYbNM_F)X98K;VL5F{ zowN+xSFg?P)=%d!i`v|cXL!#eScyk%wXa0PCw)Bs8#%!4kKNAxNn#Al1UKq|8!XGUF6`K7iA7T|> zHa`eOD3sfwKuvEKjuqy~8@c zKEmd?e4HhODTop2UpPcU$lPKkz=Dg!LC7jod^M^6T%@5X6vViE8-?F9cje^dvT&5# zJYQ#(zsYWUkT{FI6-u^?5rSb+OhNeM>YBm^Ll!SA+*vs}BUU}K25u>n$@9?>0lgW^ zh_AYVk#eot%IEHd)+De9@`^yywm}@sT42ovO4-ec*RLaQ(B6PChol!DPo_gwJDtI= zx~w=XpJbU$i$ZrqmOBt-rc=rU!_Wgbo(j$0J9?rf8j|DvkbdBcK{_0X=n@Lm**c1u z>CebJpToKq(x;q}+E-ruNEDwbFn)v3?FhreFZR1UBbV_uIy^=>T0j)(uYE!-HUF0> zYT~!Gr4BPi3*3&t%f191bD9qy?n1dMd^3mcmNYd<7_z#geV@mTT8*DCjUAyNlB|w3 zS@=pfV0j61i~Q(?-H>h20vSKMGo^f&H%`VxjNo{YxdR~+*3r!v1cD4FH#gY45*mI2 zcMU3`0a*6O92EW};9NWt_YnA;?YHPoX&!BUz|Eisu|KQm)n2Q<2!RxNb$|4*_d$-_ zWuMuWfM8QqzTuUQ^*bE4kG-5nOX&@cn$lShw|L2QbP>#2-Cr0^sXjV^f7 zWxLJ+F9%C|JM0W7h%n&V36^U$v4Wm(o$n{a0ZjNvW)gll9m`s}Q-`!JRcn5~PplI~ zAP4$9D5{;6Slyu-+#kG88fwZNe-7rz3E#h3SLdb__kD6wOhTX@p3nJT^0~&}tEww2 zgTX*Wg`1O`os-?>>3`VK^Y}OftS?Zjw*i`U!*WtO6}Ns$TwLxr@hwA*PrIl4pumaD zxRT3~wDSeMdH>%W`o;|f5|}-;G&P}qMas}CVO0z{1a!)oQTQ>^SHT@*1GqfJK+Pn| z%+;cZ%_p3|gKv>lr`l9<_3D(v7)xKr=j+{-xb^QelU1BCjR1MukpQ0@w?CS`B@r3qs1tvjsp0w zbOger0bPl)vvEFPM(~XZsq0=2B_M1?%BaIjTUTB_V8aKurjCvd3I$Dz)QH=RL5P1T z6(W`avSBhvt_YR=JKu$~Gyb4tLo%XWqSHH_N9-whN|7`|H{4z9bxJC!h? z<@a06J4|lC$VTp+M)mUXL)SwDLUtad-UT`4vP&R^>qqh5Lh6<1qVQp4iHROthjK}2 z|LE&XN+%;W=ad(P$Ws?sIFI{%B6#*JwkNBL8XFccNnjcC$msc{W(EQ7 zCFJeafxR07(n;y*UbzSIZf+VdJV%9p%_k1}0>Ta0`9a98{sDjGtParUOngTlA}9naldJVXY>pcA&iD4E#& zh0jZ~F~jhp|NB>OWbKE-q!n^TblX2rS6}v^C*P}#CPK6`FrBQRf6?zpF`dkj$qB9WQoa^o8162-}N@_z^G7N2(Aj<^Ih| z@W7~lc@tDYjqqz0JVo&7+36<<=@v})S;bFURcF~h%h+|-PlB>a-_Lon5d09OFNu&v za`5oLWl2p(NAmAXCo^gz(nr5$ojG~Q2xU?#gQQLZ^WRtaj($2jAtV&d0GN~8*7Qd~ z7M7>5r6~dP1*~Eq7!0$SdS5HJ!LtU6-q|^7?CcKiesViZ?Bo3JlZCC3(Wgm!!wM4_ zMa5%?tb?2b5(GT20G(A@UVbe&Kw<#CIS9W9e>iLj3qui_<s(*<1uW)BVRtk~(& zQc@V@Bhg<17?)O7hywg7I~eg1(!hY#(y-2Dl#k#9R|QH|G^zL7Hza|aLtQV{X@!R% z@st!u1>-fauiwdcctxMNdJGtJLVf{qCsgE%VH!$)7!`3G{Uk^Ujx`Po(Xt*~WN^un*Prp6(x)DFOEYfyaWZ-u=VK_mve}wbHWHkzfu>Nk%R3g~oAr$jCq@;{A zEew)4B>6ilP;G~G>2hI^PA$O-dtF@;YMjaDxJ)_g6w!D_}p+0UOQU{w#YrI7{5bXs~k7}J!M3323y z_{>mvKU)S(ph>vOpc6LHAgF0;$4>c28TU&qpV1Y>x+yIG* zi3mT4hBVGN&}65!Uiwla&Pmm0mfrJ_)I+e1Liwl~2SL&f8B32(?en|eZ=*>Fe#{fOF!sdu8AJP+&{@6}6w9WmnboJ)thf zd=WG}^NflDQG0YTu6F|n2b?&EbMv;SZK<{D)wOlYr}8*3oq|vg2Vat*AzMcZDiOu3 z9It7VWQtBwS5e#ZY|3RP!zJ`5Xa*Kb@!72)6Im1v4HNtbQi^LUy0?oHxu%toe2s!0 zDkc%g4osoqRIcNvqYuHv$xVNMJgvPS^RbyebJYttV3#l7ah$0#X@zP~&L6&a`S_gv zb>*xSS|v~2*gZM(z4*H&Zg-6Vc81W;(+^qu-d^KR-Ui?w!swO&`A1kviU%nR3WrX@ zZ5GLyY!R@0((fdR%u-nDqJqAvCM~Z}eaN&I^ZcXKLIv&y9`0{*pI^Olv%VbU^tbfm zrArBmKFX!U+pAsK0N#ir(B4tQwCY}*q=km%u)8Cu6^A9IuEopxPei|f- zi2{iTvOxfJgLStijxl_q5d?3CWK}r;AuJvQjED%u^lrJ{1<-?jk}kz-b4{=&Iy-Zk zJ>;YK~_F81=R=*xvA7pczYW)};8SwF;2gs0p0 z>eb3?ahqL|p&)A}ZLYiYVeZcUi8%sgPsTxN|Ko5TFZWk}zogstw=uS=H@%LsGML)Z5}|ncjUc|uKVh2bWMovs=B%c zh{90UAVUF-k!sq9m#?GE^&&^}tt>4I%rpaX7)iqvD!zQF2kuP()91nrDjU!E(5D{$ zEoi*lplqi>M;lUUZ=!i4ho1hTX%l^n`(lN`FTI;+k_S<*3kz5URce*=uUx)FQdpMS zK$L*X1P{ojGBcUg68sI0eX~*UQA#HZh*474&pGXVkxue2* zZJxV|-YzAdRL2H;7gVZ-9%Xtt`aX$v66Nl7hJPN2f?Heg>P^Ek9ZYz8#3b!&8yi}> ziOF=!GlROf-(fnV83uZLxxNityA*geeI}dHBn*EP#KFRRR&WwuL7)}c0w1zEFHKYi zOgFv#CGkWgK3)r?H*g}t<6$=L-Q(s&J+EIgJmx3;zu%SSvp48Hj|;hEY>DCl zL7+@OgS+tWpAqer29npf|nZ*3h&MATg#5fa8$v}fG7(cl1G zP0$VyJE5_~VRg)00mq!ur*a(JRXGAIN=JiD}xn+7+P|D4Eibay{Up5ar=r0Qy6 zEmVfE>}|TMwrPdPh(g;C+otZ@c~NF9Ar9{_DnU~_{ovF5_!zF{ZRnkYEm5Vu`UqU= z=y*tOFRl%l7Zf~&VkGDwylA}!%7;@$aq+sV&zbbAd*<*S!WLE3)vd0sI!-)<#HHrL zH(BZF{}~xwCkp0QydFtuP7%JA!U*W^*gUW>{qJ$6r=x=|9QytpE;$(_+(Vp2>42nK z#f{g}vGJNGd2=KBsskPSD1TV4N%hoYP&$3in&*eZ?8BdTJ-^e+FD(9`(w}rL%p75o zz!XKDm*&tC(e4hmV%NJM$5~lf$<9~UqSFdDL+JM|m+Em*oy#J@TxnrtiHm?DD^`P~ zdaCjK!e01(3j?MyR5d(K3Ax~`CXqtKWTAzzz01^gdfLPYFI5WLC4@?!G-DSrh2d?z z0A)kNAA61ec6cv!q=cH4U}#7P-s?SOB};w#_6xcMla`v=T^Crzy)Rn>P#7&0K4mo} z5bUDXj(2w}Fqq2c+sk)CLqh{w8M9Kg$snUoEgM!COk%OLwDOp>s69ilX=3@(`p%=! zI`7{-2)+kF|0r+Y>fwrLv!sO*q+Ip{t;nx!S? z_vKvun@)4(x$JOZLO(t1ZjG+Dt%P>LBpqUE|Dz^6Ib$2j2N_rHb@tf#O%( zu4>LhKnzUI6hx5j*l$V}i(0vuZ<#g?=@oRA53A-TvpD>*4aa{=0)ev-=#>A3Rv_;_d)5%PQ4%SM_vbeOLfK-xiHDGq@h~k-Od@ukh)- z)5La#k?Nr%C0~b~Muvd8c3gf$gmY;7&^XpqUbN}ShZcRYq2uTXuh-Ktit@RNb6|JO zHavU!$QlA%=5E$Sw!_e$0$2Fh*h_#P#KkRKp~Z~9=X&uG8nk6O7TYwDtzZ6q2jdUE z3@d&)M2VP~a1ejDXH=0dKR#}JIrE~SEc)rMyL8cBJIz!G2eI5Xv>6GxS)JJ8boiNM zJ1I+cn#})7t_Exx@A2)lj9DJhs&HIaP#5%;GM4i>tJ+QG?)81uBJ>&*?!(I_l>!a6 zv6St6_ILgR%~?~EkV<8t2tuOz6vP9;uqdivzphEfQ4q)aT#~Du&(t;|4}r@j5%55F z6m-C=p<8!one?%c$}UQO54F4}Rpe@}aw1no@>j_}@upRY8ptApWpn!bAAlz(T)wcV z=#mUzpuFAsDeXVn@Xk@t6wbeF1j|qdffg|$1P_dX zOaLH2&HU}_k*4S>?)1X$N4cYA8P1=kkSTsFR_|&{#u(CSlq!!53dkz`!;!Kj8}?0I zo`tqOSZ>ir`W0tQ2}MWOs|=qLz1Q~UQbLN;?;6)TI$1mLPV*eqJT`QKR0GtC0hEiM zACmOHoeYdr3P6v9j$-dh&7QVcCvXt=S#fc2RQ~s#pa6{^7utLx>awE7hlY-th{)%^ zYj05yjg4mL=^lU`k+ad;bn$1S# z?0e>SXu{?B*d8}^EjxpPxMy*!>heZU&nrL5`9^l>+!lKSXwp-K%QBczY4>w{W z;${~Y*jm7@2JRSucy|z?6ZdV}b_FXieCV2L&No!OkUI-}{rWW=u=e+jo`{5mVqvs^ z%TCSA`~*e`?7kQ%T1YoGX-7v#@U*g$b${!@l+9YW=|ihK6j>G88ag|xn3=Wh^Jbbu z4Wh?@*DT&?Y|}D5g*~q2H;VALWXbB;FI=kzbW!F(qX|nw1P$ZjrZMUSA*TFcHvFEP z6eKi-;iSu^iri$etfbQU32B8n>=9qh5-lekYC$&x>C51~p2-B?6-0$$X^5>7q-2>9 zDc^r>ZdMzv*(4yBJ(nFn zuoa$9uzF1HwXD@=iA7rKFa}8Dw>Rn6nusy@d<=t{_5;smP~L}LNb>W;J6SnSM>ml* z{NxKjm)IlH3d3_#jNjY?n3N!kbLXOSFZhY!15QZrvqUEe0!Hk+;*B6uPV+k;^t+W| z=L6WF@+6b0rA=DQ^bL?-nB*Gm@#VQh;!qimND71siP2iy?g1V_ZJ*xS#!@5U zv<73Kgi6ZiJ@2kPEmndoxuIYtgFkUgDjyOWl@Yez-zbEu{&H!~5D|NB?OHkikiAXg z-!#u*qLDof3F2R7pl0@WdDY7As-&ZUtrDL2FrI-`P+BK}wXyFi;`#H+ z_4V}!CuUdI6dH+mY6%f~>7!<~HfbG}(GDL>%ZPdlNZeJRU{f<8psh`bRbcht&Zr2< z-z#7Tp?c5Zu@HXv=kVagOZf331tz&WSN4DUV=piGlHrW{pNJy_$?T-OKET_s2tMk@ zmm>^P4tcq2GR4Q;{_W0#y#mTJ59?>HKFI?(B3Wy0uVwLU#q++jT9@i`pa4^Z%P*Ko z9?rV$d~{hoJ~=)JOG`=8(=%GxKl=4j{1)fh7*SGbC^K(D z=b34dqpcFGte45Sqcx(yh`zmXaD)q-)Lw;D*>j*D!CqninRWeZ`hg<^t&cnC;sz$NYdao!ZVEnVR02Wd|8CUE|X zXs)DTu$u*L@%15*&FF#l2mY|{ji z@cgCWmJ4DTq*5`M$8|RlaB~LhF%Zw*q@*ApcgFbn<xvGn~@K zR$;pZtoWx5_jHQeu_QON=}`g|jixatdbMdsrv9`F(It#b~JaL!14bOj}M) z9-~Kw$Fql@99`|}8%s*wzqbUYz~XP4w6TYCgFl8noi1|T3kz?G*#B$4=40dV@_TFCYIBj=-Q<+b-*Ac!Xk^WjCi@(CxyC=5-Z?KHuea++yZMUpMvgG*$ zD+}YBnxIim47`Wa4%79qj?>iURSNTQkD~_6H#s$rteg@!r05 zv1uq|Cla=0L@Oz)L!vl~7#Z->NwnTC5g0;^l5A{HK4idsebzdY9uI*wr{AgB`(rU# zS5YN>Vy1EuVc)L$^m?-#_aPH6Z%j3#J}1Qq+s*=~-CO%zz8uReX{f{=XhbQ*-hcV~ zh6X@i3tU0F0T<$AvRN8i^7hZF-qSCK%(vQK;0;^h;k|erN|D|tLe)LMN!5zOmb(^0 zaUS^UqUX`4HkM{;b2w`P_hkXx4YiPnSBj$`3f4;!j#~I22hDMig|mit zx`qav^_=OSz&jQSp&+~bYQtmpJ4lQ(#SK+^Rlk1OBak7wniPniYA(I&#~p0}0ks8B zk*#rlY6RNa)kPXv*q#M-Wm0)H4kUgX(*YDoJ$wJ`EuQ1lhub|`W#SHAxox%Dk9i(> zdLDoOt^?D#gM)*W)k+n`jmu0y_X9{}QHopx4-kGxd>n(nZ=7=55HQsF;iCTp*k4EZ z6~Nvk#l%i4YoK$yg6rro*$@$A$`6>_5I<(WME-U(Qt>-}7B7nMK8XJ8c~#t<9bedP z-U?%mxyGNI6>4+$TxEBVxElpf%72dR=8q#17&Ln0g0<|>1hQRrx@hz7`5~qzBoB&a zmYu;j!yglX<7b+7wGj_-igS_Z!rDLe8Tf#vrc50-Sd@orRSIW`TF~89Z&OpLsc7N! zKRY-GvB_eSxKSoj6Z7-~o}Pi19OWz%T8mihL}AUw#+ zl)mGRXI%#ee-kr6K*Y_=%voZ0|NHxvNt$HsiZ=xs7eN$d79uB;6tC%y+5h*eErrI| zu5!|R;fZ-Xi-g&0rYMAI)H~ZfrLT$j>CcVJlsyb6JJ90`tE)_^!@;TyJ?INO*`H1m zAM9sEMTJDA?v!CLj6^0ppA=D4k{!q#V8+4-XapUuF~P&qj3$*6C|UkW{s4BypcSiC zR~~_!EH?tJ_~SF*^wA^;xic^fP{to|*L~AXA{TUbJle0F#$B~?%8AG4*=*D|vs{%6 z)O%*NIPE8%JuBdBFJ_fu+^>)?OQA3#gy)aY`0-qnPC~&8QgWuIGI9A%qTl99HbK*Z zarN|z2p5;CCdGsJvhl>Ko8>I+imI!XVi#R$rM3Pai%+Ko!Y5LSX@vrU+!g~!uZBX3 z+Zd=W(u~=gp3s^MkSR!{0dnDDN#lHp*!W)?B=lKS_rx!<*&Udn_7-wsAo3mnynX!p zcbJ-$hv5%HDH$|G7)YO&)SclFg4!%Cq(BW$SeY_kJ_~=G>-+{s;`7xnoFYP z0cqy)i^Iws~K>tg>EoJ1LsZHs`11vCTTV!(cU zh|OgIUlcN>1Hhc&Rvnv_f85X`FE><1U&Rf5EhB(~_=VM=dA;X}5rf0S@Q~8TUM1%> zSBuNXYR3fyiAiPg_(R!3C@_XtLTnC5IcAl=peL7^9D(BdhZhT+2W2uYt`d38Yzj3* zfQh}m87QvrD%JqDHTU9)@n>Jj!&`Iqw9~frufu!H9^x2GXLChVl@?b0virF}Z?73U zw)EK2pu$q~TIfE-EE}*rnA6l_IbiDZdg=oE*t9}zJFSS)>#DvoMjjri+>sI`nJ1>` ziph^+D(dPZ?vgqk`l9*HeRKg(#@CydY?@8N-css@!(>r6B_F_!C}1=hB+*87wG2Wm z^8UXvL(-e-BYbV&^I;zszzDwOE-5dG+-G%w zMDRf7iIDnJN5^}h^uw(U!cFWdoza;#{|}#p00;z_(}ru|QagB21mr)s8$uHebY4iM z_eCJCGDt=7kt={-@p|3V6Ssyjp%==iJGzb(Vq4ZNYA)+dXas^c_3gd3-d84EQO(Es z!MM2y3N%}nm3+X>^-DLN^x12(=i8=EPBu&Wn8%YVh#9mxq0t1aDCw5|NLTK=dN{4a zr-5bc7j?fZ5Pth_Q(wiWB&4Jytikc^vtg2D8DmuyVVQKn=G}XJ1`j;>i?9oyGo|+fwOoM$G&12E?@;w9n^ZLyfSxL_>sXl)G{Jd$Sl!V?J zse1QoFI7F=7|LJ%eey*E^(h{?P@>eEuo}vanJ-NDf}by7=@v$ zbY8mSOsyFbKgN@d$evb5gyaa~`yn68P_8_<#FXEhcUFFJTn@-jP)A{SIXj%o%E)++ z*#M*oDdf>AV|?r_ES9jil?#BK-QwYiPKH?G2%kr`=ws0oj)EwG^IPYVmI*t7!s?54 z=%oIyNtO%s4z;Ye*@|x8gM0r*NEtg(a@tRZFPHpznv>XVJye1CD|!BJ5!*y^zP)q4 zofAsGkZN*j(2l+=%cP(QLX)k_-5yRZKhuuGi0hkwmUL8AtJ#Ub!EilwjSYp{ph%m( z@#4wMN0*`*j7Zl-+ImZh=iWh?O5rQoFg6|SqVD5-X>ZLO!EcX-)3UW!ZNWJ%TWy{}|!WAj6h7zd{>ZAcwD zf2e0)I!L3L-zP|peqwN(NFR&J4Z}Ufwr&dU8lYE{ zKy#zEwpc+cI&ka#K>u^H?}rRlLx`TS! zFt{e4;gVgW7v+=Uzaz4GgPtDvO>xp)!O!-Vtt~9%_h6Su1YQOm!X3Kf{le{%yd}?_14uIHE(UFlekR3oL;~^yI2VfX1D!RoItHe&x zz=(I?O#OH0{_;kJ^D8IHn|(u+f60GEOWrq)!bw3WjoY%4eK1U{eX1G}u_yBE2Ut8k zR@Ah;tTosv?4Lf}{PnA+zfVnFwWN4bOSc)25cvRxX>JBM?DXov4Jq&X_wNhNU41M) z6bvRCp#3`L)>S-twf&{;#YJ-S1e*Uo{6Z->SN#QJ9ff>lL&M!WcS5LcHqq>EQ5ij_ zeKge=kkoNaA)A&_SmCjQT&v z@YIc8l2F`cwtSzc**M}no{SN+!z*ag127I70G3D$y9gJT2ApdS790%^BJ2LQ0Fb!P*MY@aO9}|U zyQvx|u*>Kx?LuMFkV}HxNBj2A{I5daU`Ybt{37iCD3|kw5t&v9jTui8D@2o3^{uP4xd0Ky&?8UOjD@$=N@R~P1R1z_uCn+7Y&HbF{c_}6k7Y)I#S zG%o)18lMZ?6AfFiYpF#6I|@s!o&!~4?rhub@Yf{h7v`GW&{|>w4)Jr73ah8DznaM) z1QaPuUcTIRJzMwKIX$~3fm(d2i3sq6^waj!r%xL%{7*>Q-&m3&`20;oy;s!aG@n;? zE*@d=moOE9hjSIG095pB8>AG5SznL#F@{P@t;;iZGADY&jK$;ku|-s0?S zCJQs6VMFC=SF!PeSEM_KZbABY=6PTL+Ry5) zI&Q z4U7K>>Q_j~^7Iyo(|2A!4Gt?P*&M35%DFh7;ApTBMlBnRA|sZ#K{;5|l%F=Wx#6zK z#pQv?qAH&{G0duAqsJx0U-zkG<^N0H{wJo+4u2N1=!8^cuHJK)2WNiy9!@{67{w2GpG8I|-O@5?U@-2!{#ah&iR&wz zF9E=i!1D~L89j@~0JFi9?ms@i;{)l=eVv^;#8$j3PL;;w!vqyOcn)emcRn^aRq^dQ z{XS~bjK@(jjx72<`h6#w_vw8oWtUe|(^I&Y-bbN!1C9Q3^%XB30@zTjrXx3(SV4JU zCYvtAiXn)E1;lTIgEZ!&3=z19f4EI0%Fp4}bybG++3~{fW0oZhKYi*wr3c96DXR#t zW$rmo!SA3O>yft#&fb2^eZIHJvFaS5K~}2(#eyk->W9w@0-k2V2meaEO16H#C9H2( zY^SQH4v4s8;9yv%KDE)Ey}MjIZS;tlc9I890_|tdUY6+$!Nsqx&d^glX-!2Cy(YZ# zDK%4lZkV;6wxBSW$@o)(2punP)4V(v2ix+>iWRU7xivNZ4xb9L{q4N+ z|0=ySQ8RIJnggA*;&9#ytVXM=b}wIkiRI_}Gd%d;e21#J`SbQlGnK(q+<@BCS_tGY z5W%f;YhG3)y;zNHkfGu%K#=6=-qauhgy{}~;ni=ED1hw|DBEH(mZ__pdF zP^J~*%<`b@0S`@xNu~Z>Sv8Mt1twBSWF3@@Kw2R3GguOc5Dd|s7so#@j?Z!%w8I#x+xJ$gMmI2|JQW={J7wvx~l3(n)dh?gcX-> zj9WxB8)e_WK^zP@@+>T@wBDyXBZ&_q9wP!!j&5FA(0cHxkcXA^KKXwT%HU!^nRz^r zeZQm_7>o3?#S)|AIVzkPxL>p^ESR}i8)S96|F*+`oKNB73Ca%p;~PYXD_8c=^Dsa| z&gm?(t!aUKeL;b$re=ifl5QGg&#h2@|IYqhpE=Avn%yiUA9OhnH?C<4@V)ByEzkQ2 zRi)oYlXQclt<>8+Ld&s`{IVkHiHn!l7~fTcVNcf}Kun+(E7tK?k_Up3pjUl~;`NmI zZg`08gb5G(#@d><@5Ookrk$Mbak0}nsr!NJA?&>2PX}*{j2NhCNWo_JCiN^&)86WLTXY%g8w17kL{=Zj$I6l zjHYL1-Vd*(?(~i2@dqL!i|#zL4*F~i8W10S5f=0DA=_Z!t8Rpx_X$^p>x}K0;Kq^; zJekPtyT(kPGWRJu`C-C`l*r{fp!TIX`pJ&*k2JO9!^v+JPvM}u`v#LQYwEMG#Wr9Xr9HIom#Go+Z5^i3-gA68Yt*H zhMWwn5{4ys0fXVbp9}vLSSFyp2i#D0qqVZbf4@Cl+@Ocz(Ojc5YgiMpnOKLy(?=>G zDZg|vL#;f(#n0I;V+Nrg=Yi+INcfMKuBIjqSyfF}kyq~h`O};j*%VCI zVxCPyv+m{%s}D9t$KG%6W0na#^wSF<$Q~Nx2D=Y0KSJ(sbn4eEalDBQXre;i z5_bDjf#M;?lVAy6uKvsxzf|)4fBGR3g8xj5Am9R^Yc&+~eyGR9P+F`jec;Vnxk2Se zLO{Zu`in7`hXo2QcoFy+Y<)j}g2SF2rfC>JKBlCA#49l^rFeWDbd{5qs!F?{4$bYV ztu?u%tu}!OZkYv0;rtwDfCnWN?eG08pnBvLVK;2}>$YpXk%c8$%&#~)yu2FuEzX6|s`<=wTbKuG}oOG%YrGg&`Cd1h6og%mV)h`?p7wO+QtWRiGMIufvnSKlBldwd(IWX=8}~rd6>dY_zN5B26g~2!oZADF<$3;asUuKJUHM}DbfM5bHeVh zI1PQBM4P;A#YG|JtJRGQWW)5u4}0G$2vEn1?S!eK0v)Pd8Mnr6;n^)@r~3&pGR{LL zH59x14Ni{l%|&pF?R~yJ>D)g#>i{WA1$A}KYvn&~M_oejr8Bg=N8d#BF*J&S4Z+}*wnb2T)^U_nv-)^`(gsL>;Qy9)(BAXJO8h+4CJw(J*T9g zsKbbE9YLK+zh;|;=Aybrt=)B1;o09UK#12~>xa{K_mcnir9g?9bjTO;zc`t8b}c@w zJzW%-D)n~zC_TUjDTIfcH2^#uMMwE8i2I)!UAlJL=d6WRojel+-1qOV1)k8xTIiB8 z%aaq6fj!~HiznonrRHx3OfJh3=Q1F$Hi^$Imsr|6{2_u-NmnK!k`IP75I)TVI@)3!)J1X!QTJpYyLz2+4;-J$4DgG^guV%|iX8l<( zyp}y#|2c<2n6c;UQu|d+IH$6#-d&3dZk{c6wUFLXc+D^Sb@tisHkYhWShUK^n5MNtU6kzu}={luPrS6#HQf=95d3Y zF%kA*bd(x0M!C7TLL(v)D0pWM&HRc(Nf4nrwRfXL%vca~gc0Vn^dTP2b%Y3w2bZOa z?1qoKcR>ff>$(QdJUrw=V|p$I>F8VOH$T4kmRm5TWAH>{e}pj#H~>&&Kzn9E$&Xhz z)$nWwcn6KYkt0&)|qm8DgTQ6pzqSU-4v@_xI zLWEh+!nE?aoAvM1ly8i3o*~Dw2eneuTK?y?rqNAtLYl*8PYakTynb-h<%*{37aSD) z=_}j6)H3(*Gr|XvS`il}@hEz_?p*xC=g)^5$CV6%9og@2cef@yu4_%=k^~sH_ej#4 z-{o(;JIEum?B!*ut@T4Uzr^R(xPMR7Qky)^OiB6~?5f@*QrKSUGicx|&@E}7-`Unc z`33a}mpfxr484EA5b^2L44nDmM_sCrgRiKl2z_E${0k&8y9X!7$B?)PBM6Y+Ks>$? z+yA`2z7ATHdS~AlE-W@pPVVqPMG7hI4_3?5_Zle5Lj&|Y(8oX`gD4Xld<(8kP?~^Q zG%FUCZRkh8M8*})@WsaB-s0N{)U@?U%gc@H1n#19*+=>gry-!h>-V z_ReX)?N_uAJ2IQ@lzkk?-DW*qz!~TR8OK$l!}T0GR&P{-@{{Lk3y)HZd92XQ{L<1t@F20WdZ7&>EmHRS?);`ZWPkm*fy02?^dn#?x zj$q=KJT)S})}og8tm?%8OL#`kOM_GcI1@RYrv!s{0NeLJ>`N`WtL-N#*?X|DxYo@i zzk6~f(&qqz40H2JxQ%OSz`v_RPLMFZ=pw|!65+(QJw$rN{@mR#?li{ezC`5Ks~I0T zM#x_NG1S@`{PSlX6cUkBl7rIgvCI{GTqprkQ7U&Wb#-xpUNRmzBS6gxhB=W7bz8o$ z7+sb^&PUYjY8f8NdWj=ryd|3TLwEC0?=();9;J~VfxQJb9H$ch$z3^E>@_8 zgIF?9y(%kTrK(~FPdh}xM@0cO&OHH2d~Ej(RLj6S1PGj2nVz7po<@Ds$fDCNL8o9U z8ECSC7Il#813&^-6yUkYo#Vm;suxv*Lu-TW4`}(gd`m>~EaVlGRlzOjv)pt^{P>Z- zMw#C$cRRbS+=8;w@;~rb861Q{=-vI5V+e}{QW{j78{-X!P}W|MEBru2WWlWl`zHII ztzDFwb+}~@&?9jTE%v524+Ru7rf{i5f6h91SX(O$l1Px4?m!%gls1Ps4+m6RcB9)V~JG{M5WB8OfKXq-|`g(g!(zorB zItjLa;}}1Psic-3E^&4i+itqf)auO#isI0i*ytGjGCyex3t3ZB(kR5&KN^~ljOPH) zd_@JA4Q?ZptReY0_i`Pd1INQaMmjX0q{D7p*urLy4 z-kIz8QlDSH{@+71Gz6UUo^lJ=G$5lbh`RzgaZQE*5!l}_g>D+icLsBp4;(&ep^21CWBuI{AqrB)p|Pb)dArx;Ishc*xKB@%f;mbvoXHp3^V44 zL4CU9n#xDNu9Zm&6J0BMCVaQQf3bAsKq9LWD(s!jk72g%f_EINhm7z zbdj;*HZ=kQg)O0lEdqLq;VyvjIU)9cwqRL{wvekdNcvfa))kldxHur`SiI)grE^uD z+FhJ@PhKn05-y#weC=o8au0~hKYwEBkf2UL((A@^uZ}o1UdIi_Ay#-`AdS2%)9k^U z)W(M7!OC9U{t z)plsY>zRvzh4%e+xq%-Sg=&+}^;!1UZXIG$^|{-j6Ry1P9hOc2GN?%c4jo!3fHIRvZ44*;7H@oafb5z(e+59{}Qve14RK32nC5`-U{gD6eU3T80yjq`@C9OdRy;bjxN^XlraQPOnwVvr}A#d|kD?oScKhzd*lKxVnKL3$SiAYba%}tQ_gM z?R{!SdomIiTfhBrb7Gode=P(@Ej5Rn`{zs1i}Unp{Sv!W`O#+m<(*3xjdw7nlXBD_ zLJRNxTnXP#oZ645+SN2{D{-g^ww%m6D4@V}T+RsKlOSoNe~R?I@UOR6UHE=-!3MF2 zOe05MXwQtFE&7fNh#8yQmC3x49zii{m=NOEcQ|Ao-w`=Zz0Ie4y6OIB8vZeCH~NHPDW0yUDa>*1EfTO4EPJeDG@5))@KSJWVhAvGxpVkgd9S;gqXLBeka@^OXxhC|mXPUPIm z(z%_j>SkVl9#MZV?!e@<-Pp9{bKwot@Ww>_m&$MOy4gv)x~iKN(}l(`g=O#v3#Z4V ztUHWJecp!g2~YtjDBHtUF2Wz-XpZW_8q=4GCX24Ubz*d;Ic9KpC_@b7uY#aB8<;cL&76=WsNwVm(4 zDf+0w|A%p+xGyfEZ0xa2`n%<){SC(v`O%w~HvKc$Z11-(U<&*o4{%fT&|gUSdu)hO z>-IBL97X@G>z24#r1w9ZfH#75S99y@>J5(BITc6skB>pj^&C1JL5v6@YarP!T%lOK z!Bg9_I9I;6-)HGQ&@LIU=DcZ%VKRxVYG!@@2+02Flc__6aK(75t~pzsF+!+BP<~1d z9#PyGMA zDS)zJ_yb<70ngIar_|t@e@IA|{KYA?=*G@~Hj}vaP8DYuQFU`q{8_C^_I-BtF6%ON z1yCfi^7srvCn=B{aEJr`2q5jm19SEJq+x)B5;06u$w@cZ4V>rM7^%fVyy3=V1NY4J z(Bewq03gZ%QZF?Hn1eKxqh3_YMiUT+T#gOQQ77zVAahp&$DSsL9AQCcA~n>};UT_{ zCRS2abuV|1jEZ^_M4vFutay(_Cssy@2emMynWa2DzuiZG zB~iVY1q@4!jLz`F!V_)JaO<7jqRzLyI`i0jL1736yf5B`^yYoMuSwPV6uB6)Uf**o zi<7sX?RgvGH@SR3LLh^KLJ%4!lOS%|iFjAe{``LDK@DejxlD6()lpYgK_8)K}B}SkP!m>+x3e@qw~e(XvUYr@$uTR zRFvdC2PNsgOG(egc4whLWSbGBx4+;po134{%6_hr^6>n4?b17_a^T}b({4F-w035^ z|M0QKx8s^Tx&Q8m3p-oe$-;UL7M2iTNskp)50#P~i2IF>UJYjuSwY1YtNy0f22 z_d)LH>}=Ss1Iq&t!0~q;jE=%HISQ6vChGW{F{4&n?sxz6Jc%OiZjtGflD0SqMsUS4IqnO*(Gr8lky1e9&>c*sI;2X;-<9yva}P#a}jl^jnO zJL=V;e>C_2K^NiN+^xWsK)0$^^<}(a4wG+Vc@!p1&U=@e3-N?Ow%Y5Z|HZ0*mU))= zzfM+VCYh0um)=GxH` z^tT_)NqD0L0<>NoC}-15cLG zgvtrFnI0xY!xT)G(UvtMd?J07ZK53ZyShDohWTdDYjf+`8)!l^l2u!%@2QFX$g^p%7He9XXcj~Q8Go9JPn@;;Ya6+%RM>e4}`Cppn$xuFTblxzF?T< z+8g-HWQ&Yj|1S2qWA1sfbE=yB_mMk2g2*`GryVC#VvC6+3e_BJEQunQogEy=H(z@8 zl24xmeV+kgY{OtWVw$&s84G#t6?Fb9dHwI*nvSo;W;g*1I;5u6F>4D$i|lGq-PYy) zIBF)ChI@!ewv52n9knCuYxA7?r@J*XveL&p8r`*B=sbA!6;NQ#4 z1n%H#Ru-fV>c%Iu3C`A1&ROJNfjkWS`lDWZcrC5I^XkJ?YjyhZfz^np{Ox8-xt!<~=cn88-)7Od(bW3QvwfW0TN> z?OK4yaTkCg$HvAWhsb-TaSeF|IK69c>HzX7EZlBfJ28TwV%X*YsfkPX&TQEnte<*z zbvXB8t2!tuSJGJPm@A*$^&L0Vz*P9Cu@(SkZ6X^N)~3X(5HOR;wDjzAs4}^ISX_2C zWKm!adAnVt#T~Z6kVB3`prvVOQlV#CjEhMA+=F|YhI*{wP%{|Wj&G)f)FOwtambv6 z(M?d$6IN^|p$+}VhazpQM=+?k94HfB-Cp=4T*HOnS=WB8WA(cmAM@9O=wQg= z6A)00DRo`b3Iv@{9ho{6!JC|y4L%2>Sk~-LZ!gRy7uWr^x2)K|n<#I9KsGT2oGPqM ziMS{UN_nyIL8txY9Uzh5%Wwbo7>W2+J?m7J4gq^`=?uX8z6D+I{vt2|X9G&{mL_*= ztV(oKc-j-d>x=Jjv;b4%38==!6E71|Cl{K7K9axR^fCMerGt&RK9oNrCU=p%a!K;N z-ySWNuk?qrD1T5WmZ26Ma9KK{O1u7Ax|j=~bH4>?S_NhK*?PiAN7LHuZO{&fz4jmhcf${ViVK;ebB?4Y8Sykgw{}2RS7lH)~ojI=!0n) zYoTadSooUty|?!cWQ2kbFw5#jwOyHxt=ls;at3^(bGmI@qizaj-Ra#2>oMAQRksp+ zh%VZGJh+|jFso#YkYML$mlC2dou6mTK3=k9fp!_C{^DmbBsv~BIC>LY#XU4Dl!o(g z7iwDhM}Ol%7-L>0;UdhCW=v1WllBj5t1m`Zs-hp=_T%8Tt@|EE;#!~i!>zDjEzRJ1 zyHE*986yWHU1-uUU&vH1+^3VAw_lx=pQwU^_)MVTs26I&q0)2zsNY|Vg?}kXufjh0 zmw(As2WiK+;oKY!Ts~gjo#z*r{__Tq7yzgSA|1^64sAnx?E4qEYa1)n)VJ!~`)|vW zD=RQFJPDYphna|$lZ`DeKffIvk6rnk;bIoKDH8lmrB1POHO^yJT z-Td~APJnyZv?w{t3cZvvJPtnw2dN=|TUtdsj}9f-h%nY%fO7QT#lP1lZbl3W@jm!2 z`^Vv-2;@i%yXV*B_H6BufpbO@>CwE*>?;0mOdbbC=XO3#V8Vn?Dg813MJXpKg}LE2 z2}arVsJ}Wh0Ce~v=(a`_ONKOxB0_Uz{l#A2+t#*LjHL}?;2pXNGryi8V^KC%+dGEw zdmN0WLv*8}Zm@*PBSt>We&Y8#UhkQ#d%iWc<6$ynsg&v#Vez8WbE}>zf{3%+raI~d z(GzM3{1BYTwnrYoji(n8#5T^;7${ojD)U|S}zhZo^`Vy; zUPzN0@ZAS7t%fx5#3RdfY%iw;1D<0yRw_Thr>P0n2c5*jlT%PM!jhw_qZ1kxRWfRm zlB+-Ry#AG#(cAOi5t7S4I;P@8Ib7O?ITzh0^~}sHDo}{FNCpIo!Q?5*LM?_^R77xq ziYVCWH>X9D^7{a4z2CeMq;Z4Rd;Blc*Pd|s8RR@_MWArMuJJq@cbHL9wC&AGeR-v# zXGHnx&2SH+(UU8>4DK5qTti9Ua}kjU3Is~3J^Wu6d_+keAI=#iF1B7?aFd3K{F`|N z7=e)|&;C-NbVZGFK83b(UF>hL@0@p~pLPY}T?!wL_1QX^_Fw<6gY66QGXQL$HUf&b zOI|*PPL5BLJQ!lj58hpal0n;(a{MS(axDsZ9_pWcy>M?qmQ8#^rhxm7&UGiPlW zHM@JEb>dZZk~mx##WFB$c?Pk(4@s~6!-G~l>lZeyXscvHUEQN!($@uVUDHKrYJOn4 z%!nTu7DN}uAdOK~AVJ6(m+6p_iNtAYW@fI5q~YhbRnP4Gg=982w;LfQWjbc2rWQE4 zLKcyFT;T@MqPP1GY=yhKhe)JA5%q8RAw3fc=~C+cXHZ}{gnCQf;bAJKRYy5bgG=@ITx=dR1(4f4&ehEO9 z(uIlAKg5=%&B1JHx*PoVB?r6&t!T^zG9dSuBZ!Uu>QeQI>bm&0sh!?YUCk6XeFanZ z=T3d+XSUwj15l^@@ncof)_o);4vGgSfZp7YK&`1Kt$)q#cG_(FqKVy?U)-fcmA{|! zwnHRtXXj&(0|9Uw4Hsma(|heAw`uFOdFy$F;U3M;M@FEB6N7I+;D2KKi*509vyLHK z;DPusf766qIT#tgxY)91O7Kt;e6}|(tg9oVFSRq!HB`BV3n3EJkYWBbd)0>jrJ-G# zY%~sChII@Z*FPT^-bTw_Vyeqtx~)&R=2u{_wj^pZ!0&;9fq%RRgyn-*2bg! zrbQR}xJ+6?0KWx27p9xLl6 zi9q2o<{dM*WG(&oQm zdEN}p+nJoJ%K6I|e=E8+9&6Y!p`?nrpbcFd-dMPfL@sXUt6amVly#LEa!aw1hJ|K~ z{Q0ww#r$A;yu-5IV3jQEEkoK;oJp$UY3fFN?NlTXcc| z+O-gNBBl)MoEs~vt3YDHH40q{=)f)s%GbJE&DhB3;w+}S{95r$xxM8tG4G6d{((XT zk{r+5j85BTW@fNgePW`Hu5Q<0`1Dtn5#g$-aRRlBUCPQy`|Yx3$OYE)ta5BmG(m@$ za3zG*dQDind27Qe^#MYXh9MLUP$(o=|Fj{Rx;}dvs^Hl&@yT?_J!MHfI5SAY(HWrx z#Mf#bh~7Vee44wQoFGhw;$9K2eRQU!f?i18W27146M~^VhV5|%T~~ZYQ1dNecPM*? z$rAkVFj)c%1}t8t!fNhg4fGdng)a^I5fwutGsY1rbV?+1GW_D;3&7H-PW5xIO@J_hS)P6#XaJse$FEuf;8K?-{AVx?>ogM(!tAi^3n z-~j-8oZNT$>o*^MqB-82bZ3k-SNn7I>2NO;v^X+24)<+jNT3l-BRp{P^@EziEJ}p@ z-M0)j>uMat-j(zDjwohUR(OZBV$35Y=-?)VmFh7Qw*4zP>j>LO31s03gyVzxsAC={ zV39*;_|VIq3I?ZVGj0t{!$jD;w|SY@>fNFo08n~kze~Kcvj&fYgn_36!EpSFZwFr2 z(A2cbDL)+4j=Ck>j(`lyU)^c?@4h`jMDm(RP(L9g(ImrAz93&tfExsxDRPj`f$a9&|mjQeCA7|I~llC_>zZeEIJdq&BX$e4*=?{&dE576w*x6V!aaW93y5f-WB8gFYPf zJf@A5x_9;BY3sY-uOH6qwA9k7Ht^zV}vOYP-)nH#heK8JR4oUV^wGul#wD4!P`Tp~)*c z1$@SCl*KOqDyOF#U^mh&8sWO<&vbe&VQc%N*K)kns-$%ug>d;ma9R)C7pTMZ)MBy7 zO7!Cevh;tgP_s>o#CO{m`^(R7Z0}cBaVwuj?Pwf;Xw9>k!O*`xtUI6FIUWkCnG=u^ z6GLK_@&~Bf7*7`Q?3F(%CX&RC|Y) z-?6y3_xTZ3a&ljai0A6R9Woh|=;bgHO}yZ%Ev<=92tPZ>kP`IrTy8&QN~dCyL^4IA zB2f~NHrv1~jxUWMMkCrH#P0=bwGI@lzV$m{ z3_c{jV~D^l{TFA_%yZ?{_zET0f{R(508i?N1S>UDR2-E(dOI>GbBszcMs?V|cCu3t z2#&)+y9?WWOpZ|m)hJgGb9i%0DcbbY8$S71MnVSulIQ<)YzHJ|oy0u)Pf|_e@$Dj+ z{XI@?hPR?pN4$p*SOwyzUw#X~VI&k$B*t61xZBC_it`5n1`|Gr=ueVCUJjxQpUZhy z@$9O0)on+WAk-X13eF72Ro5DAA^&l+nT+3jaMOWm@Zyv zH=-@Fvg5Qhz%>a8LW4tTB93zlAYksbgYo`SBQC7ePo4Lm;!gS_@Kb|8&%G=JL1OJziWNTsi8FkO-xhkU+B#Uy*rl!w|1yw)bD9hqkFth^E`} z4Jl$g*1#T>;@6~8(%ILU9} zDtymhD`s-G1pTE|^49y_M~>`TEouB8m4{uS-EU*uJ!i8tM-JHpDDB}C1e5v&j$8Hr z?AB}WYr%Rc6o=u|fj~AD706m}%>kP~nEk7c)`R-(-5(N2qGY?i+VF~=kfQMMArY1t zj6{O0tO96aS9f=)3JT2t7n7YP69V~t40~unHAHvfq%}1fy}r4UmY%MvqTnQyf~HrX zrpNS)uj#QL?N{n}We9Dpk0rUN#RG9}j-&)Zo5~w+EEh32o zk;mF+RTW{rn;QCG&Jwym@(@cPWQ_3>B}m>S#_I0>SJmF~$p5+Z_RJPtn~Wc$LObT!fA+ToKOsa1 zilYSm=45R0YVz`Kn#u4Cx!}?Nfn3YM!8)q}mV}t;D7b>tO1qIU;{Wu zlbo58a}D}VK{JmnQ&&`snBHpGsz13~eEPtXW;5~R?==!;4tJy$Da3-Xa!j0fmejyR z9Jk(uh74P1gqv<%h|RGMqSMgQ!mfUl5t`(oTw##(wc+yhYZg|hJyGa)7P1Oe#4W!- z1_v>q;FiIv^&K8E^u)k|tXpLx;Q1uZ4jsuWOPX@`FyBBJ^qjCd<>m^2wD(>>XimDV z)%nJjpTO4NB@E_>iW2mYv2JcIH$~mJAT5sZ{{2-oPw&QH9=WSoygU}U?vPuQR;ttE5_CZarz1>9P;&t$|l$67Hbyr$yr2OsrMtGDzsCS2TcL%Mh z>4(Gr*URDeNrchq9Qd-gP*O%`tJ(k-LdF&y67Ej$-a>l4C&3Zqj1nRydDt^kG7$)R zkEeeLC(Y4}3QcPU9=2OkKzx8pp^HHhO~dj5|M!1`zF8|uo}PfAhxal#uVz>uLx)z1 zN=HbJf??r1b&l{k=pFm-%cFG9r6gFRp&Q{m5S|#Q(5|o@k55S<9`01#>^%9af`adm zXe5J5HhvLyLs5Nk@3j6hV`Ab-n-?!8YTVQNw<`4R1ku&P?Eq+q7P*yRVCvQ=uUu?k zuO%EYci7l|c6LHagjo9Y)^!(m_rHhN>F^;jDi|5gw0%#Lvqp)%1(-Z`Jdk}z>ghpD zjXXp=FoXp_*r?}h80bFi3zw!i3RthK$d4pamt60sU9n3>Desi>r)2~wW8z59tJh3F{#kEx5SQ{mpf-Z zM2US^=3t%Sw?gv}$e_B7n2riPFA z`T0_hzcg~TN))u0kaG&x8zz4;x{w3OfS|c~lls=JiqSlfqD@vC3@9s5^R%)FJMFWz zdw?Dx5{ISr*Y_-%SJKiQ?{34#`8_XBgPefTH~@6Ywzd|LKa7kh-pS-qLUXuOCul)e z2C^d}B6W$>6|nBHB|oHxQmW=TA8t?GF_+v?zxA z(=g+oV?$bIpZot^hqyqCKYYEvfByz#N}TtIKk_vjO~pRUlN|84B`!<{L7>)myqZM?(v zsnnDdr|Nn?6F)=pwiDV)ad0&7^~U8G<;rVl^`8sig{mPM0|Myj!cjEo?`1L_wB4P5 z38#NZ?PI$}=qj&4>0Z>>x5i&bv+jH{KepbGD1E&VNrz(ECTNdFz1R{g+v>csVzuI3 zE}F8$UKr|_b(v(FQE{P6x?y$2)_;4dqj*4pdBG&)1>G$kDJ6;Zd+BxIQ+>H1CTNix z&js%ZhI7(`Fz=>5ALODVn_-*0IG?@%Ah9MZYY8k|@dLAb->HdS1Cv;5$emf5lheKE zf{?1V``POW0|f-KQu^!xh9!-RU}%Ro@keg%$Mp2+df%FHt2mivgiA|GYO1Ky!dqZZ zK=Jpcx4WgKnWU_!$kEcWIxp|%V&PN53<*BvE3c(M5)R6)+S(NL)g6x%70^`t%>{(^ z!2``FPYUfOtH&pI1*l-#c@yx)1{ctYJdXSXQL4WwC2{)eY~$@#fC-Dr{H zR(&>lu+xdoAHR8KsH)HL(^_dpNy6^+sfMC-yh^KT_0Fo4V)huq*W$Hq;OCj zny(a)Xi6goQ**rkmT4<1Y(c2O39Sdh%xU{q)9~RIYtR$D>Vxx>P4|bX$@y0}e;dOa zz|n-tK;CaeqfG-)O+m&{s!05^R9jZdEW+@J;l~+Q83_UqPb5Ogq<*@WXn#5I5gx}} zYaFNNh|pY6=6Xr!FHMz@~i_CL!IwnA%Tw{ zPq49np05e;%~F#A*tN8L&X}yHrIwZ)Z>cVU{Fvnl2r%?~00Ouhe(z&1i?IaairB^W zwz2mkNO0QRX{|ZRPKD8dK- z%jeIMt$HeqeHVHN6E%-{U3EqgrMwM-OV=I_^C$V#}wh)S0zj+mmyw z3)nFw56)kMZf`g70U`ws4h{%zPZ@^nmsC@=Z&n~D z+`qnm2@i+zx>JzntvvI0h7K;lrvOF`q*DK3nzhLG>RQ3#Vu9!KRQ4UVruXl(MXl-v#jC}ddB+?NgCHpau1v;~8 z82z}_vdDAc_(aZma*TKkhF>bkiJrxVyJAYaeN=OS}9xMRXt$IFn^nKv<7gUN!8Nh-Am-H8l}HL&HDU<`FBv$LKv=2$0d#C&}$lNoy_PUhh>10D+B za0>sa0vNX;rVS*cL_|ceh``zcxP&#@QeQu1((XC2Oc0MA$5;_=On~O(L%-7zMeaI$ zBnCqTksN-Z$v`_DEG8I2VaNXjPq##!^6Kg!%--D6d{@d<;Nkm&gWLT`wBFZS2sDaP zHYrDgB#)XsX%EL}_{CYLO9HDP%hX13DAC(QMfEm(v?MVjB4jz7W63?<*Un$R>D(h) z(fC%tZYv)#AugFoByw=2R)N*X_w0FU-4B|at!e$UZ{snak0i6J*FQ<}?@mOBQ`6k? zU#LI)!8yTDjQ+`FjVpyuIgp5+J8r$X?sGo=gXeaDI2{1Fi<|4TE^JgDy0@kofLNRJ2w9J?U z^X7BVPHuU3v;3eW zy|pz|P9O^H48Hf;{XOylDSSvjhA{{vuqM#miY2QS6O0UxC&s-;Ynmi%{*!J-;I%aJ zSn9|XQ;bfQ|D3G3t*X+!OuKp9+atbwL(Fc5nqiHY2GQ6*^`$b^aYtQ| zsB=5FKzvyI*V7VD?Zs+a+_1}+l<`&wb3L!$59P#PL59MJ9 zK!-lM9_kJ9#+n&ydkHAi{QUB}5C{m!YW)7xb#HG9-rmgY>~MUm5bE$3&U^J=9z`u; zFy11!g96l_3M5$-r=>YV&>Os83XdPp#Fd3f02=}(P7QyF-#^x6|6Vl>#Y5U$InYZ$KY{s%02e2>_ii|8`$BOz9q1U4X2bz#vz;AB5WYen z78tnC&U~S%)9WFrW3}nTT~-9F!bn}a*01npZoH;4gyX!8nHl0{Pkv}U+vpm@q+dWb zomT3?Nx5+5+ zuk1CSd3Q)>llfRCGQf%VDwlvst}-RJ(qr+y}6ztoL|jl+>`Ap(%9 zw{doMo(}DvB9)k=->0@36pcLJY)qdym{)I>`8E(8M_$_WGqN=O{B(yd_0yG(UyJ*- zc6S7~;H+B(@jwQ!-h^FZQ)H zTJ(+EEDZ8syoI0EwZJ#Nw|x|J_3B@{ikGmowD46KC@H|O1g_zT@NgjCc>*jGktXdU zO|LG2n##h00MQ&Aicc0PC)WY1yr!BVD9|&llDT?fL07E`Z8<_h)gXd|+A(;@HpHBl zua^3C9FWW1^UYv^8!sB89WUOY}Uj~lke8mj&CwsqUTRNg{fdWduYk`sW z+(F(XP$h{ioVl$HPzB^HfSce^Jv+k#&P zrGDg%;E=EDkr$92N{LrG8lv*L5A|w40Bxbru~wyWI88H zyY*A_JsdVEVjQ|p$UE@`@RxdaHK9b@6p28!Mc#?`kReu+Yt}&WyWaGk9}&x)Jf87Y zCCAuWB(E28#5D8G39Fyz*-naO_nf6Yx?mIi{G^#*S{9d7tCK^&T!+*z_k7|77D-^l zj=n*3ry!k$zy|++pjf*YG&+m3TZX)w*wt(1+kB^_7H5uTe z_1*vPqiypzDU83EM!sH)#d=sxnf19dOw|z*)-t|}nc6U+S0YZKJN9tBN`R&PH7W+5 zLEZ^$%xY>TUS5P|HkOuIAR2ISDXpsVba9bE0a>ip7}{jYg9dp7NXC>DsPP&_#-r5l z-xs~p@Cm}(#kU~&39P`1Dk{xrM*N(dE{3F( z+`7&dzG&vZk5vj>yCq_iSej=u2`&%o3SAs5b#sP$cTbow^3pZB_au*tio)nMZ{1u= z**OiPzl zs)*GD(-UwxU-C^7wZO`ge;Ynab`JtCr3xF~CzrleL zld}+=H;RLrA+H0br+<%epculPWMwlAq8X~cBbhRa!2<=;^N(jQxhSugMdDCiw%S?- zmE;e$ZYaQ-Gu20+kg`nj4}GisTwK<{lWP_U<59G1(0{)=b)a@nUEK+UXkW6}phX53 zc%sHIGFFgR=;%D@w5TMIe4^;n~b;DM@wL(0a5cMg!S*Z!#THFZh`+j4gmvR z*j3!(uR~oZ5eaj!qnX(}%uZi(ZqVzHX95E|QRY1UTT!ABNoHC$2G0Y;A*KUNO+{4! zH4qOUjz9~^rvBFUHj)Kbs&lpw48&rk>k^ z8eqgS!$u3g4jz6~7Nii9Flt3Gir$ef{S>zH+3Lc4xa#2bYu$u#yNYoXUd~64@8L$k zLR(j+B#@z{g1BT$AG4A2=mdAJ{*pQADL9{?O{qWJJn2?^nG&DpfEv9;Y2@JLQ+*OQ zoVK>HHeEI!X!~(9q9{8^9)Vwu<}2ULGWx!H+I{s`Z~5ccZ~9kYeN*S6ylf$XgjAZ7 zIS9W3xzu@n&E^eS+Vqr^pItqo?)e4SE!B-T7@r}SaW(0_a&k_0ajoAOA)ve=Okrpt z;{EI8r74q;o=+LU0OiE(NTC|<{uC=9q2UXIqivoStj_!p%(lP3IabEWKlBz=-Wiia zL!t@f4F)=+%Wa<-T!WT}&rd>8_e4Z|7CNI-Qc}Qj!hkXuzS=?;3_L2Bur;-{X~ld% z^9?*X09O(*OiJ6kb(Od5yRXW@rzC0oY2^kz6TlnShq?EAez}@mocm&BYw{#g1Ox;S zz@%d;b=s|MhL6vv6xz#>os|g;B+G*bcLW95NyDJc-QCSCO>lru07D>AoTp=}bxtWk zg`fX{S&LrLw6+6+#1uIp8s(izvKJp?9Q~y0Qe82@evhdR0QM%ummvly6_3BRN-5XM3WYywM@N-|s(OdQ)o9Cd$ z?I33~BIBZ5|J~M^ee-eRm0s)b+lP@?2r3!z+uHc=XZC$ZcYHBpu^z?ohWGv<2t;F{UP)|jZesW$)Q`cA8{(8iK#WOl=sl_HQET7Xf4jWg zzJsxa9HGy(fE_tr3ux3peHgAP?#Mw7p&>B)0+SR}a&NK4IXEd;0el7e4ieQ1;Y|-d zD2i!A7eFkcubzzHGSa+q6j*pMJ0mW+XpPXC}?uc(JxF}M$^hz{5TLq6y2 z2cBg>i}>~ptf#?|cSQ^yKlTI|2vEI#e~MpPyK2&m&|}F_EWR2<6Jvjg%qPGP5Pc6^ zJy+LoqUfloa^L3~jO1WZ5cJrhq~@oi6A%(Qf%oT?RkBj<$NHiN{~1a+_TkbJ!v9;3 z+>*|z{&7V&?x&r`)lNFki0_Tu4~LjKh7%6Xl+}$-Yctc9Y%0-j{bdv7nz6AFg1zgS zj5YJwS6CW48YOok%Jo=r7|omgrM0z*zP0tL$WP^WbiA~Ym*dr(y}ixU*jej zdUkTPVVZ9|P3x`uVMMT6kxlRGa48IxHdDc-5WPThtkkjU2%1o#+b^gbrG~*>0hMm! z`RO*8wBS>P7D)*7Dq>Ema?bb&wuD$%`{dY|hyXf-WAe#Ip8lmjk6@O;_zPUN` zL_H>&%;8%*jd{#4e3^7rjn754`tI%!lgim4sS0rtwsv+fykz~_gMsvc9UI#pl-B9# zvjB+f+;k_V`Q7CXB6|?gAu1Za1(!?_#D-B;QVfs}JD$KW-zEIlpl@89kv zhwV1&LE;$zW%q3odHC?G<Z`G&|GYqG_hxu}ORlXtw+17hRS1woYHVzduWcRVZkcVHM^|&TIjvKkZc*^x zPO!Ya=Rx?L@X%9tY)R`UeS(ZQ4R5igs`*V$x0qxylnFToDUe+`5S%td6?-gKdZn0G zka-iZ>-khl`>5fP#rh(N3MW4DPu@ zKa=BE@VTn=Gi#Ig-m$RLf$3B1d-}9FZH!_lVphz$4?)72H`c_Y%5T2+$58D>{Fym- zj_FyiZ)mMtefMB?$c>&?dZOII!)kCQ3K>q!2r`$p@Qz61=RnU^RerFQ*k`NaQVbj~ z%Xvx5TK9#cwM^1!qWE*^Yno7vOW9;5lmxionJHp~oEnqQI^gmLG7$th9et3-SY830 z@8=f7JYyHLXa9~+)*5_}1#DuR3V?~)uBtPcu~j&}U$&^g_X}A&h(DTMHhQe?H|#;Y z9;8EAN*jPJPOKA87rF~4?Nd5tYY`CnPoHOJ9suYoKLg!{_xp6zylQ!B-8atimqLHr33G2s8CDW<*k z1VZEZVdM@Cu>ESNAw&=G@aPKPiS)-n9&4nfr#}QMrjgNn$&82Dhuq6A7*V23iQz`! zp0tT#0cyh>`@olcbjRBhF!XT8!SqM^xrBs^yu1yh@y0NyN5?=T3XEi82;B7!&JaF1CU8O?4+OjKmk-@9k>?AgYT76~x%Rj@*ah8y%eSs!E6lAr47K^x@3 zub`g${aH%F?RJfq?9I`(c-Wt{eOgNTWI+jg5-Bn_?3?r9)3>Pj??bcWgX&M;YE|Sr zSE(zNPeg~8G5H&-VKyTQ;S>lC9M+g5OU#A%_{~I#o^YPhFx% zbjov#$2x+EwQ2U~%gd8~*R*y%-BVJKRq$GvSEgey3!}-PHz!`G;Lpn_zm|*iMDV-X z2*{V<*H@E{i9rmG4NevTN2-8J4uoiCJi0>|B+kJ_y6C?0(==>hfzaJ|7GV7uT}f*Y z+mx1;W|sUIvKGOi2^Oq2@`SkJRwTLK6z>Ep3S4qWuW+iSKEueak47D4Q^&Zt5WLAR zPaU{QA497;wjU5M2p}hju)0{!v|`A&7=pa%!R>6YM;U#@L~!9e8PR$QGdZ{s?)sJ6 zb9SsYH&dBx!0>AuAe)O&jh_FFJpVhC&`8^}+gG`@d9iC77G=t2_3nROm<2_HrKPqi zEA}Fo8bJVSW#sPD1HuCK*y?K0@caGiNowERZX=Rn zrr~Qjve-uJqcACRfGOc5$fYGDTHF5qt@n5c3+crjjt&?mV^>hhB}`|$M`2|KTGMR=JbLfr{sWd;wRh0DgJrTvbwNjf0LzBa?@llOVtr6b?pJ5b{8DSkIPu*1Jy*u*8 z(aT=8?^rnWt@ITpEEcgZE1p)c%lBOyE;e#aB$& zk|9{w;!%JjapbS}eI7S#zc_M-E`Wf503rvY&AfWz?`uj85vY^MP-Vh)6VxPX-SDNB zoomF26(R8MLemIPJxCv5L8I}gQBvH6G;W84gam&8J(d<1<@;xK5F)W*n|}j1xcc8fDiEMb5OP|2>A^W`EyuK=C<+Vd@N{(i0nUSPwE8gQep_2tLUJ&Mf9`v# zKu|u3?B^>_Y#spcH(hvY!?^}>KLU+S7Nes{3JXR-L^A`T30f4wEpZ7cEJ|$7lkFLe z0}(qgWWU4c`_C%culX)MF67lf9n93jDJF_Rgh{}G7Jd$Y)HDY;!(^Uppus(ST6SQn zZ(&g-rONA2qDrR$b*&__mT$G=yrCg--g5SPZJ@J@nn3sg3RLzn~kuvm0E*+r>dZ5RT}? zrq!wQHddir^>2U|1H1J*EvC`0VKWDly`6)IWk=DePe#gCEyvmxBoahHqKA&_w{i^# zU(xJry*SzS{<8R*AqNS1jX}-I6_adE&L!$UXg2e-Dnl7gUZyxYf!aG#JG{-8;~;xt zY{|L3dIs2_Af33*q3PDgtxz}eMAyUG*to)0|0p^0bMy0*@Q}=JaCHj`8hkVh(}*I# z$Hdj(fralS+)JChx)A9+Y_%dlbtd}rvHr~X_|Jn)T`>P)Ak;-x$?PO>QA0-qZRKZX zHSLl66(|G(p1@736R$oz1OpFTz%CQC!v01Q{D5#vd3y^+7SUsS?5Bdtot1?}S>|<> zZX83p=K?s^`$3TbbZ5U3-Kq$AoN3!zbH84b#0g!|lUp!9qzUcu?_8Fmi2`G0ON*Fn zG)Kn;_H#mD8mFGDDVSJXDR&W$ok)Q5!VvQN*ccz*3YVgZwRHiM1m|afoLYanJ>fC{ zVz}FcDLqTk*s5HvVnpZ`#PH+fG@*sF+R)3Z!rTJrnlg^(ysA%omA9EloHo_+w^ zRZf2pLIlAfUcUn_5xoGt$ngHYW|&m;2S7VR9S@oJ0dI%g)}g)k!oL!Byl#(6)H>cw z@|vc6D)rz{`0wCWW6o$l*U8g|s+u|n-(DUn^j)c};^j$un3VojOI33*L|*x~srGcG zAtTjgd@hQYtGrGc&3tOHYGN2qEvHMBZ#z6YRanp8NKL)uArgHv=8=L82j#}*w&3va zxX^UGdg3RS^_e3p-Pc>RL1nCegAAEw@2E;{H#c04oDYEu1d_U7?PF=lC@lOOj6xdg zg@`DhqG~LP_{rqduUj|e#^9%3(!2R7g@7Cj-st^Q0LxA_+64=B*s1)`dk+|otgM%% z_FIYM9gw6Lv=4EZaMaIu%*3U{h$WD$-i!JQwFjMS@4&!-j69kG@$Y(HTH3TdEg+>z zLK=gA8DC@OtyBQ!_mMOpU+2NEk`i3xg(%-dbF(4nY@vU3b|%543Va6U_2)b7H$dbZoE50WwCY_OB6S6Yd@2D73lOKQ!H5o_E##5OG=F=fB=RB7aU+*n{i~niaI_;} zbwK{M5K#xBuRm^hqkl>0xc~S%&DEKcv)*1?SrP*Dw8(^S=2oId|B6cVo$*ssP47xY zcSDKq-`^{KqCyaeMq^N^?CslG68U2wm{EsKT#n?LWi+v$j>S}^z$cZ%o*TQ zf(Lp=NeG7u#XX#Q!1n1AH|$(J7iJzL#h@Q>KAfGIkwpQpHRl7pb3BP=9|&u%5MGHZ z(NBp=YPr`a`9hqy7}j6WZBT)O8mFuP%bzvO#bq`iz!!nm52zqCVxriHrOi61cXVKn z)$wM8oT}*83o6J&g*yrUbue`~YA?B6oc=(r8lIWigm&rTnb@>>N-Bl^_1NpeF?buN z@p6cuhP@&r{3g3XTnHejR2XSxjpgKa0Z0%(%@kG@$+}5PEFLQkm`n(FW)C)2`L*AV zm+1-lmT;lKmGuwCJB31S1@)Q6JpUsvD8s?hU-c}w}{PgM=p4sPGNH>s!PVWGYrm9u<2 z-CIzyewX$-QawHMz^}ybBm3ECD;=-c__sbH%+cX7pX1)B9YYIq zrEb{#EjZD7pI%lU=3@92K@dq;qZctW6#CgK#iU4VsF7wGv5&K_pn^2yh<*E3MN<0qgW)5=a@gnl9Bd=|k8H3-DYgiovMw%Ee z-!lR*2Mz!zVz98Vz&z}5(RP3a%QtCo{Og5UX^ZIh&KsfyC)D5*@hU2Xn43Yq;OPYy z^SKk`x2H5;4e*yoe2sf#%#l*JO4<}AU(0;4y$ z`V08%izX`QAWOx>_@g#?R*)45W;AV5ec%IWghy(A1z{fkdKRw?L*dckeRa8}ZRCRhBdgGtu z$LWZZ&>ciPE!y*3I@65zOcBo6kreY!+9pmEwe~V37?M{s9ueFNbiEm6=lbk;W9=gY z6#6EMq<6el8+Pv8ApShsxcK}zm_Q|XZNBKI-04K`DVy#{PShgdQFq;HuWt)YzRf^iQgpkO-c2tTT4s7^lMnt0XqAe(d=Z-CACe zCP2r+?JUYcvbDPU`{)^z9{|3~WK7JV7<_Z&<>2#3S6qCkRUume8&IuF@^}g8M=?1~1s>sm=fVP*zl=(jFks!_+~vJ znHbgtMyJPZ9|B?51tr6)S6;v|1)&UN0s#?_FqqF)`(0cdQ)s3UIS4OQk>7H+m1TTE{IG>69rq!ILC&?&xDT)72N};We$( z<)o8|^CAN!jw$620A>u#UdA~#E^JFWc`q&8duZ&i>9=^u4_Cp~C*vQ8#WF;BM1yYr zX*@e_Ne}plOXx7&*jSzSn5qQot3`$buaz8ZvB}S;XCMAPOSs?8)Pa{Hd8LC~f+b|m zjKGIZSqp>^x6mRYBCyom@q}@*>o6JKAVCAjzv7p`i@Vb+i>SC^QM>;9ItVQP-f#j8 zESYG3YO+BHU0OT5G`*DwC5fC>3yP?Wb$q`hpI2^VUrY z3jn82RZ_E=%}FWG)8+8)t*-9(*DJk(*dJ2wBfWQIQ~#>jgD$|B<5K81ld;! z=@F(J7H* z8y~y8^fPfIW49`HSA~%q7Q{kAknrxLs%m4yg1{>AaJDWXIT@lRVX%va2yU>(?s$Tj z1mge9C~gIvb^Vl0R5u{UBb`vLTTR45j8S9@{fc&Vt~r@mMkcnqvtJg*2hPq`loF=h zD^B~Wdk+kV#^85`p^`*3ZX>WyJS;D$fhID38xmf??34GSV73Cf7B z`lEU0&_oO@(+NQi$NI&qa?CRLDrA#*Y@oGnoN;O0QpmUApK!{2>3lvpIy9rZ-K)rc zPJwVp3ly9k>*x#4c|lGh9osde&=n2(w-%db(c-uMyQN9DC&F;<8!2LE9v?? zD$JBg&M2$^GrZ7)2XLf3QB{pfNmCM76;z2)RtPUGEv?h*@9r)L4=3nZOvZryoX$Q{ zP-ru4U$taj`SC{evmFwBjFFv`QTNVJ%9sGF{Pu_=^xd(a`}6 zK8#PJ&5r&Z?SUi))Y5Q8`t57^X+VfhO>6x|SlS0v#%=-e-K&x^&nt-ENI#MeI`wVJ z{A%Fv>(%8O8}4;;+vhD;eamWUAcA0OV!{nlWCr*l!iCC8`f)sHz%a1N z?>IIxatd(i?d|Pz_VcBtR1v?kpCK{X*>p+?EWX%D?+rOXSU7)pa;<%IdPq%01+rn` z7yN?-1yWLU1qHL4Yf-eH-@kufR0Pzm*6#>;%L&(5!gFdIWyHl$W;ut}iHYwX-AoFF zy{+z?0VQRl&tV4&q}V@KzK>|}R(rf)Db!ukVoU_p-dgT|4{0OkKw?;uXEHV<$LMoU><^w14gPa0zNXm6pjQdL z;i#mV`QqZGq0Aw_-4U)=6Z-}pQ7_ajW4$(hZ5_~S5-%tyjN-h=r5<#84k8!WkAvYA zG1U5g))D|KKGdk^aE6~TP_sCA6-^x>g(j@;zDYor=s#UAwfQ$$tVN^c6&H4#X7VB8!jbP z6W4FUPG0Y|Ck=Ih$CmBCTLQti_)2a;-LfITQY1C(`nJWitfJxp2gl!~s>aypXiOwD z^Vvm26R1*NqkUoYc40=ti`%6@WzY~&UJoRaH0?;#=zb1Gmbdp!go7St=P*Zi*O*;; z0Ao-)yL~8Gp}2v=3F3twf!4s>d-7$`CS!q3Xy)5X4>t%$!~%3f1F*(z3GwwbHfsa~ zG?uP`Tz{-o|Kwr|e%0^%kk4V-b8~;JX?Lj) z3XPI=3u;+b+uf}3qN_Id}V!^Zt|$=HJ3f}o&y zZikG(w1U)|A#3(JnD0}a`iVQp5n)H!%z^UL2VK8vweLrvBHCS%x0{vZ!=e}p;g+@u zQ4K85t9)=D5@LH)xP&UF3oIV+hCm#SY&0;f<-)=+M8EKpg~0gS)?ph80I&ew zBqPEMH}YVAASh^E#S*@=us{-6Mnm0g1kuu_rim_FJD%V|ywAth5BI{r=;#3`sWesI zUP9to+8TTsdU{w^2Hy2_bt8yAA0_WA3|0tEeDmfV|6p8n^cR6hX~=VKI+(PHi;DwI z5fs&MRnCqzow<3krW;7+YQw35{{qcRfMvpJ{jgDwF}v2BF*F=)6b^hpfOj&578U$T zX#7LhQ^U*7aPcgTc6U!eRsQN#drM0I5VK&I3+)1@E1P;`K z62eDs>O8grzfkvk%DZrhul^b-%y5M4sl4+1t~FPV4)pcC@O#LDVRkNVb2G{@g8x!; z;9Bkz1DibSC)(*rx~Vs}_V)Ke0E)M}yHK(FOfhoyUCP$^{F}^w-8mhk6FS*C`8bKx zc=D-OCV^aEYFrhznlwyikF|zdf0*K4wQ~R-XA<78EshkzRv#TwPg)QqkGN=^q=3s;uav^g#puEJ&pg=q`S^smxLkG?+F#V`;hr#?S(`&ubB9_F&U{r>TqHwAQq}aG z1E4Nq0}dnY+ioD*Ku4{{9Z^k?)TS3yhN#p|Th$9IH1drHuko)e`|Ow;11l zJDu>HrKRcP;CQ9J?82a~hR>$0?I-X9qSDe#)~*8^!VrSSI^MwcdJ_a8zne*iIKnTW zegJQNmzW3vgnow}euv8`4iFuY@Vp_03~CgfI}{{ss2yjcaARAOa9m8#s|RSa$}wFy zb4p7ep^^wL5R}=~)gRu!hc6EU^CI^T>SfzP}3 zf-xEV^Q@ z!A~>aU>hzx2s8rHR6#*D9UUB3S08uxG)BA_Xz!koYLV;pC&E%f=lUxcl0?>W#koE* zu=ny^ndtX-sBQB5yPPH@%$vI=`+0@zCt1SJ(xQ?^zkR>znXS+-lj@44HE{$QPU>k$ zg;RfkNgxXGGp&R!Gj&l}=kMaG;ndac+~@2*ql;V}2 zsd**Yv}B=gA|j+QKeF*07CJopNZa(?%tOem%CwqRI9r9y3Fa5p8=P$@D$(%U zJ(py)A7fu?|8+meNRlQB$Q-@B3i9PJwR;?=m1ui@?Zf(UfB(VP*QMFn#qX*3C`VG= zM@-?G7Z&0&A(ytYl67(_4@Bc^IrDG*nVy-Nia`OqUTx&L7K`-6-!@4l`z zt(8^UB=hp~Cq_jzR#u)JOgZFNm`ZhAy>bP-Olbl>IT@9>hyr1Tilv#Emz7mbR+jL+ zdlFyXQ_!1CmvqnP@d^q~e*31PqVm3*gog}(8!osKst(5H%=p1kAc;cYxxPKks3-)l>M)r}=tZ-vozU8QvO^tQIv)jhEa zzKx)$`o00%Z(!xt?b~~v56sNW`tILHIJo)w(E@8_Q(z2pc6Nc2hw@DjI3)+tL`oUE zv2;T@MDOB}49n`AyYwQ+Ey0EN1^kNQK|Sf-6?C6JZnhoF4t-ZRU-&iswLwSDM5_HL zo6c)WsL({6FX(tXHtze#V{TK6IPcqDk_V36sD%6ScV0aA6Y0gE{_%j?dCh9?WxI#T zyQT7nQS%S~imjm`?Av=P zo~;eh$v1q(R#X}|I6S<#UNo+m;V#7^KoiUtd5pWvWj(BVX#4IzZ~-txGDM%rJTPV5XtfqNDP1SIrP@j)U@M;sXl|Ph8ZL z1VV3w(fX77x6b=t8s-c9+R-6BFk_G~Z1+)Ma#9e&-(amE@^3ZkV(6%5q16c?6o1>W z7whXY!HG1!>CP!F;K$E8j~6!o?oYtaTHVeb9;TGxn+xhNNRy&&lftVD7n63Uu zi1aKAB%@$S#L5jv?$ z>@Afzhk08q>LeQL9)jD9I7N^nCR7-m&1kA10U|^=C%<53sp6IUOKoWamzRf?U~%QG@p6Xsv$PvLz#j3maW!xG+i&ch1|(D8EFoo!9bQ1(GmWb zDiKAm$?hbuP$C&^f_NdA+j~4yMv-H|9{()lx17f}l`D@1RQ;X`&#mWw?S`>0%mrZj z3}RI1lZ#I2!UY&w9Di=j)B&p!4)xJ9cMU?RKV2bA($tW2RGu#zUrs|D(=yckYok}s z+FBZN<&X&Lplr&VoLm8wKrf(DXR9gTqB3@Ny3*3@hCl4@Ux5$?KBf*+sP-p67hWg| zfc5HoT?yOA=c0CjPj52$<_mx4Z*CGAW(40vxJ}8;VL+xpg!P`1$l|5MOX$QL8um5i zH7j05C+1du~E)hIAR#bsmgJ0p1 zGt&u`HYsg5c;f91=Xk%+_CGgk>nA=wKCl)uF_EygPA(|mO|d4&6_?Q%0Xs>3{d9jx zzl~PuY^@u(ZH(;Q4=%WfK0YUWjXQmSd+&Q+QS|oG6P!C_Vb6XVSs>)1!=FG__N81*Rp4u6K)$*}8#gWvWW885dV zZEzr>&HToRTt9k`uZfeG;Tlp7XR#7xF$Qw%1@k5FSS5RRr`7b|$)k^kUH=GlnY#rylb9$Bpoffb=Ub{qK;7F9G&(3ZZr)_Mck}?RI;-t{y z_<#O95)gnP{$Xf{grzslC7cMD?h_NUni@PjJjI!rUkci2WKA6%Ev>BZ5O8ob7_d{L zu#y@?{El{)V5|pTeU=wE)Sv3=PWIlNgN&lSwwAt8E{~DZgoBIk2L3Au-DV&)HDkl| zhq)H8w;F5uLuh^m2_+SVqyjujL2-&lxef$kTnA6Y%CLX7p3pJ%5> zK+Cq|UUn7qo4?@;??X1^I^TQ)PZS#)8>H8hQUgxFeYtlnGn1xjDlI04#ppdHkcxvQ zlPiWQhAouzo80XT4GqoB54z)EcJI>kH>_!V{Ci@0JPZU#KjR~gXKyvpigG9z??av{ zT;hFSzk+_Or>Ey~usBc)G&MAaKS7tQ-{fcJ#&a_am=i!m1rPL~R|J`pM0cyM$mSgy zT!emvSp0ntc6ROK$SH&L7{VpG1h%fIqmizY9us0a4`b_ZcMYZz^^$dNeeJnl$mWoj zZz<7gbL1*jxsKO<0AH;O%}(Md&j%&fS)OxA(B$5VISF-2H%sOZeK)JhiKl z#1JpqAN{Y(gPM()4PZ_S=bXJ5)r8Id^7JMIqH4r43M)Q!Ej8`ggT!+)`AMp*S=@Dn zjXyVg|19K8Zawy>oLL^_F~wrA;B>@8bXx>Rt3;@6CSKMXIOWRrl*GB>cHVg98G|AW z&2|7EKJq><@9U;BmVZ~Gq|DjXW5h~Od)eD)Nmsd?;VNXRiK-&m{ zG&SQP5CX<}(A{MXG9rwj(P?Qjv$GGmxJdC|mFU%ga+qqDOeBJcxs+;0wWV8suOh`| zI?4<}^}yeiJ3KKm0;*071QIXW+v&=+ZN0pXAglMz&6}w(w@glsHp_>@62t*HIlLl% zB3UTv>XOP5nEuq}EHA%F(JIx4W+Lfb(>W;4nvO@R8>{L!QzAvYu?GVvF{CFGWrknh>s>xqF6=~l{RROZpqtK$7)fi=3MSJcOLjW)LqOAJ_J**=}2QCVs_Mz=gsaQtvqafY>}U)3_rqXDxLP=5P-lJl55% ztEtgZSHFy4V_?0OHhK8&stE++>FerW3t0=z3~Y@8*9EA&`FMFZA04UplLP*8ZgSH1 zbbm~kEmy3@D!GC+*aR5NzuMY7AuPPdD9k!2?!?B-tfr}{sid;9zDRGFyT2yx;A2zM z`9%yDT$?r9v$6TynF8Hdv*qwCd1#K9Q{^|8w{v5dMZQSji2Mxg=_XcRykF#jP`mg67LWU3`D*em^Lt= zph!9UCYe(-*|S@Cc}V^v7k%j(KYX3d=>YG?T{CSBY;r=@NTS^oOsE1*+lSe*Wa2Azr8JIua*Ke8epwk48XzYfgr6arLBx%LK(X<(o*GQz_tmLbzyk#{kOiiiNs zXUC!w2Ga4hq#NO=s^H}KHa9Z^rIdt#b_?@*%9O|U0caB{E4i_dsMkDvFqd$v(fcn& z1vL47kdx-MlIF1m(?;kyTA_%Ddz_)h3eO=AvG@a78QGoLhN+LAK1ni}g+t7M!}eBo zZjKDO0xl$E)igOfS6s&51+YE2Ndmhcz>eX7h13Law<^DT2$j+mq}HA>bi-L-&T~;s zOsoYG(jaT@`i&bfUEiK{4>HS-Lw!A5wi}}UH`T#)H(Lobr)fchy@kJ8t#xhGea(i)zL@37+Q?=c}_TY*@ z_?I7lL9viwaZ)4(qDzz0HDXlN-Zp#^o)vMjvvzt=g!9N^TTyY}M9#K=;a)LSpKcyA zlhtf0nXP__09BQ15HSN|QGvNb)5YuI zy4fU$t=c=jr;@|goMtF>Ne<1o*zYC6(Jx)P12~ncXrNbQ@rl|$>5&K+A6?#(=Dum{ zZ|CkHC8g-)YvR&Q!Jw4z_U6>0dl56LClKL?*|~J~B>q0m=81z*n-uedfFB%y*X1Ap zwJ?90Qh0gEdSm(E(aGUcNNS%hvW*GoYUTj8q2v1KeGLt`KSSELn z3%O;+phdCyFX{7ve3O!J1%k7W_Esns3&Is}ldYh)RPTm<9o~Af5J}_Oz8u)Q^(;BV zEc40et&~1o1PIR$-<6h^GYJWiilDBTA$xma5ElBt?N@B)ViX=HqeZxMR(YXTDLkoQ zzS3Q3C+8wC@$}S=2A)eY$cJEZJ2bT03Z7U?%T>@LY8J;OJU;>uCQL7iKJ1I<(PCZ9 zWgtia-2q{nP3MQ_5KJ>YI5;shnj9Uy@c5cW98U*b$ZD+j@DT9LOD|?$hlhSJi1R() zTIxvv(i-U7E(Wfpd?UMyoD`r%1QpbiYbrzd;m+MkNE_S*8dJ{QlV`yAp12RF>dj@1eZ&&FPC6(JySWF7k4 zzMh^K+BGV1DVgLfapN=Nf8X3$*2$`TTgiu#;;a%(dMPk*^*U1p?3gmB6r&2AnQd6` zmG0XqJ$GH)J{0eXTJ=^m$!=XnGwJBol}dTJrq=4YD_))4DCk*xyiu9CU>rP?4&U>q z8oOQ=>RTP0VxfVwtc4_+{wDoA{URZyQ-|aWl>C?44%2Q?TtYf9h$LQf*mYqPAX0c{QzW8x3IckD!{@!QsJCJj6{GRcg~ z;Ui*1kRPDnNoegjIIUiVlK_+t@HoP)S%kY46(7rL|r^LeAgE>-#Y zXg)sn5;gdvI&b0tE%CG6Do>Lawf^ z0Q?ZiD=h`HDdhV=pDI~EOB2{#816{uyT(s>0O}M&4%fZqH-H?4{OH5E^Y^*ARgL8t z&y4N4j=)5HY<6++64_wY0R(cG=IHWN42&ZovvT-cwJh zzlK@f`Ux}gSWAQN`=|j=JKj2G`Nc)ji{iqzO6ZBME?Nvu z_3k(p!eHV1_cuW(b~!9;?{J$`mYI>!($bb}Io6~l_e0VGN_StYXw{{cV!^+7Mj%ou z%Y7vKg;u1s!N@#YcLAH-uHt?~{{d@pAIBZYZ1*}ToC)1PZAoML%x}&B)2ys+eQsRu z)~&l{y@uf!bXFy{Rl@C?N=|~)cOnyrbX09Q%Y7QGoOt3ECOtL{Q(c1YpvbS|zd8`1 zsr*<|ez!Vue0J8Jp9CNKU%>cmdSxMhbzg+WE#~}PxoEO5DGzsTZ4WPftC4Jdo+W)R ziqC}6ccyuU)wuF9h=ZzI_zn979C1TbdRmzbVR_(~jk8xym> zQ%#PgKW3(Lhk{~|TQNTF6Sp#~*AYiTPBCGuRH`G(ycrz;Ll)>+qTIGK%O6De2<8r8pE6LUh~oiq9D zh3Ue)(ChS1x6w$?Tg(&mudn#3S_m3HQ4PzVvn{a&gDgVbI+X4D^;B)Wq*(1PV!_4bJ*-mY6HT#9TkqnWQ}e49NuAj~c=PRo|zuC8b#3k@=^l_o1H;8Zp3 z_o_#0UZX;OP&%m_$GY@0aou*Wvrn!iHmRjfdYuAxlc5F*D<&)E=$^|g4OaOjMXt6M zyt=eRPww3LwddeaQnFEd`{=3gQBTc;hS$Wa=4)RxY-Oz2FzFNCVk6B&Sol{J7*8IW zE0b(eay;Ao@mhbB4J=<%%j`RmqiOi51hF?Ld9uO%%8>)IQE-_Yp{dDyigWoO#$Ru)= z+&o-eyDKD_K2DIWE1IhiUf7i!YxLq>G5q1!1hFecP5|E4>c-dpE<9|^3z^^4l)S90 z*W#O}iAhN{?x)`Tlz_5c-`D`!wlSNNt)X%vC06qp*}sgZ=;jmM1t%IxEXLQ&{w+*C zJz=m{xYqJ@w|FXnL3(TM?~fClBf9%`4554S!gm}R@8OhvA!n2{)59Xe`oLw*`95RS zKp`?HlUgxM`V&X9FVCS0$Dj9T5;6QJQz^~Yfp{$i1%i~k!5cDe*#Z5imS|d+_nc9T z7W0;_W+pDyp3b%lvs?WIQ=4q}3qZs&SHimW`jvj7py;JQ;_`Uc>Ns(727}nZ%Td3F z2gwN7iO~pNI^_LAQu6dPWrwGOgM+7MHE@k99;{JB}J%e%uxbric?kPDT zXjoyxULjrBr+?GkCMX0Q1%bOk&rNDepa9ZQ($kbKg;f?Th=kH2U}*H6j1S@Hz-s~7 z1r&3zuvXslE=oWxmDSd}o*Zm~RIYV>|A~d1B0d6HxI#igbaZsr!|t=OJ;@iNu+V13 z#Pla(#%u{|3B8)0mbR1{d~ZG`xAwNu3jA-tC4D@)Y;WjbYWwH(sz|lxyttNymWEeS z#(HSMD!qnTeo0PkUf$n@_Asqdg9|AH{Al2EcX3gdkTB8Iyl&WB#x76LEnpEXSwvyU zNCF0Jatex!@TKLS2Gx#Ynm-43${PbPQd(8D_4FP;`L|WKUE*MyR{+Qpbv-?zh(I;H zl9y>IiM_Xt(iv6L@zX8e8r>Ab9wfLmKI>cj?754TwYj7B;9={#Q!6PADV-vwMFd4eKx5J28l7Y?Iq~O-y{4DDrc)@(eG*PQEn;aBZ)f)B-X0~b zk@B+tD%w`oSBq&3!==VQ{17z^#4vPnib7uxkp#hMN^)}1)3uxotI5-&qxq<)$>ho5 z+yNQ#lyJ1U@e7!L(c*bMC@Bc##7`w6BY30`(?7soJ;SCrT$Ie9IXkanxeYY}tZ4ym z$-pAPrBKK0u({DcFz`;$t$WE?)w;VP5JLeMCO)XqnaByCnAVk|H7QTr*He+>Q3NJ)MWz$@kzwM+ zrdRDxRIwX@5=C7L`0)xY;y1Ulhwe`!T&eQg90MWwBWY>eOP4@Lp%tG1A2?P(bN@kq z@ZD$HT3Q}k|Bilz%0+fwL1JGP6ewPC1GOqVm**FAil=9*@Ga5NwdX3o>+0$0>+i?# zfAhvoO4ZqA5C8IBacDhEZQ)T4GNKBX>*;1@DRe(QgXX%gPce9SaCYzx3Y(sRUcf>r zw=SphIyIyrOY+`p3HXuf@)ENpkiY(bY=$7$=fm{YbWqOm@ltnGS4MhcdZUj-6{k<~ zx=?pdL6VrW+U1tEuOCt|Iq@)Qs6(g?Z!R_yOP{#t1{QHoxcsWMLAU_`L08^fNzRdVuM2dF9ZXNJ z8`Nfn>;J>2$9z`;IUq=DXxOEv0DEp|D6^Yn$uJogdG=>8ICco~1cV_}hq$R&(vhAy@P@N~U$`%xy{h4`}`oOCKB(7jc%=0rec6PRO{I-1v>BfN)QE(%q zRBb)hd<;N_3z9OBQEn0nq(%JsPF6Pqqe`fWho^i%qq5y}Hnt6XuF|nEBIx&3$>{Fg zTbw&T@dKpFeOXh`UG=Bc)n?>=@SO8qzaOlqpZK)iYZEv3Q*CBNVFQ5Q9E@GXH9t(( zPe&ZMgE5yzRl;W%PkS`F^UG@|aduMbj?Q4uQ_2?{R9_&-C?zL?ShXdDfNn<9h`qmr zjlZhqV`UkqknqY$&%1v~J=yVbvDa7Pmp-gmc<*hFze#%3p>&Oj+~}>s3x0a+dt?N- z#NvrW&9CLjWs$aK$w;e|fUh0P2fOx`7EUf!K4VR_^(7>_V)jl}Vor)0iaM%1jyQ=| zs$US`1O(#Y{+Rx*`X%=0NJpws=5~hGFy_(MZ!d{Ro=7=T!8M;_Wl%b{=`NI}bP+XY zgbZ5#X}$VTMp;?N<)Emu!J!THC~J|FG{|V&t}^HtS1#Yo`Hdva;r4(KWxpf;(D(0* zvLsL|Runz^JAVmQ#7|EqFW%h4SDjm1KYvPsb0p4{pG_uSYwTiIH#s>}pPpwvt}$U+ z1{5RMBWtsQ#cyJFC_2ylYlZ2XFHl%RVW6Xvq^U21a8M{HDIxxGY8M(s4&!o=(L%T} zDg)*}mfSZ{*o@ghMk;br*Fz;Q=9i76eOh{o>9{9GnuWuwS4#eTGNrLlij4c(wd_ZvUTK=;J^WV>kif>?X zvUB1AhZPs+5V*;hWNG>Ml;h1qHShS<-s9);@}zBL zOA9rkLRM2;+&AwE>9^LQc&~nlYGd3d*AaM$fnd-y^J9i2jx%{p3#(m31IRz)*BWL{;O0yXnuIYc(5d?h}fT&Wcx!O+_6<8ZCjkeY)@? zRTocR^kVB~VPbL*j!8~VB5@Rba*pbci_~{A9Rc|4TkS8H2GV^{-Jfrr%HQ}Xxk5d_teoFlM#5ysuQ=u|g@9nh6`E!hu>3PTRh z<3ALDEjYCKpdtrdF!3u!={`lmi_r#5d6vX0ERGwm>#XH?TKRI`=syc)Z zLZVdy(%@kx)V6Z6bJg$r_;-+GHvJoD+8gkwJO!N#oK3K83=nPJK7`EQ{@&hyKiJRe zVfF)YTjmgJ12NLHFrrU;cRmV31Nfj>*wn5`@L`!%g#h{|18l9|obivnX zR?Ycl17brMX3d?2HT}H|;w@Pkl8}x}KKmJ0 zfwWwuw1XT^H0;ev)&b898y|}g7LYQi3Y#5-HnSb#T|rL4Zvhq$3ak96L?t3gW8j^9 z1JcoyLzDjFx49`~2+2v=dk+qhx#E*zwh%o0qWZPQ+uQe6+bb|c{f;Mo(%K*J^ODjpDHFYkkz_^jpbzm)$o71Krhpy8O} zR;-&bubN`ahO4MNv4dR0tjp9;#RbMfpdm(}6@~*wLnA4zL4=2A&AA5dG8bp(vtX!+*oXQ|!*!U#^XD?Sb>becEy&w#KED0)V)-LP=yLx5|y%uK<^4PM} z`Q4wN6Vfe*TPJ5bw$x;%_JZBNlp8XVm^U+-gsTNciJNL@m?as82;43qvxpx~->S4Q za+GbO4p3ae=OtcxjUe|Fv``WT3D|~TE)#m)Kw%Hy)y+IaqX)lzYZj=Ac4{{+_xWFI zUm6eP`?o!z3`v8Qu_Viott?HUEG7F+6xo+7ku_PP9~0TL7cwC#yFqrc$AmK3l_F%{ zvK#KNe)s?R-1m#;dG*YzOf%8Q>eBQ#^%p_o zs4JP-;LwIp@UsN#~M+4uwI4t-mm*H5K1mm{-f=4Z)T;@l8>zq zo1a~EGiln+!QT1)TeIPT zO5fSbm%MFk9@;AP&%sE$kGd*{CxA~|lzW+(sr9Y?9T=34j@C z0vG(Zpcc9$>(Fbyp+wM;yr6qJY+#5% z;e0s+8z1OhfaI-~UI3PW>~52+{P7z7=qA54dlYfLke7-{?Jf8a99FR?+9!~-*xqja z{8jzuI_i>xn4t3 zJtBJJ&pH$=fR(ZW<=A9ettm z)f#L6>B-a3uCHr-9Y{H!*OwssYj3Cmsda#=!NGOv)Iz}K7o`|;J6l_$M4tN5{huLc zLDUbbOOUpj9H}e;cm`weTN}FL(R1_jqOHJf&T$ZEOdt4`mR8o=Tho{^nQim!^WwMa zHpz4H_ne)}2OZJwt?D=iIYg?;KtokGDKU{b6D;Q7 zi|Jb~;W!I#kNeO-y0Vhz{Q2H1$SG80D`?EYdH_1#Aw;5TX!9RH)WH@38MMUs_>Tm- zgIFMX?{_p-V<*9ED?xT|To&;CpJ9JBK;l|8z`#J}Y zkuOKOG)_5Q7XmJ0W2lXN{~jgZ2$tFX;)ar69pFr&)}tW92nyMTiIl#aKT;{pj17`s zykJKEK28-zm1X-V!F3T>YTwwZ~z zLwZ09Vz8L)4*KmA7bQDIIF2$_sb zKY+~k!w04A^y1Fn!f;K&1)invUsTCR%COT>&E<@%C4GM|XmTNhh;2eHp6=gy;2y?f z6>HIA3|%uWEjKsgQ0cl<)RumJ#`?X=n6U>};J`Z19i-0^mHYrU4*+-I%mUmOLCrwT z5u{H5*zR0UeovgBK(rcQS$abOTmA!>4Kw$c$5StM{UdrsR1`|#c3ZJvNUHCNea6l& zd`kmzOjML+M}t@?2pxbNOHK|z=%0aBleQBeKA(zCu5Lk*r-VSrV~;|is8$_vNpUyN zX{A0G0$}_hL}pQS?K~@s6I{^!`xB`*RomLy_J;#@oMjJ$ih-vAhHcu6oExzAa?x)J zDs+7(Q3||_e?uBq1D0!vcMHYKAh(;50p~=?uC1ayL&B&kk{6-va)`~iub`eImLhR zq{8&V!n&6fpo)ShDU-ABPW>t^_XMaLa3CnIIhmSTSp|u=+}EE`MAAeH|8s5WaHx?U z0%0Yj(ssc%`o}9w->@j1m9=#E5|)8@Y@Zy98Ljpd78Ha8cPn(aOw8kuz&i(u-hyULXhjh=9sLAl3`?YO zrCW_8nOBM^N9@zhhNr{?LaPaVuBL|Wt~*_w@s!aYmSUY8}? z)~_9Pu9bu#L_|a+B`eFS#~%(IJ$khMPnPgI0v6u9EE7e;FDwYQ(R;BLe5ybOr+Ybt zxw)K-zjcnGq(t5U`gCZd}+8_v=WVIL1(sR-hNbQkn8|wSf91zFS;E z!cNljMYg1rR34w|y*o2NziIjjJ)3{J2nh>oX=wrYtjf4ROZOHRR!q#Xd>E8|)Z^~F zPjb-%YrDGRSzVcjkg{G>K5K?npCOv-yOU~A?V7Vex%0Qw#Y>(QwPSm5(K!or1RW%{?` zBO}Xzq(-$t0lNjwtwAb!ed6?KJmeNjN}#0X3Qhp{_W%^H9!`O7Q2WIF-TB_N&i!5d z9)4EbNn&n&akpzR{>1716xhx%cK~zQ|8acpzJ|v!-shq>^=UY&0vecNGyFAq!rJ0; z*^A=8GpHu=LKHLq#r84ye5Vw-m9JBU~n=I z(sRtr%<;M@o!PTj5I8wbH9X=mMkOo*I{X1Km4t<$5*X}OpJvytu+gi&v-vzWW-0g> zBQ5=Hd|5s-1|%kdfgTn{s}jpbA3~arziSWZ4L`rUS$W&Aa|)zO)fp`DPK%BKn+?MaAYmN49pD8 zfV~--87c@kjYv*mg~;HyTV;V+y$s};i;A>OOvv9B{Wk#32OB?nlzaKd0cA1A;z_1R zD_<7w+5G%{;^v-mW3`qqB31*9G4f3-V8GM$u*rCi?R};PqX&XUQ>!3Rtf8S*SeWnH z#T}#k&_)s9_rO@Xe}DeZ+z8|wz>Nf*AB;-6_Q#(tmJymB_{u-6tdR7FMtnLa3}Ymw zn8wHJH>&q@?O7np5U@nOKM5TyrbqWWN3~m7W@h)t19qeFSZ2~WylTeD7Di6$g59MT z)#sK!@NbT>mGAgsaLwuLqrjPy1c$HKfX>P>pXq4qKi2;2+%Zpc-Xb%P%WVDtQT6fz zj&JSGLbYij^eH7@fVvrYW6V}yB;$M5);uCy`A$>??C&is6m;6aE`hQ(nr^15!2S-F zC9=gbo_khFy(oPa$=OwD-=mC)0Gb9ZFeNFBP> z=#RpwN(Pw83S_wCk;_wD#`OT2O46yR;1780g^p!Re5)_T4jq3!c6P#kFf}l!065dH zUwRy@+a)EB;vQBA(rr#;3L$BuIcv{`mN>Sv4;%rqtZ-3OkyxJ_e4Z-nSqVM zQ6v>Uyy@pF-WD{#j<(2q5ReIkL0uEV7tq`kB8s(2ppehArypG5Kgh)3P$ZGnqbtzf zw$d0f3qbmgR*^&pzni@x2g`^g5~0n<7O4nDR*f6|9!(;Y*Xu3NXqZ(v1HNLR(?>Ql zlIjQ3X&=YXQ)-pw>Umnub!*vWlv8ZQj~G6p0pt4^-&97$UUNgvgsWD2r@BQss z2p4u4I)5j|>ni3WSZBYQuAfmhg{<`5(9_~4Yt&^r_m(h3%BfVbAe3W?UxJ#07=lWTO-yNP(amIQx)R7+Yza+x_%m=`#ee zFzqk%a|+m&w?uexKfq-NgPrf3$DD0ypa;Ef*A0dl@2#KtZA?YcP<*aHSIE}N!JmKS z5e>a6=(;SDzc2Ro^(k`2$dXC2WFoYDQ^sWQsn(28WPP1*o31JVUZw>`g|N*&1wsgI z$Fm0JVllZnGY_u7%0b2;8&cgt9=x59^OhiF6}WMX7YQY3X#dV~WL)av#WPG42=J&- zzUC1_--p6lEYoYBWlxV=$;9;-2xZ$R6Bj-olV%B>sp>+@b1{?;n*cE|Id(m_etW*F zD~hn<)>{BP_446b{ACUssUGtNCWZpDLY8H!^}cAR7imT4z~3Rt8MFOwE=_T!6ZT)| z+K7ZP#9FIxfew7!Ngg)Z>8iTg%F3c5FJO1Pd&l1?0v6@2VOM)QJTv?I&cv=Rnj=BL z4uWpWSF5XI%%8f$Ew_yieII-;I0LyX&?_?A+1uC$2gMf?x;s<|3J~>sYg+YA_|mgy zzJ7w8LPZ0ILdtG|VaNkNhbX%z*#?6zuo~elmcXsPg(~4-R-T*^T_|&UG0qjE%y)u? znK@K3v%&l`ln4lHbW|@dyy@+vYl5T^t#PP4SFwQ*ny%GSl-N%k%19AniNwiSv_h-t zY*2ldLswq8;v*blg@^>ll0j>g_gD3jh%aKt z$_5?ZrNw_O{hB(*R@iT(r#I$S32omH9}TYe1a(*=(N+MiO;i+2VOB&l?H(@to*N15 zr+Wsb&zISq%ZH&>>q5T5hziWFfhxt_U@nyp&xt<~A>7iJ@&Q4RL1=W_m|0vL1pvvN z=flIR(NR%g3IIv4J}+GRt=Heu)pc)s&0*F3VvI6YRb-MwWd-mG;CDe5Q0d)^goNIK z0SqzBo&`UeE&r2KNC39Hiov+%u2QZpu79>HY&kStX{i!3$=`wo5yvEALR{RcLKJh< zz2)u(a}**ka3enXRuSz~syj(ULvW7jv1pg*P&vN2`hw!E7bkEiUPewdlZqS?9Ot(= z0#?NOn3kkaow17ce^QQjF}Twy$5@+>+-Z)-@Gt*4tR)Zd7Kla9GwpIyC_dZ8(C>zE z9g?73Iod<&fdfu6RYDZPDlGQ*aU9$OgjA1Y%oG$nY?YL}73?)SWrx2T*FT)!2B|sX zL>}x~s8TLR2RJ65Kmgwsg1R)-EF^5hDTaWXEp187d1e$1q*dc=KGz0li*)zo&(taF zy9flOIUR}u_gFV~R4gtT%^5+%aZe*|J^Dk%gk1EsKq`hcO!+?{Qii4-hV_le{=(bO zlosqI*rHl2ai|Dc?`GU=Kj%tbff?SlZuyx^lL)c_r`%xf0>_J!zXi~qY#!R&_vlr< zY*{i0nkt$vrPBFmWZS;NF*O+_mKM>9KgaH38cM-Y4nIERbtp_=hLKGbw9{b~ve0W`WWKW|d$ zcK3mG1;jZJeZH%FS5vdAB)c7xAn@nwy9FR(ldxEg65TIH;Q^fHEYM~2Al|&Sz=)(j z9?0I6L-SZvM4fLULPwQ(YQ+Bbu`UKVBnlC>Z#!RCFOS^GVrS-1<3cijAFb zP`y-%+?$9KG`%&^21gm6YKG3li{T|MmCNB~Vg}D7d5Vqz2wbL@3jC69q+YEgYkzr4{)%Rds-$20tHlNI4?qC za}40g0u`Jr8;G1T3NkZyG!b%d*{D&91fkg~P5zV9^`;h%2X6XB$OOKA4JLQfuMTQm zcxP*?`!98q54x=M>KhxgjgEFuY!QskqmWTj(XV$2sz&BqHT6_&PrK*m_opnh%4K-n%x?K_gn1=AmbI~irk&*W7^ z!9{fQ3%>?;dL5@Q(RT2ea5$iZl0+Et4lX0Go2R%Y^KO=wI=2dvL9+!;hd z->Tpw6*y3AM_>gA3uT7oz>pf(1%Z6TVigGD>MjfR&{qiOmaxJIsXq0 zXyhQ%gy9=kMH&d;&PIUDNsy=_CgY>3rHKX12MRf6AaEFwSKKrt2Q|Y;cR;su>4m{W z(c!cuqs#+gJz!KIQp9%3=dXC3|N9*_+cANO7|}poRiuu-zRn3qBT+P1aM)zEozRC@ zuZKzhntZ;c{vx9rTrrXgfr?HG7`b84?@_wORK8w(`A)15$;1LUM1Tg^`@OdsafHC=qG}+Hqoa_5Q)AjQF}a z8`^`Z7DLI2P{i^kSmLSOcb5ps{JnfT>>~@i-cR3RRAk7b))RmdC+U zLb2UoFV1Q%56IB{_pZ~qg#_Bp;GTVR0-2dFb|(<9P>>9v=26Udd7mT92ldBpNr_4T z@UlVhMn=dHIlc=z?1eLS^EHLKMWCQV(U1-q2Si?nKytQ1rVjA~MITC`h)UNu*MM?> z=#=;4376l)!-2P-x1Dt%np@s4*Q0G67_v*wwm{Q|G!aU}Qw8BZ4s*a5D8;Zm1n64< z63C{+uy`%>PKz#o-|qyv>+1&#-|GUVBf-m9b!40fZ##^3$P1~&YV*UlTMoX>aXlB; zWI?3<6c$@r8WwT4tKy`!^Re4zo#6ou?h%mN)z>FVZ{9735QgnEtdRu+ znmUQ8ojWZw$=Ofj(hYjt0#`bts!s1pu%WQJ*wjj zh0=MmRoV7^a?m{_ETg%th+Jw;m6C1GL&bFGV1dTI@XBwM=v!}Q`x6a)eH3)`dwb4- zX3*S4SiEf@<3Sb8*e`(OG&#s8SfGO~q%PdHHf%+*W*_=HLDvbf&xwL*Ogsx?YhDzV zUMTUxoO?AK2kWKA3#ZK>bx1AOU`@aAXORUuURO{&ONhuABQmH}poO78$mOzWrTt#q z>)FD33W*Lfvc)&oJvgzWWto!yc(X!+6W4svvxWO4ldP{)TBh$9Jg|gAXC9=v4#*6) zbt*{>c5TE+CG=5ikRF(bYs9%

Jod;l@dg(V))$64GC_C(p#twsn)Yk@P9#U$oKs zIsVD@KB-f_0Zc*f^12fp#(r{cPYJ5j#2ML8JB*Y5tez$nz4;jFxyGvpm$Hy()2hPmBHN!VeR;-mbO?t1{0AAg3`{YE0 z77;qN0`rLTm2fJLm0ptJBT!@5wZTygvNf{zui0R4B|nN;r1~Ojc0aZ=W985MR_Q43 z6`vMzgOIBw!{|@@uG^oeO&7M<<1_sq(Wi{}jncEiZ)p3{r&wPdJ@?*r^iHD$bgS0T zAn|4OEuBgd#G!B~#eo%C$VGSXWkzG_)OUq&9Sl5Q6fkwmgFcfMBct8VGir+#8EaVl zmf8K4=T#ykw?@ckWy6&*1T)^FR@L9w%b%TF5XNIKuz(0qnNei}UwiIoB4hDJuGJEyb#}8XxiJyd(GagM&M|*3RXSH!-)9Uo8Ct5~k zH<=$5eD@mRU2oymO55@-rrngf;TYku$c}$3V?@MzrFIud@U&w$mU)8SNG&(n?Ym1( zUK9%OYu%8EJ#&=3I>jut+4EzGLYfomEUPvXHDEckIVYFra5t~)kxSq%;P?euu4Ie7PJg-OWRZ!?(Pg8x-9@c?-+_Nld_$( zgs2vj+z2)l+oB)X3iWqCqghThAVER}}lR zKXsCOSz%}$8<0ZRy>iUUM z&ja@Tue$=Dq*DXG{`z3mtta>>%e;;GCul{~j+6gnf zCN;CzWL=#W(y3E#7V@aBeZ5ZFF*aZJ-0TA9sjR8`2X=^%lnk6Zhn=TxXwBRyi^o4 Date: Thu, 28 Mar 2024 14:29:25 +0800 Subject: [PATCH 75/79] [core] Rename to compactDvIndexFiles in FileStoreCommitImpl --- .../paimon/operation/FileStoreCommitImpl.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java index d3ce76b0a65d..5c6343956752 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java @@ -221,7 +221,7 @@ public void commit(ManifestCommittable committable, Map properti List compactTableFiles = new ArrayList<>(); List compactChangelog = new ArrayList<>(); List appendHashIndexFiles = new ArrayList<>(); - List appendDvIndexFiles = new ArrayList<>(); + List compactDvIndexFiles = new ArrayList<>(); collectChanges( committable.fileCommittables(), appendTableFiles, @@ -229,7 +229,7 @@ public void commit(ManifestCommittable committable, Map properti compactTableFiles, compactChangelog, appendHashIndexFiles, - appendDvIndexFiles); + compactDvIndexFiles); try { List appendSimpleEntries = SimpleFileEntry.from(appendTableFiles); if (!ignoreEmptyCommit @@ -272,7 +272,7 @@ public void commit(ManifestCommittable committable, Map properti if (!compactTableFiles.isEmpty() || !compactChangelog.isEmpty() - || !appendDvIndexFiles.isEmpty()) { + || !compactDvIndexFiles.isEmpty()) { // Optimization for common path. // Step 2: // Add appendChanges to the manifest entries read above and check for conflicts. @@ -294,7 +294,7 @@ public void commit(ManifestCommittable committable, Map properti tryCommit( compactTableFiles, compactChangelog, - appendDvIndexFiles, + compactDvIndexFiles, committable.identifier(), committable.watermark(), committable.logOffsets(), @@ -360,7 +360,7 @@ public void overwrite( List compactTableFiles = new ArrayList<>(); List compactChangelog = new ArrayList<>(); List appendHashIndexFiles = new ArrayList<>(); - List appendDvIndexFiles = new ArrayList<>(); + List compactDvIndexFiles = new ArrayList<>(); collectChanges( committable.fileCommittables(), appendTableFiles, @@ -368,7 +368,7 @@ public void overwrite( compactTableFiles, compactChangelog, appendHashIndexFiles, - appendDvIndexFiles); + compactDvIndexFiles); if (!appendChangelog.isEmpty() || !compactChangelog.isEmpty()) { StringBuilder warnMessage = @@ -437,12 +437,12 @@ public void overwrite( generatedSnapshot += 1; } - if (!compactTableFiles.isEmpty() || !appendDvIndexFiles.isEmpty()) { + if (!compactTableFiles.isEmpty() || !compactDvIndexFiles.isEmpty()) { attempts += tryCommit( compactTableFiles, Collections.emptyList(), - appendDvIndexFiles, + compactDvIndexFiles, committable.identifier(), committable.watermark(), committable.logOffsets(), @@ -565,7 +565,7 @@ private void collectChanges( List compactTableFiles, List compactChangelog, List appendHashIndexFiles, - List appendDvIndexFiles) { + List compactDvIndexFiles) { for (CommitMessage message : commitMessages) { CommitMessageImpl commitMessage = (CommitMessageImpl) message; commitMessage @@ -606,7 +606,7 @@ private void collectChanges( f)); break; case DELETION_VECTORS_INDEX: - appendDvIndexFiles.add( + compactDvIndexFiles.add( new IndexManifestEntry( FileKind.ADD, commitMessage.partition(), From ac27b66f995875e8ae715eff78719af6988db191 Mon Sep 17 00:00:00 2001 From: YeJunHao <41894543+leaves12138@users.noreply.github.com> Date: Thu, 28 Mar 2024 15:23:24 +0800 Subject: [PATCH 76/79] [test] Fix test in KeyValueFileStoreScanTest, make it more accurate (#3111) --- .../operation/KeyValueFileStoreScanTest.java | 58 ++++++++----------- 1 file changed, 25 insertions(+), 33 deletions(-) diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java index 8ea35082a5cb..c58136339807 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java @@ -23,8 +23,6 @@ import org.apache.paimon.TestFileStore; import org.apache.paimon.TestKeyValueGenerator; import org.apache.paimon.data.BinaryRow; -import org.apache.paimon.data.BinaryRowWriter; -import org.apache.paimon.data.BinaryString; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.manifest.ManifestEntry; @@ -148,19 +146,19 @@ public void testWithKeyFilter() throws Exception { } @Test - public void testWithValueFilter() throws Exception { + public void testWithValueFilterBucket() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); - // 0 <= hr <= 999 - List data = generateData(100, random.nextInt(1000)); + // 0 <= item <= 999 + List data = generateData(100, 0, (long) random.nextInt(1000)); writeData(data, 0); - // 1000 <= hr <= 1999 - data = generateData(100, random.nextInt(1000) + 1000); + // 1000 <= item <= 1999 + data = generateData(100, 0, (long) random.nextInt(1000) + 1000); writeData(data, 1); - // 2000 <= hr <= 2999 - data = generateData(100, random.nextInt(1000) + 2000); + // 2000 <= item <= 2999 + data = generateData(100, 0, (long) random.nextInt(1000) + 2000); writeData(data, 2); - // 3000 <= hr <= 3999 - data = generateData(100, random.nextInt(1000) + 3000); + // 3000 <= item <= 3999 + data = generateData(100, 0, (long) random.nextInt(1000) + 3000); Snapshot snapshot = writeData(data, 3); KeyValueFileStoreScan scan = store.newScan(); @@ -171,7 +169,7 @@ public void testWithValueFilter() throws Exception { scan.withSnapshot(snapshot.id()); scan.withValueFilter( new PredicateBuilder(TestKeyValueGenerator.DEFAULT_ROW_TYPE) - .between(1, 1000, 1999)); + .between(4, 1000L, 1999L)); List filesFiltered = scan.plan().files(); @@ -180,16 +178,16 @@ public void testWithValueFilter() throws Exception { } @Test - public void testWithValuePartitionFilter() throws Exception { + public void testWithValueFilterPartition() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); - List data = generateData(100, Math.abs(random.nextInt(1000))); - writeData(data, "0", 0); - data = generateData(100, Math.abs(random.nextInt(1000)) + 1000); - writeData(data, "1", 0); - data = generateData(100, Math.abs(random.nextInt(1000)) + 2000); - writeData(data, "2", 0); - generateData(100, Math.abs(random.nextInt(1000)) + 3000); - Snapshot snapshot = writeData(data, "3", 0); + List data = generateData(100, 0, (long) Math.abs(random.nextInt(1000))); + writeData(data, 0); + data = generateData(100, 1, (long) Math.abs(random.nextInt(1000)) + 1000); + writeData(data, 0); + data = generateData(100, 2, (long) Math.abs(random.nextInt(1000)) + 2000); + writeData(data, 0); + data = generateData(100, 3, (long) Math.abs(random.nextInt(1000)) + 3000); + Snapshot snapshot = writeData(data, 0); KeyValueFileStoreScan scan = store.newScan(); scan.withSnapshot(snapshot.id()); @@ -199,7 +197,7 @@ public void testWithValuePartitionFilter() throws Exception { scan.withSnapshot(snapshot.id()); scan.withValueFilter( new PredicateBuilder(TestKeyValueGenerator.DEFAULT_ROW_TYPE) - .between(1, 1000, 2000)); + .between(4, 1000L, 1999L)); List filesFiltered = scan.plan().files(); @@ -312,9 +310,13 @@ private List generateData(int numRecords) { } private List generateData(int numRecords, int hr) { + return generateData(numRecords, hr, null); + } + + private List generateData(int numRecords, int hr, Long itemId) { List data = new ArrayList<>(); for (int i = 0; i < numRecords; i++) { - data.add(gen.nextInsert("", hr, null, null, null)); + data.add(gen.nextInsert("", hr, itemId, null, null)); } return data; } @@ -329,16 +331,6 @@ private Snapshot writeData(List kvs, int bucket) throws Exception { return snapshots.get(snapshots.size() - 1); } - private Snapshot writeData(List kvs, String partition, int bucket) throws Exception { - BinaryRow binaryRow = new BinaryRow(2); - BinaryRowWriter binaryRowWriter = new BinaryRowWriter(binaryRow); - binaryRowWriter.writeString(0, BinaryString.fromString(partition)); - binaryRowWriter.writeInt(1, 0); - binaryRowWriter.complete(); - List snapshots = store.commitData(kvs, p -> binaryRow, b -> bucket); - return snapshots.get(snapshots.size() - 1); - } - private int getBucket(KeyValue kv) { return (kv.key().hashCode() % NUM_BUCKETS + NUM_BUCKETS) % NUM_BUCKETS; } From 93fda44edf336fea8a79dc0d5b131083e2292839 Mon Sep 17 00:00:00 2001 From: Jason Zhang Date: Fri, 29 Mar 2024 14:13:13 +0800 Subject: [PATCH 77/79] [format] Set orc.compression.zstd.level to 3 by default (#3114) --- docs/layouts/shortcodes/generated/orc_configuration.html | 2 +- paimon-format/src/main/java/org/apache/orc/OrcConf.java | 2 +- .../java/org/apache/paimon/format/orc/writer/OrcZstdTest.java | 4 +--- 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/docs/layouts/shortcodes/generated/orc_configuration.html b/docs/layouts/shortcodes/generated/orc_configuration.html index 92b1a9106754..ffdae855e2fa 100644 --- a/docs/layouts/shortcodes/generated/orc_configuration.html +++ b/docs/layouts/shortcodes/generated/orc_configuration.html @@ -40,7 +40,7 @@

- + diff --git a/paimon-format/src/main/java/org/apache/orc/OrcConf.java b/paimon-format/src/main/java/org/apache/orc/OrcConf.java index 6aa357868f17..a7fa1a21bc8c 100644 --- a/paimon-format/src/main/java/org/apache/orc/OrcConf.java +++ b/paimon-format/src/main/java/org/apache/orc/OrcConf.java @@ -114,7 +114,7 @@ public enum OrcConf { COMPRESSION_ZSTD_LEVEL( "orc.compression.zstd.level", "hive.exec.orc.compression.zstd.level", - 1, + 3, "Define the compression level to use with ZStandard codec " + "while writing data. The valid range is 1~22"), COMPRESSION_ZSTD_WINDOWLOG( diff --git a/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcZstdTest.java b/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcZstdTest.java index d5f8f8bc7cc7..0bf90b38a4a7 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcZstdTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcZstdTest.java @@ -145,12 +145,10 @@ public void testZstdDirectDecompress() { try (ZstdCodec zstdCodec = new ZstdCodec()) { // write bytes to heap buffer. assertTrue(zstdCodec.compress(in, out, null, zstdCodec.getDefaultOptions())); - int position = out.position(); out.flip(); // copy heap buffer to direct buffer. - directOut.put(out.array()); + directOut.put(out); directOut.flip(); - directOut.limit(position); zstdCodec.decompress(directOut, directResult); From 3d6c5fcba939757f53c38bf43ffd9cc7d699a65c Mon Sep 17 00:00:00 2001 From: Yann Byron Date: Fri, 29 Mar 2024 14:39:03 +0800 Subject: [PATCH 78/79] [core] add deletedFiles in NewFilesIncrement (#3117) --- .../org/apache/paimon/append/AppendOnlyWriter.java | 9 ++++++++- .../org/apache/paimon/io/NewFilesIncrement.java | 14 ++++++++++++-- .../apache/paimon/mergetree/MergeTreeWriter.java | 9 ++++++++- .../org/apache/paimon/migrate/FileMetaUtils.java | 3 ++- .../paimon/operation/FileStoreCommitImpl.java | 7 +++++++ .../paimon/table/sink/CommitMessageSerializer.java | 2 ++ .../paimon/index/HashBucketAssignerTest.java | 3 ++- .../ManifestCommittableSerializerTest.java | 1 + .../paimon/flink/sink/CommitterOperatorTest.java | 6 ++++++ 9 files changed, 48 insertions(+), 6 deletions(-) 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 f61083f11a28..544e766ae0d7 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 @@ -66,6 +66,7 @@ public class AppendOnlyWriter implements RecordWriter, MemoryOwner private final CompactManager compactManager; private final boolean forceCompact; private final List newFiles; + private final List deletedFiles; private final List compactBefore; private final List compactAfter; private final LongCounter seqNumCounter; @@ -101,6 +102,7 @@ public AppendOnlyWriter( this.compactManager = compactManager; this.forceCompact = forceCompact; this.newFiles = new ArrayList<>(); + this.deletedFiles = new ArrayList<>(); this.compactBefore = new ArrayList<>(); this.compactAfter = new ArrayList<>(); this.seqNumCounter = new LongCounter(maxSequenceNumber + 1); @@ -113,6 +115,7 @@ public AppendOnlyWriter( if (increment != null) { newFiles.addAll(increment.newFilesIncrement().newFiles()); + deletedFiles.addAll(increment.newFilesIncrement().deletedFiles()); compactBefore.addAll(increment.compactIncrement().compactBefore()); compactAfter.addAll(increment.compactIncrement().compactAfter()); } @@ -233,7 +236,10 @@ private void trySyncLatestCompaction(boolean blocking) private CommitIncrement drainIncrement() { NewFilesIncrement newFilesIncrement = - new NewFilesIncrement(new ArrayList<>(newFiles), Collections.emptyList()); + new NewFilesIncrement( + new ArrayList<>(newFiles), + new ArrayList<>(deletedFiles), + Collections.emptyList()); CompactIncrement compactIncrement = new CompactIncrement( new ArrayList<>(compactBefore), @@ -241,6 +247,7 @@ private CommitIncrement drainIncrement() { Collections.emptyList()); newFiles.clear(); + deletedFiles.clear(); compactBefore.clear(); compactAfter.clear(); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/NewFilesIncrement.java b/paimon-core/src/main/java/org/apache/paimon/io/NewFilesIncrement.java index 4e980bd31f58..b2f63070d131 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/NewFilesIncrement.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/NewFilesIncrement.java @@ -27,21 +27,31 @@ public class NewFilesIncrement { private final List newFiles; + private final List deletedFiles; private final List changelogFiles; - public NewFilesIncrement(List newFiles, List changelogFiles) { + public NewFilesIncrement( + List newFiles, + List deletedFiles, + List changelogFiles) { this.newFiles = newFiles; + this.deletedFiles = deletedFiles; this.changelogFiles = changelogFiles; } public static NewFilesIncrement emptyIncrement() { - return new NewFilesIncrement(Collections.emptyList(), Collections.emptyList()); + return new NewFilesIncrement( + Collections.emptyList(), Collections.emptyList(), Collections.emptyList()); } public List newFiles() { return newFiles; } + public List deletedFiles() { + return deletedFiles; + } + public List changelogFiles() { return changelogFiles; } 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 537d838d0cbb..1c94a2108561 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 @@ -69,6 +69,7 @@ public class MergeTreeWriter implements RecordWriter, MemoryOwner { @Nullable private final FieldsComparator userDefinedSeqComparator; private final LinkedHashSet newFiles; + private final LinkedHashSet deletedFiles; private final LinkedHashSet newFilesChangelog; private final LinkedHashMap compactBefore; private final LinkedHashSet compactAfter; @@ -107,12 +108,14 @@ public MergeTreeWriter( this.userDefinedSeqComparator = userDefinedSeqComparator; this.newFiles = new LinkedHashSet<>(); + this.deletedFiles = new LinkedHashSet<>(); this.newFilesChangelog = new LinkedHashSet<>(); this.compactBefore = new LinkedHashMap<>(); this.compactAfter = new LinkedHashSet<>(); this.compactChangelog = new LinkedHashSet<>(); if (increment != null) { newFiles.addAll(increment.newFilesIncrement().newFiles()); + deletedFiles.addAll(increment.newFilesIncrement().deletedFiles()); newFilesChangelog.addAll(increment.newFilesIncrement().changelogFiles()); increment .compactIncrement() @@ -253,7 +256,9 @@ public void sync() throws Exception { private CommitIncrement drainIncrement() { NewFilesIncrement newFilesIncrement = new NewFilesIncrement( - new ArrayList<>(newFiles), new ArrayList<>(newFilesChangelog)); + new ArrayList<>(newFiles), + new ArrayList<>(deletedFiles), + new ArrayList<>(newFilesChangelog)); CompactIncrement compactIncrement = new CompactIncrement( new ArrayList<>(compactBefore.values()), @@ -261,6 +266,7 @@ private CommitIncrement drainIncrement() { new ArrayList<>(compactChangelog)); newFiles.clear(); + deletedFiles.clear(); newFilesChangelog.clear(); compactBefore.clear(); compactAfter.clear(); @@ -306,6 +312,7 @@ public void close() throws Exception { // delete temporary files List delete = new ArrayList<>(newFiles); newFiles.clear(); + deletedFiles.clear(); for (DataFileMeta file : newFilesChangelog) { writerFactory.deleteFile(file.fileName(), file.level()); diff --git a/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java b/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java index 2b4612229bc1..ddb11a658786 100644 --- a/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java @@ -86,7 +86,8 @@ public static CommitMessage commitFile(BinaryRow partition, List d return new CommitMessageImpl( partition, 0, - new NewFilesIncrement(dataFileMetas, Collections.emptyList()), + new NewFilesIncrement( + dataFileMetas, Collections.emptyList(), Collections.emptyList()), new CompactIncrement( Collections.emptyList(), Collections.emptyList(), Collections.emptyList())); } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java index 5c6343956752..3f2ad87ac4a4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java @@ -572,6 +572,13 @@ private void collectChanges( .newFilesIncrement() .newFiles() .forEach(m -> appendTableFiles.add(makeEntry(FileKind.ADD, commitMessage, m))); + commitMessage + .newFilesIncrement() + .deletedFiles() + .forEach( + m -> + appendTableFiles.add( + makeEntry(FileKind.DELETE, commitMessage, m))); commitMessage .newFilesIncrement() .changelogFiles() diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java index da70c766d6b1..65a647edb9ee 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java @@ -75,6 +75,7 @@ private void serialize(CommitMessage obj, DataOutputView view) throws IOExceptio serializeBinaryRow(obj.partition(), view); view.writeInt(obj.bucket()); dataFileSerializer.serializeList(message.newFilesIncrement().newFiles(), view); + dataFileSerializer.serializeList(message.newFilesIncrement().deletedFiles(), view); dataFileSerializer.serializeList(message.newFilesIncrement().changelogFiles(), view); dataFileSerializer.serializeList(message.compactIncrement().compactBefore(), view); dataFileSerializer.serializeList(message.compactIncrement().compactAfter(), view); @@ -116,6 +117,7 @@ private CommitMessage deserialize(DataInputView view) throws IOException { deserializeBinaryRow(view), view.readInt(), new NewFilesIncrement( + dataFileSerializer.deserializeList(view), dataFileSerializer.deserializeList(view), dataFileSerializer.deserializeList(view)), new CompactIncrement( diff --git a/paimon-core/src/test/java/org/apache/paimon/index/HashBucketAssignerTest.java b/paimon-core/src/test/java/org/apache/paimon/index/HashBucketAssignerTest.java index 7bb11260a336..0ed83d3e0200 100644 --- a/paimon-core/src/test/java/org/apache/paimon/index/HashBucketAssignerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/index/HashBucketAssignerTest.java @@ -111,7 +111,8 @@ private CommitMessage createCommitMessage(BinaryRow partition, int bucket, Index return new CommitMessageImpl( partition, bucket, - new NewFilesIncrement(Collections.emptyList(), Collections.emptyList()), + new NewFilesIncrement( + Collections.emptyList(), Collections.emptyList(), Collections.emptyList()), new CompactIncrement( Collections.emptyList(), Collections.emptyList(), Collections.emptyList()), new IndexIncrement(Collections.singletonList(file))); diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java index ee279c097a17..ae01b58323e7 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java @@ -90,6 +90,7 @@ private static void addFileCommittables( public static NewFilesIncrement randomNewFilesIncrement() { return new NewFilesIncrement( + Arrays.asList(newFile(ID.incrementAndGet(), 0), newFile(ID.incrementAndGet(), 0)), Arrays.asList(newFile(ID.incrementAndGet(), 0), newFile(ID.incrementAndGet(), 0)), Arrays.asList(newFile(ID.incrementAndGet(), 0), newFile(ID.incrementAndGet(), 0))); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java index 0a3a8fae791c..26a7717325a3 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java @@ -302,6 +302,7 @@ public void testCommitInputEnd() throws Exception { BinaryRow.EMPTY_ROW, 0, new NewFilesIncrement( + Collections.emptyList(), Collections.emptyList(), Collections.emptyList()), new CompactIncrement( @@ -318,6 +319,7 @@ public void testCommitInputEnd() throws Exception { BinaryRow.EMPTY_ROW, 0, new NewFilesIncrement( + Collections.emptyList(), Collections.emptyList(), Collections.emptyList()), new CompactIncrement( @@ -333,6 +335,7 @@ public void testCommitInputEnd() throws Exception { BinaryRow.EMPTY_ROW, 0, new NewFilesIncrement( + Collections.emptyList(), Collections.emptyList(), Collections.emptyList()), new CompactIncrement( @@ -367,6 +370,7 @@ public void testCommitInputEnd() throws Exception { BinaryRow.EMPTY_ROW, 0, new NewFilesIncrement( + Collections.emptyList(), Collections.emptyList(), Collections.emptyList()), new CompactIncrement( @@ -383,6 +387,7 @@ public void testCommitInputEnd() throws Exception { BinaryRow.EMPTY_ROW, 0, new NewFilesIncrement( + Collections.emptyList(), Collections.emptyList(), Collections.emptyList()), new CompactIncrement( @@ -398,6 +403,7 @@ public void testCommitInputEnd() throws Exception { BinaryRow.EMPTY_ROW, 0, new NewFilesIncrement( + Collections.emptyList(), Collections.emptyList(), Collections.emptyList()), new CompactIncrement( From dfcf1e1e72a46c73adf2f789ed97bd2293a32637 Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Fri, 29 Mar 2024 14:58:05 +0800 Subject: [PATCH 79/79] [cdc] Fix that kafka message value might be null which causes NPE (#3113) --- .../action/cdc/kafka/KafkaActionUtils.java | 9 ++- ...ValueOnlyDeserializationSchemaWrapper.java | 71 +++++++++++++++++++ .../KafkaDebeziumSyncTableActionITCase.java | 59 +++++++++++++++ .../table/nullvalue/debezium-data-1.txt | 19 +++++ .../table/nullvalue/debezium-data-2.txt | 19 +++++ 5 files changed, 174 insertions(+), 3 deletions(-) create mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaValueOnlyDeserializationSchemaWrapper.java create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-1.txt create mode 100644 paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-2.txt diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionUtils.java index fcd0eeb8896c..64a543bebd6e 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionUtils.java @@ -83,9 +83,12 @@ public static KafkaSource buildKafkaSource(Configuration kafkaConfig) { Pattern.compile(kafkaConfig.get(KafkaConnectorOptions.TOPIC_PATTERN))); } - kafkaSourceBuilder - .setValueOnlyDeserializer(new SimpleStringSchema()) - .setGroupId(kafkaPropertiesGroupId(kafkaConfig)); + KafkaValueOnlyDeserializationSchemaWrapper schema = + new KafkaValueOnlyDeserializationSchemaWrapper<>(new SimpleStringSchema()); + kafkaSourceBuilder.setDeserializer(schema); + + kafkaSourceBuilder.setGroupId(kafkaPropertiesGroupId(kafkaConfig)); + Properties properties = createKafkaProperties(kafkaConfig); StartupMode startupMode = diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaValueOnlyDeserializationSchemaWrapper.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaValueOnlyDeserializationSchemaWrapper.java new file mode 100644 index 000000000000..5e6b96670bdb --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaValueOnlyDeserializationSchemaWrapper.java @@ -0,0 +1,71 @@ +/* + * 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.action.cdc.kafka; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.util.Collector; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** + * A class that wraps a {@link DeserializationSchema} as the value deserializer for a {@link + * ConsumerRecord}. + * + * @param the return type of the deserialization. + */ +class KafkaValueOnlyDeserializationSchemaWrapper implements KafkaRecordDeserializationSchema { + private static final long serialVersionUID = 1L; + private final DeserializationSchema deserializationSchema; + private static final Logger LOG = + LoggerFactory.getLogger(KafkaValueOnlyDeserializationSchemaWrapper.class); + + KafkaValueOnlyDeserializationSchemaWrapper(DeserializationSchema deserializationSchema) { + this.deserializationSchema = deserializationSchema; + } + + @Override + public void open(DeserializationSchema.InitializationContext context) throws Exception { + deserializationSchema.open(context); + } + + @Override + public void deserialize(ConsumerRecord message, Collector out) + throws IOException { + if (message.value() != null) { + deserializationSchema.deserialize(message.value(), out); + } else { + // see + // https://debezium.io/documentation/reference/2.5/connectors/mysql.html#mysql-tombstone-events + LOG.info( + "Found null message value:\n{}\nThis message will be ignored. It might be produced by tombstone-event, " + + "please check your Debezium and Kafka configuration.", + message); + } + } + + @Override + public TypeInformation getProducedType() { + return deserializationSchema.getProducedType(); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java index 7aba174d3dfc..ba96331621d7 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java @@ -18,9 +18,24 @@ package org.apache.paimon.flink.action.cdc.kafka; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT; + /** IT cases for {@link KafkaSyncTableAction}. */ public class KafkaDebeziumSyncTableActionITCase extends KafkaSyncTableActionITCase { @@ -103,4 +118,48 @@ public void testSchemaIncludeRecord1() throws Exception { public void testAllTypesWithSchema() throws Exception { testAllTypesWithSchemaImpl(DEBEZIUM); } + + @Test + @Timeout(60) + public void testMessageWithNullValue() throws Exception { + final String topic = "test_null_value"; + createTestTopic(topic, 1, 1); + + List lines = readLines("kafka/debezium/table/nullvalue/debezium-data-1.txt"); + writeRecordsToKafka(topic, lines); + + // write null value + Properties producerProperties = getStandardProps(); + producerProperties.setProperty("retries", "0"); + producerProperties.put( + "key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + producerProperties.put( + "value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + KafkaProducer kafkaProducer = new KafkaProducer<>(producerProperties); + kafkaProducer.send(new ProducerRecord<>(topic, null)); + kafkaProducer.close(); + + lines = readLines("kafka/debezium/table/nullvalue/debezium-data-2.txt"); + writeRecordsToKafka(topic, lines); + + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), "debezium-json"); + kafkaConfig.put(TOPIC.key(), topic); + KafkaSyncTableAction action = + syncTableActionBuilder(kafkaConfig) + .withPrimaryKeys("id") + .withTableConfig(getBasicTableConfig()) + .build(); + runActionWithDefaultEnv(action); + + RowType rowType = + RowType.of( + new DataType[] {DataTypes.STRING().notNull(), DataTypes.STRING()}, + new String[] {"id", "value"}); + waitForResult( + Arrays.asList("+I[1, A]", "+I[2, B]"), + getFileStoreTable(tableName), + rowType, + Collections.singletonList("id")); + } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-1.txt new file mode 100644 index 000000000000..fda10c2c1eb8 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-1.txt @@ -0,0 +1,19 @@ +/* + * 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. + */ + +{"before": null, "after": {"id": 1, "value": "A"}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "test", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-2.txt new file mode 100644 index 000000000000..d8de79c2948c --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/nullvalue/debezium-data-2.txt @@ -0,0 +1,19 @@ +/* + * 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. + */ + +{"before": null, "after": {"id": 2, "value": "B"}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684884000, "snapshot": "false", "db": "test", "sequence": null, "table": "test", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684884000, "transaction": null}
compact - CALL [catalog.]sys.compact('identifier')

- CALL [catalog.]sys.compact('identifier', 'partitions')

- CALL [catalog.]sys.compact('identifier', 'partitions', 'order_strategy', 'order_columns', 'table_options')
- TO compact a table. Arguments: -
  • identifier: the target table identifier. Cannot be empty.
  • -
  • partitions: partition filter.
  • -
  • order_strategy: 'order' or 'zorder' or 'hilbert' or 'none'. Left empty for 'none'.
  • -
  • order_columns: the columns need to be sort. Left empty if 'order_strategy' is 'none'.
  • -
  • table_options: additional dynamic options of the table.
  • + To compact a table. Arguments: +
  • table(required): the target table identifier.
  • +
  • partitions(optional): partition filter.
  • +
  • order_strategy(optional): 'order' or 'zorder' or 'hilbert' or 'none'.
  • +
  • order_by(optional): the columns need to be sort. Left empty if 'order_strategy' is 'none'.
  • +
  • options(optional): additional dynamic options of the table.
  • - CALL sys.compact('default.T', 'p=0', 'zorder', 'a,b', 'sink.parallelism=4') + CALL sys.compact(`table` => 'default.T', partitions => 'p=0', order_strategy => 'zorder', order_by => 'a,b', options => 'sink.parallelism=4')
    orc.compression.zstd.level
    13 Integer Define the compression level to use with ZStandard codec while writing data. The valid range is 1~22.