From 5e6c05f5c25be31dddfd247bc31d915f64de4547 Mon Sep 17 00:00:00 2001 From: moresun <738159199@qq.com> Date: Tue, 24 Sep 2024 15:01:15 +0800 Subject: [PATCH 01/33] [Flink] Integrate open lineage (#532) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * 1 support lineage for kafka、lakesoul datastream source to sink into lakesoul 2 suppport lineage for laksoul datastream source to sink to doris、lakesoul and kafka 3 support lineage for flinl sql submit 4 support yarn and k8s application mode Signed-off-by: maosen * modify lineage dependency Signed-off-by: maosen * add lakesoul lineage listener Signed-off-by: maosen * remove redundant items Signed-off-by: maosen * default to jdk 11 for build Signed-off-by: chenxu --------- Signed-off-by: maosen Signed-off-by: chenxu Co-authored-by: maosen Co-authored-by: chenxu --- .github/workflows/consistency-ci.yml | 4 +- .github/workflows/deployment.yml | 12 +- .github/workflows/native-build.yml | 8 +- lakesoul-flink/pom.xml | 16 +- .../flink/lakesoul/entry/SyncDatabase.java | 62 ++++++- .../lakesoul/entry/sql/flink/ExecuteSql.java | 17 +- .../entry/sql/flink/FlinkSqlSubmitter.java | 27 ++- .../flink/LakeSoulInAndOutputJobListener.java | 158 ++++++++++++++++++ .../sink/LakeSoulMultiTablesSink.java | 3 + .../DefaultOneTableBulkFormatBuilder.java | 3 + .../source/LakeSoulLookupTableSource.java | 4 +- .../flink/lakesoul/source/LakeSoulSource.java | 3 + .../lakesoul/table/LakeSoulTableSource.java | 4 +- .../flink/lakesoul/tool/JobOptions.java | 24 +++ 14 files changed, 317 insertions(+), 28 deletions(-) create mode 100644 lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/LakeSoulInAndOutputJobListener.java diff --git a/.github/workflows/consistency-ci.yml b/.github/workflows/consistency-ci.yml index f95902f97..253f5d2d9 100644 --- a/.github/workflows/consistency-ci.yml +++ b/.github/workflows/consistency-ci.yml @@ -56,10 +56,10 @@ jobs: - name: Init PG run: | ./script/meta_init_for_local_test.sh -j 2 - - name: Set up JDK 8 + - name: Set up JDK 11 uses: actions/setup-java@v4 with: - java-version: '8' + java-version: '11' distribution: 'temurin' cache: maven - name: Set up Python 3.9 diff --git a/.github/workflows/deployment.yml b/.github/workflows/deployment.yml index 5b26dfa81..8d9ed61a1 100644 --- a/.github/workflows/deployment.yml +++ b/.github/workflows/deployment.yml @@ -15,10 +15,10 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/checkout@v4 - - name: Set up JDK 8 + - name: Set up JDK 11 uses: actions/setup-java@v4 with: - java-version: '8' + java-version: '11' distribution: 'temurin' cache: maven - uses: actions-rs/toolchain@v1 @@ -51,10 +51,10 @@ jobs: runs-on: windows-latest steps: - uses: actions/checkout@v4 - - name: Set up JDK 8 + - name: Set up JDK 11 uses: actions/setup-java@v4 with: - java-version: '8' + java-version: '11' distribution: 'temurin' cache: maven - name: Install Protoc @@ -151,10 +151,10 @@ jobs: with: name: lakesoul-nativemetadata-x86_64-pc-windows-msvc path: ./rust/target/release/ - - name: Set up JDK 8 + - name: Set up JDK 11 uses: actions/setup-java@v4 with: - java-version: '8' + java-version: '11' distribution: 'temurin' cache: maven - name: Install Protoc diff --git a/.github/workflows/native-build.yml b/.github/workflows/native-build.yml index d64de6f66..bed3450f7 100644 --- a/.github/workflows/native-build.yml +++ b/.github/workflows/native-build.yml @@ -27,10 +27,10 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/checkout@v4 - - name: Set up JDK 8 + - name: Set up JDK 11 uses: actions/setup-java@v4 with: - java-version: '8' + java-version: '11' distribution: 'temurin' cache: maven - uses: actions-rs/toolchain@v1 @@ -64,10 +64,10 @@ jobs: runs-on: windows-latest steps: - uses: actions/checkout@v4 - - name: Set up JDK 8 + - name: Set up JDK 11 uses: actions/setup-java@v4 with: - java-version: '8' + java-version: '11' distribution: 'temurin' cache: maven - name: Install Protoc diff --git a/lakesoul-flink/pom.xml b/lakesoul-flink/pom.xml index 605b8e9a6..2ab1b17c8 100644 --- a/lakesoul-flink/pom.xml +++ b/lakesoul-flink/pom.xml @@ -92,6 +92,21 @@ SPDX-License-Identifier: Apache-2.0 ${flink.version} ${local.scope} + + + + io.openlineage + openlineage-flink + 1.19.0 + ${local.scope} + + + org.apache.flink + flink-connector-kafka + ${flink.version} + ${local.scope} + + org.apache.flink flink-core @@ -526,7 +541,6 @@ SPDX-License-Identifier: Apache-2.0 org.furyio:fury-core com.google.guava:guava com.google.guava:failureaccess - org.casbin:jdbc-adapter org.aspectj:aspectjrt diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/SyncDatabase.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/SyncDatabase.java index 952b3e2db..b49c37655 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/SyncDatabase.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/SyncDatabase.java @@ -14,12 +14,16 @@ import org.apache.flink.connector.mongodb.sink.MongoSink; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestOptions; +import org.apache.flink.lakesoul.entry.sql.flink.LakeSoulInAndOutputJobListener; import org.apache.flink.lakesoul.metadata.LakeSoulCatalog; +import org.apache.flink.lakesoul.tool.JobOptions; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.table.api.Table; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.bridge.java.StreamStatementSet; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.types.DataType; @@ -30,9 +34,9 @@ import java.sql.*; import java.util.*; +import java.util.stream.Collectors; import static org.apache.flink.lakesoul.entry.MongoSinkUtils.*; -import static org.apache.flink.lakesoul.tool.JobOptions.FLINK_CHECKPOINT; import static org.apache.flink.lakesoul.tool.JobOptions.JOB_CHECKPOINT_INTERVAL; import static org.apache.flink.lakesoul.tool.LakeSoulSinkDatabasesOptions.*; @@ -52,6 +56,8 @@ public class SyncDatabase { static int sinkParallelism; static String jdbcOrDorisOptions; static int checkpointInterval; + static LakeSoulInAndOutputJobListener listener; + static String lineageUrl = null; public static void main(String[] args) throws Exception { StringBuilder connectorOptions = new StringBuilder(); @@ -88,9 +94,30 @@ public static void main(String[] args) throws Exception { useBatch = parameter.getBoolean(BATHC_STREAM_SINK.key(), BATHC_STREAM_SINK.defaultValue()); Configuration conf = new Configuration(); conf.setString(RestOptions.BIND_PORT, "8081-8089"); - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + StreamExecutionEnvironment env = null; + lineageUrl = System.getenv("LINEAGE_URL"); + if (lineageUrl != null) { + conf.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); + conf.set(JobOptions.transportTypeOption, "http"); + conf.set(JobOptions.urlOption, lineageUrl); + conf.set(JobOptions.execAttach, true); + env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + String appName = env.getConfiguration().get(JobOptions.KUBE_CLUSTER_ID); + String namespace = System.getenv("LAKESOUL_CURRENT_DOMAIN"); + if (namespace == null) { + namespace = "public"; + } + if (useBatch) { + listener = new LakeSoulInAndOutputJobListener(lineageUrl, "BATCH"); + } else { + listener = new LakeSoulInAndOutputJobListener(lineageUrl); + } + listener.jobName(appName, namespace); + env.registerJobListener(listener); + } else { + env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + } env.setParallelism(sinkParallelism); - switch (dbType) { case "mysql": xsyncToMysql(env); @@ -223,6 +250,12 @@ public static String getTablePk(String sourceDataBae, String sourceTableName) { return primaryKeys.size() == 0 ? null : stringBuilder.toString(); } + public static String getTableDomain(String sourceDataBae, String sourceTableName) { + DBManager dbManager = new DBManager(); + TableInfo tableInfo = dbManager.getTableInfoByNameAndNamespace(sourceTableName, sourceDataBae); + return tableInfo.getDomain(); + } + public static void xsyncToPg(StreamExecutionEnvironment env) throws SQLException { if (useBatch) { env.setRuntimeMode(RuntimeExecutionMode.BATCH); @@ -358,7 +391,7 @@ public static void xsyncToMysql(StreamExecutionEnvironment env) throws SQLExcept conn.close(); } - public static void xsyncToDoris(StreamExecutionEnvironment env, String fenodes) throws SQLException { + public static void xsyncToDoris(StreamExecutionEnvironment env, String fenodes) throws Exception { if (useBatch) { env.setRuntimeMode(RuntimeExecutionMode.BATCH); } else { @@ -374,6 +407,14 @@ public static void xsyncToDoris(StreamExecutionEnvironment env, String fenodes) DataType[] fieldDataTypes = lakesoulTable.getSchema().getFieldDataTypes(); String[] fieldNames = lakesoulTable.getSchema().getFieldNames(); String[] dorisFieldTypes = getDorisFieldTypes(fieldDataTypes); + if (lineageUrl != null) { + String inputName = "lakeSoul." + sourceDatabase + "." + sourceTableName; + String inputnNamespace = getTableDomain(sourceDatabase,sourceTableName); + String[] inputTypes = Arrays.stream(fieldDataTypes).map(type -> type.toString()).collect(Collectors.toList()).toArray(new String[0]); + listener.inputFacets(inputName,inputnNamespace,fieldNames,inputTypes); + String targetName = "doris." + targetDatabase + "." + targetTableName; + listener.outputFacets(targetName,"lake-public",fieldNames,dorisFieldTypes); + } StringBuilder coulmns = new StringBuilder(); for (int i = 0; i < fieldDataTypes.length; i++) { coulmns.append("`").append(fieldNames[i]).append("` ").append(dorisFieldTypes[i]); @@ -408,7 +449,16 @@ public static void xsyncToDoris(StreamExecutionEnvironment env, String fenodes) } tEnvs.executeSql(sql); - tEnvs.executeSql("insert into " + targetTableName + " select * from lakeSoul.`" + sourceDatabase + "`." + sourceTableName); + if (lineageUrl != null){ + String insertsql = "insert into " + targetTableName + " select * from lakeSoul.`" + sourceDatabase + "`." + sourceTableName; + StreamStatementSet statements = tEnvs.createStatementSet(); + statements.addInsertSql(insertsql); + statements.attachAsDataStream(); + env.execute(); + }else{ + tEnvs.executeSql("insert into " + targetTableName + " select * from lakeSoul.`" + sourceDatabase + "`." + sourceTableName); + + } } public static void xsyncToMongodb(StreamExecutionEnvironment env, @@ -443,4 +493,4 @@ public static void xsyncToMongodb(StreamExecutionEnvironment env, rowDataStream.sinkTo(sink).setParallelism(sinkParallelism); env.execute(); } -} \ No newline at end of file +} diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/ExecuteSql.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/ExecuteSql.java index ac59c6ed3..53b6f3bf0 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/ExecuteSql.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/ExecuteSql.java @@ -42,7 +42,7 @@ public static void executeSqlFileContent(String script, StreamTableEnvironment t Parser parser = ((TableEnvironmentInternal) tableEnv).getParser(); StreamStatementSet statementSet = tableEnv.createStatementSet(); - + Boolean hasModifiedOp = false; for (String statement : statements) { Operation operation; try { @@ -75,6 +75,7 @@ public static void executeSqlFileContent(String script, StreamTableEnvironment t } else if (operation instanceof ModifyOperation) { System.out.println(MessageFormatter.format("\n======Executing insertion:\n{}", statement).getMessage()); // add insertion to statement set + hasModifiedOp = true; statementSet.addInsertSql(statement); } else if ((operation instanceof QueryOperation) || (operation instanceof AddJarOperation)) { LOG.warn("SQL Statement {} is ignored", statement); @@ -85,12 +86,16 @@ public static void executeSqlFileContent(String script, StreamTableEnvironment t tableEnv.executeSql(statement).print(); } } - statementSet.attachAsDataStream(); - Configuration conf = (Configuration) env.getConfiguration(); + if (hasModifiedOp) { + statementSet.attachAsDataStream(); + Configuration conf = (Configuration) env.getConfiguration(); - // try get k8s cluster name - String k8sClusterID = conf.getString("kubernetes.cluster-id", ""); - env.execute(k8sClusterID.isEmpty() ? null : k8sClusterID + "-job"); + // try get k8s cluster name + String k8sClusterID = conf.getString("kubernetes.cluster-id", ""); + env.execute(k8sClusterID.isEmpty() ? null : k8sClusterID + "-job"); + } else { + System.out.println("There's no INSERT INTO statement, the program will terminate"); + } } public static List parseStatements(String script) { diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/FlinkSqlSubmitter.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/FlinkSqlSubmitter.java index 729859b56..15e40aa2c 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/FlinkSqlSubmitter.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/FlinkSqlSubmitter.java @@ -4,13 +4,17 @@ package org.apache.flink.lakesoul.entry.sql.flink; +import io.openlineage.flink.OpenLineageFlinkJobListener; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobListener; import org.apache.flink.lakesoul.entry.sql.Submitter; import org.apache.flink.lakesoul.entry.sql.common.FlinkOption; import org.apache.flink.lakesoul.entry.sql.common.JobType; import org.apache.flink.lakesoul.entry.sql.common.SubmitOption; import org.apache.flink.lakesoul.entry.sql.utils.FileUtil; +import org.apache.flink.lakesoul.tool.JobOptions; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -32,6 +36,13 @@ public FlinkSqlSubmitter(SubmitOption submitOption) { @Override public void submit() throws Exception { + String lineageUrl = System.getenv("LINEAGE_URL"); + Configuration conf = new Configuration(); + if (lineageUrl != null) { + conf.set(JobOptions.transportTypeOption, "http"); + conf.set(JobOptions.urlOption, lineageUrl); + conf.set(JobOptions.execAttach, true); + } EnvironmentSettings settings = null; StreamTableEnvironment tEnv = null; if (submitOption.getJobType().equals(JobType.STREAM.getType())) { @@ -47,10 +58,24 @@ public void submit() throws Exception { } else { throw new RuntimeException("jobType is not supported: " + submitOption.getJobType()); } - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(conf); if (submitOption.getJobType().equals(JobType.STREAM.getType())) { this.setCheckpoint(env); } + if (lineageUrl != null) { + String appName = env.getConfiguration().get(JobOptions.KUBE_CLUSTER_ID); + String namespace = System.getenv("LAKESOUL_CURRENT_DOMAIN"); + if (namespace == null) { + namespace = "lake-public"; + } + LOG.info("----namespace:table----{}:{}", appName, namespace); + JobListener listener = OpenLineageFlinkJobListener.builder() + .executionEnvironment(env) + .jobName(appName) + .jobNamespace(namespace) + .build(); + env.registerJobListener(listener); + } tEnv = StreamTableEnvironment.create(env, settings); String sql = FileUtil.readHDFSFile(submitOption.getSqlFilePath()); diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/LakeSoulInAndOutputJobListener.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/LakeSoulInAndOutputJobListener.java new file mode 100644 index 000000000..21310abbf --- /dev/null +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/LakeSoulInAndOutputJobListener.java @@ -0,0 +1,158 @@ +// SPDX-FileCopyrightText: 2023 LakeSoul Contributors +// +// SPDX-License-Identifier: Apache-2.0 +package org.apache.flink.lakesoul.entry.sql.flink; + + +import io.openlineage.client.OpenLineage; +import io.openlineage.client.OpenLineageClient; +import io.openlineage.client.transports.HttpConfig; +import io.openlineage.client.transports.HttpTransport; +import io.openlineage.client.transports.Transport; +import io.openlineage.flink.client.EventEmitter; +import io.openlineage.flink.shaded.org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.core.execution.DetachedJobExecutionResult; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.execution.JobListener; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.net.URI; +import java.time.ZonedDateTime; +import java.util.*; + +public class LakeSoulInAndOutputJobListener implements JobListener { + private static final Logger log = LoggerFactory.getLogger(LakeSoulInAndOutputJobListener.class); + private OpenLineageClient client; + private OpenLineage openLineage; + private List inputDatasets; + private List outputDatasets; + private String executeMode = "STREAM"; + private UUID runId; + OpenLineage.Run run; + private OpenLineage.Job job; + + public LakeSoulInAndOutputJobListener(String url) { + HttpConfig config = new HttpConfig(); + config.setUrl(URI.create(url)); + Transport transport = new HttpTransport(config); + client = new OpenLineageClient(transport); + openLineage = new OpenLineage(EventEmitter.OPEN_LINEAGE_CLIENT_URI); + } + + public LakeSoulInAndOutputJobListener(String url, String executeMode) { + this(url); + this.executeMode = executeMode; + } + + public LakeSoulInAndOutputJobListener jobName(String name, String namespace) { + this.runId = UUID.randomUUID(); + this.run = openLineage.newRunBuilder().runId(this.runId).build(); + OpenLineage.JobFacets jobFacets = openLineage.newJobFacetsBuilder().jobType(openLineage.newJobTypeJobFacetBuilder().jobType("Flink Job").integration("Flink").processingType(this.executeMode).build()).build(); + this.job = openLineage.newJobBuilder().name(name).namespace(namespace).facets(jobFacets).build(); + return this; + } + + public LakeSoulInAndOutputJobListener inputFacets(String inputName, String inputNamespace, String[] inputSchemaNames, String[] inputSchemaTypes) { + List schemaFields = new ArrayList<>(); + if (inputSchemaNames != null && inputSchemaTypes != null && inputSchemaTypes.length == inputSchemaTypes.length) { + for (int i = 0; i < inputSchemaNames.length; i++) { + schemaFields.add(openLineage.newSchemaDatasetFacetFieldsBuilder().name(inputSchemaNames[i]).type(inputSchemaTypes[i]).build()); + } + } + if (inputSchemaNames != null && inputSchemaTypes == null) { + for (int i = 0; i < inputSchemaNames.length; i++) { + schemaFields.add(openLineage.newSchemaDatasetFacetFieldsBuilder().name(inputSchemaNames[i]).build()); + } + } + + OpenLineage.SchemaDatasetFacet schemaFacet = openLineage.newSchemaDatasetFacetBuilder().fields(schemaFields).build(); + this.inputDatasets = Arrays.asList( + openLineage.newInputDatasetBuilder().name(inputName).namespace(inputNamespace) + .facets( + openLineage.newDatasetFacetsBuilder().schema(schemaFacet).build() + ).build() + ); + return this; + } + + public LakeSoulInAndOutputJobListener outputFacets(String outputName, String outputNamespace, String[] outputSchemaNames, String[] outputSchemaTypes) { + + List schemaFields = new ArrayList<>(); + if (outputSchemaNames != null && outputSchemaTypes != null && outputSchemaTypes.length == outputSchemaTypes.length) { + for (int i = 0; i < outputSchemaNames.length; i++) { + schemaFields.add(openLineage.newSchemaDatasetFacetFieldsBuilder().name(outputSchemaNames[i]).type(outputSchemaTypes[i]).build()); + } + } + if (outputSchemaNames != null && outputSchemaTypes == null) { + for (int i = 0; i < outputSchemaNames.length; i++) { + schemaFields.add(openLineage.newSchemaDatasetFacetFieldsBuilder().name(outputSchemaNames[i]).build()); + } + } + + OpenLineage.SchemaDatasetFacet schemaFacet = openLineage.newSchemaDatasetFacetBuilder().fields(schemaFields).build(); + this.outputDatasets = Arrays.asList( + openLineage.newOutputDatasetBuilder().name(outputName).namespace(outputNamespace) + .facets( + openLineage.newDatasetFacetsBuilder().schema(schemaFacet).build() + ).build() + ); + return this; + } + + + @Override + public void onJobSubmitted(@Nullable JobClient jobClient, @Nullable Throwable throwable) { + log.info("------lake onjobsubmit----jobid:{}",jobClient.getJobID()); + + OpenLineage.RunEvent runStateUpdate = + openLineage.newRunEventBuilder() + .eventType(OpenLineage.RunEvent.EventType.RUNNING) + .eventTime(ZonedDateTime.now()) + .run(this.run) + .job(this.job) + .inputs(this.inputDatasets) + .outputs(this.outputDatasets) + .build(); + if(this.inputDatasets != null || this.outputDatasets != null) { + this.client.emit(runStateUpdate); + } + } + + @Override + public void onJobExecuted(@Nullable JobExecutionResult jobExecutionResult, @Nullable Throwable throwable) { + if (jobExecutionResult instanceof DetachedJobExecutionResult) { + log.warn("Job running in detached mode. Set execution.attached to true if you want to emit completed events."); + } else { + OpenLineage.RunEvent runStateUpdate = null; + if (jobExecutionResult != null) { + log.info("------onjobexecuted----jobresult:{}",jobExecutionResult.getJobExecutionResult().toString()); + runStateUpdate = + openLineage.newRunEventBuilder() + .eventType(OpenLineage.RunEvent.EventType.COMPLETE) + .eventTime(ZonedDateTime.now()) + .run(this.run) + .job(this.job) + .inputs(this.inputDatasets) + .outputs(this.outputDatasets) + .build(); + } else { + log.info("------onjobexecuted----jobresult:{null}"); + OpenLineage.Run failRun = openLineage.newRunBuilder().runId(this.runId).facets(openLineage.newRunFacetsBuilder().errorMessage(openLineage.newErrorMessageRunFacet(throwable.getMessage(), "JAVA", ExceptionUtils.getStackTrace(throwable))).build()).build(); + runStateUpdate = + openLineage.newRunEventBuilder() + .eventType(OpenLineage.RunEvent.EventType.FAIL) + .eventTime(ZonedDateTime.now()) + .run(failRun) + .job(this.job) + .inputs(this.inputDatasets) + .outputs(this.outputDatasets) + .build(); + } + this.client.emit(runStateUpdate); + } + } + +} diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTablesSink.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTablesSink.java index f561bc49b..8a8984000 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTablesSink.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTablesSink.java @@ -133,4 +133,7 @@ public Collection getCompatibleStateNames() { // StreamingFileSink return Collections.singleton("lakesoul-cdc-multitable-bucket-states"); } + public BucketsBuilder getBucketsBuilder(){ + return this.bucketsBuilder; + } } diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/bucket/DefaultOneTableBulkFormatBuilder.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/bucket/DefaultOneTableBulkFormatBuilder.java index 5f932f525..489e65a77 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/bucket/DefaultOneTableBulkFormatBuilder.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/bucket/DefaultOneTableBulkFormatBuilder.java @@ -33,6 +33,9 @@ public DefaultOneTableBulkFormatBuilder( super(basePath, conf, new DefaultLakeSoulWriterBucketFactory(conf)); this.identity = identity; } + public TableSchemaIdentity getIdentity(){ + return this.identity; + } @Override public AbstractLakeSoulMultiTableSinkWriter createWriter(Sink.InitContext context, int subTaskId) throws diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulLookupTableSource.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulLookupTableSource.java index 9fbb63336..a4552a5a1 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulLookupTableSource.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulLookupTableSource.java @@ -67,7 +67,9 @@ public LakeSoulLookupTableSource(TableId tableId, catalogTable.getOptions().forEach(configuration::setString); validateLookupConfigurations(); } - + public TableId getTableId(){ + return this.tableId; + } private void validateLookupConfigurations() { String partitionInclude = configuration.get(STREAMING_SOURCE_PARTITION_INCLUDE); diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulSource.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulSource.java index 05af12a12..12736ebde 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulSource.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulSource.java @@ -80,6 +80,9 @@ public LakeSoulSource(TableId tableId, this.pushedFilter = pushedFilter; this.partitionFilters = partitionFilters; } + public TableId getTableId(){ + return this.tableId; + } @Override public Boundedness getBoundedness() { diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/table/LakeSoulTableSource.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/table/LakeSoulTableSource.java index a943e6baa..2e3cd4cd4 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/table/LakeSoulTableSource.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/table/LakeSoulTableSource.java @@ -89,7 +89,9 @@ public LakeSoulTableSource(TableId tableId, this.optionParams = optionParams; this.modificationContext = null; } - + public TableId getTableId(){ + return this.tableId; + } @Override public DynamicTableSource copy() { LakeSoulTableSource newInstance = new LakeSoulTableSource(this.tableId, diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/JobOptions.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/JobOptions.java index a86d3b980..87cd97904 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/JobOptions.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/JobOptions.java @@ -11,6 +11,7 @@ import java.time.Duration; import static org.apache.flink.configuration.ConfigOptions.key; +import static org.apache.flink.configuration.description.TextElement.code; import static org.apache.flink.configuration.description.TextElement.text; public class JobOptions { @@ -120,4 +121,27 @@ public class JobOptions { .defaultValue("file:///") .withDescription("Option to set fs default scheme"); + public static final ConfigOption transportTypeOption = + ConfigOptions.key("openlineage.transport.type").stringType().defaultValue("http"); + public static final ConfigOption urlOption = + ConfigOptions.key("openlineage.transport.url").stringType().noDefaultValue(); + public static final ConfigOption execAttach = + ConfigOptions.key("execution.attached").booleanType().defaultValue(false); + public static final ConfigOption lineageOption = + ConfigOptions.key("openlineage.executed").booleanType().defaultValue(false); + + public static final ConfigOption KUBE_CLUSTER_ID = + key("kubernetes.cluster-id") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "The cluster-id, which should be no more than 45 characters, is used for identifying a unique Flink cluster. " + + "The id must only contain lowercase alphanumeric characters and \"-\". " + + "The required format is %s. " + + "If not set, the client will automatically generate it with a random ID.", + code("[a-z]([-a-z0-9]*[a-z0-9])")) + .build()); + } From 39686f09bd9694025cc6cce80d959c6f11e14eed Mon Sep 17 00:00:00 2001 From: maosen <738159199@qq.com> Date: Wed, 25 Sep 2024 11:27:36 +0800 Subject: [PATCH 02/33] add kafka lineage into and out of lakesoul --- .../apache/flink/lakesoul/entry/KafkaCdc.java | 48 ++++++++++++++++--- .../lakesoul/entry/LakeSoulKafkaAvroSink.java | 40 +++++++++++++++- .../flink/LakeSoulInAndOutputJobListener.java | 27 +++++++++++ .../LakeSoulSinkGlobalCommitter.java | 25 ++++++++++ .../flink/lakesoul/tool/JobOptions.java | 6 +++ 5 files changed, 138 insertions(+), 8 deletions(-) diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/KafkaCdc.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/KafkaCdc.java index 1fb03b086..fd8a1a162 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/KafkaCdc.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/KafkaCdc.java @@ -26,7 +26,9 @@ import org.apache.flink.connector.kafka.source.KafkaSource; import org.apache.flink.connector.kafka.source.KafkaSourceBuilder; import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.lakesoul.entry.sql.flink.LakeSoulInAndOutputJobListener; import org.apache.flink.lakesoul.sink.LakeSoulMultiTableSinkStreamBuilder; +import org.apache.flink.lakesoul.tool.JobOptions; import org.apache.flink.lakesoul.tool.LakeSoulSinkOptions; import org.apache.flink.lakesoul.types.*; import org.apache.flink.streaming.api.CheckpointingMode; @@ -43,6 +45,7 @@ import java.time.ZoneId; import java.time.format.DateTimeFormatter; import java.util.Arrays; +import java.util.Map; import java.util.Properties; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; @@ -53,11 +56,10 @@ public class KafkaCdc { /** - * @param args - * --bootstrap_servers localhost:9092 --topic t_test --auto_offset_reset earliest --group_id test - * --source.parallelism 4 --sink.parallelism 4 --job.checkpoint_interval 5000 - * --warehouse_path /tmp/lakesoul/kafka - * --flink.checkpoint /tmp/lakesoul/chk + * @param args --bootstrap_servers localhost:9092 --topic t_test --auto_offset_reset earliest --group_id test + * --source.parallelism 4 --sink.parallelism 4 --job.checkpoint_interval 5000 + * --warehouse_path /tmp/lakesoul/kafka + * --flink.checkpoint /tmp/lakesoul/chk * @throws Exception */ public static void main(String[] args) throws Exception { @@ -164,7 +166,30 @@ public static void main(String[] args) throws Exception { conf.set(LakeSoulSinkOptions.LOGICALLY_DROP_COLUM, logicallyDropColumn); conf.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + String lineageUrl = System.getenv("LINEAGE_URL"); + LakeSoulInAndOutputJobListener listener = null; + StreamExecutionEnvironment env; + String appName = null; + String namespace = null; + if (lineageUrl != null) { + conf.set(JobOptions.transportTypeOption, "http"); + conf.set(JobOptions.urlOption, lineageUrl); + conf.set(JobOptions.execAttach, true); + conf.set(lineageOption, true); + env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + appName = env.getConfiguration().get(JobOptions.KUBE_CLUSTER_ID); + namespace = System.getenv("LAKESOUL_CURRENT_DOMAIN"); + if (namespace == null) { + namespace = "public"; + } + listener = new LakeSoulInAndOutputJobListener(lineageUrl); + listener.jobName(appName, namespace); + listener.inputFacets("kafka." + kafkaTopic, "kafka-public", null, null); + env.registerJobListener(listener); + } else { + env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + } + env.getConfig().registerTypeWithKryoSerializer(BinarySourceRecord.class, BinarySourceRecordSerializer.class); ParameterTool pt = ParameterTool.fromMap(conf.toMap()); env.getConfig().setGlobalJobParameters(pt); @@ -207,7 +232,16 @@ public static void main(String[] args) throws Exception { LakeSoulMultiTableSinkStreamBuilder.Context context = new LakeSoulMultiTableSinkStreamBuilder.Context(); context.env = env; - context.conf = (Configuration) env.getConfiguration(); + if (lineageUrl != null) { + Map confs = ((Configuration) env.getConfiguration()).toMap(); + confs.put(linageJobName.key(), appName); + confs.put(linageJobNamespace.key(), namespace); + confs.put(lineageJobUUID.key(), listener.getRunId()); + confs.put(lineageOption.key(),"true"); + context.conf = Configuration.fromMap(confs); + } else { + context.conf = (Configuration) env.getConfiguration(); + } LakeSoulMultiTableSinkStreamBuilder builder = new LakeSoulMultiTableSinkStreamBuilder(kafkaSource, context, lakeSoulRecordConvert); DataStreamSource source = builder.buildMultiTableSource("Kafka Source"); diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/LakeSoulKafkaAvroSink.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/LakeSoulKafkaAvroSink.java index 2c9f9b55d..edf8ac869 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/LakeSoulKafkaAvroSink.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/LakeSoulKafkaAvroSink.java @@ -7,6 +7,7 @@ import com.dmetasoul.lakesoul.meta.DBManager; import com.dmetasoul.lakesoul.meta.DBUtil; import com.dmetasoul.lakesoul.meta.entity.TableInfo; +import org.apache.arrow.vector.types.pojo.Field; import org.apache.arrow.vector.types.pojo.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.flink.api.common.eventtime.WatermarkStrategy; @@ -23,7 +24,9 @@ import org.apache.flink.formats.avro.RowDataToAvroConverters.RowDataToAvroConverter; import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroSerializationSchema; import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.lakesoul.entry.sql.flink.LakeSoulInAndOutputJobListener; import org.apache.flink.lakesoul.source.arrow.LakeSoulArrowSource; +import org.apache.flink.lakesoul.tool.JobOptions; import org.apache.flink.lakesoul.tool.LakeSoulKeyGen; import org.apache.flink.lakesoul.tool.LakeSoulSinkOptions; import org.apache.flink.lakesoul.types.LakSoulKafkaPartitioner; @@ -137,7 +140,42 @@ public static void main(String[] args) throws Exception { conf.set(LakeSoulSinkOptions.BUCKET_PARALLELISM, sinkParallelism); conf.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + String lineageUrl = System.getenv("LINEAGE_URL"); + LakeSoulInAndOutputJobListener listener; + StreamExecutionEnvironment env; + if (lineageUrl != null) { + conf.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); + conf.set(JobOptions.transportTypeOption, "http"); + conf.set(JobOptions.urlOption, lineageUrl); + conf.set(JobOptions.execAttach, true); + conf.set(lineageOption,true); + env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + String appName = env.getConfiguration().get(JobOptions.KUBE_CLUSTER_ID); + String namespace = System.getenv("LAKESOUL_CURRENT_DOMAIN"); + if (namespace == null) { + namespace = "public"; + } + listener = new LakeSoulInAndOutputJobListener(lineageUrl); + listener.jobName(appName, namespace); + listener.outputFacets("Kafka."+kafkaTopic,"kafka-public",null,null); + DBManager lakesoulDBManager = new DBManager(); + TableInfo tableInfo = lakesoulDBManager.getTableInfoByNameAndNamespace(lakeSoulTableName, lakeSoulDBName); + String tableSchema = tableInfo.getTableSchema(); + Schema schema = Schema.fromJSON(tableSchema); + int size = schema.getFields().size(); + String[] colNames = new String[size]; + String[] colTypes = new String[size]; + for (int i = 0; i < size; i++) { + Field field = schema.getFields().get(i); + colNames[i] = field.getName(); + colTypes[i] = field.getType().toString(); + } + listener.inputFacets("lakesoul." + lakeSoulDBName + "." + lakeSoulTableName, tableInfo.getDomain(), colNames,colTypes); + env.registerJobListener(listener); + } else { + env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + } + ParameterTool pt = ParameterTool.fromMap(conf.toMap()); env.getConfig().setGlobalJobParameters(pt); env.enableCheckpointing(checkpointInterval); diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/LakeSoulInAndOutputJobListener.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/LakeSoulInAndOutputJobListener.java index 21310abbf..c3d70c18a 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/LakeSoulInAndOutputJobListener.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/LakeSoulInAndOutputJobListener.java @@ -30,6 +30,11 @@ public class LakeSoulInAndOutputJobListener implements JobListener { private List inputDatasets; private List outputDatasets; private String executeMode = "STREAM"; + + public String getRunId() { + return runId.toString(); + } + private UUID runId; OpenLineage.Run run; private OpenLineage.Job job; @@ -47,6 +52,14 @@ public LakeSoulInAndOutputJobListener(String url, String executeMode) { this.executeMode = executeMode; } + public LakeSoulInAndOutputJobListener jobName(String name, String namespace,String uuid) { + this.runId = UUID.fromString(uuid); + this.run = openLineage.newRunBuilder().runId(this.runId).build(); + OpenLineage.JobFacets jobFacets = openLineage.newJobFacetsBuilder().jobType(openLineage.newJobTypeJobFacetBuilder().jobType("Flink Job").integration("Flink").processingType(this.executeMode).build()).build(); + this.job = openLineage.newJobBuilder().name(name).namespace(namespace).facets(jobFacets).build(); + return this; + } + public LakeSoulInAndOutputJobListener jobName(String name, String namespace) { this.runId = UUID.randomUUID(); this.run = openLineage.newRunBuilder().runId(this.runId).build(); @@ -77,6 +90,20 @@ public LakeSoulInAndOutputJobListener inputFacets(String inputName, String input ); return this; } + public void emit(){ + OpenLineage.RunEvent runStateUpdate = + openLineage.newRunEventBuilder() + .eventType(OpenLineage.RunEvent.EventType.COMPLETE) + .eventTime(ZonedDateTime.now()) + .run(this.run) + .job(this.job) + .inputs(this.inputDatasets) + .outputs(this.outputDatasets) + .build(); + if(this.inputDatasets != null || this.outputDatasets != null){ + this.client.emit(runStateUpdate); + } + } public LakeSoulInAndOutputJobListener outputFacets(String outputName, String outputNamespace, String[] outputSchemaNames, String[] outputSchemaTypes) { diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkGlobalCommitter.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkGlobalCommitter.java index e368847bf..64ddb80f4 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkGlobalCommitter.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkGlobalCommitter.java @@ -11,17 +11,20 @@ import com.dmetasoul.lakesoul.meta.DBUtil; import com.dmetasoul.lakesoul.meta.dao.TableInfoDao; import com.dmetasoul.lakesoul.meta.entity.TableInfo; +import org.apache.arrow.vector.types.pojo.Field; import org.apache.arrow.vector.types.pojo.Schema; import org.apache.flink.api.connector.sink.GlobalCommitter; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; +import org.apache.flink.lakesoul.entry.sql.flink.LakeSoulInAndOutputJobListener; import org.apache.flink.lakesoul.sink.LakeSoulMultiTablesSink; import org.apache.flink.lakesoul.sink.state.LakeSoulMultiTableSinkCommittable; import org.apache.flink.lakesoul.sink.state.LakeSoulMultiTableSinkGlobalCommittable; import org.apache.flink.lakesoul.sink.writer.AbstractLakeSoulMultiTableSinkWriter; import org.apache.flink.lakesoul.tool.FlinkUtil; +import org.apache.flink.lakesoul.tool.JobOptions; import org.apache.flink.lakesoul.types.TableSchemaIdentity; import org.apache.flink.runtime.execution.SuppressRestartsException; import org.apache.flink.table.catalog.ObjectPath; @@ -141,6 +144,14 @@ public List commit( StructType sparkSchema = ArrowUtils.fromArrowSchema(msgSchema); TableInfo tableInfo = dbManager.getTableInfoByNameAndNamespace(tableName, tableNamespace); + LakeSoulInAndOutputJobListener listener = null; + if (this.conf.getBoolean(JobOptions.lineageOption)) { + listener = new LakeSoulInAndOutputJobListener(this.conf.getString(JobOptions.urlOption)); + String uuid = this.conf.getString(JobOptions.lineageJobUUID); + String jobName = this.conf.getString(JobOptions.linageJobName); + String namespace = this.conf.getString(JobOptions.linageJobNamespace); + listener.jobName(jobName, namespace, uuid); + } if (tableInfo == null) { if (!conf.getBoolean(AUTO_SCHEMA_CHANGE)) { throw new SuppressRestartsException( @@ -171,6 +182,20 @@ public List commit( dbManager.createNewTable(tableId, tableNamespace, tableName, identity.tableLocation, msgSchema.toJson(), properties, partition); + if (this.conf.getBoolean(JobOptions.lineageOption)) { + String domain = dbManager.getNamespaceByNamespace(tableNamespace).getDomain(); + int size = msgSchema.getFields().size(); + String[] colNames = new String[size]; + String[] colTypes = new String[size]; + for (int i = 0; i < size; i++) { + Field field = msgSchema.getFields().get(i); + colNames[i] = field.getName(); + colTypes[i] = field.getType().toString(); + } + listener.outputFacets("lakesoul." + tableNamespace + "." + tableName, domain, colNames,colTypes); + listener.emit(); + } + } else { if (conf.getBoolean(AUTO_SCHEMA_CHANGE)) { DBUtil.TablePartitionKeys diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/JobOptions.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/JobOptions.java index 87cd97904..f0ab3c42a 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/JobOptions.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/JobOptions.java @@ -129,6 +129,12 @@ public class JobOptions { ConfigOptions.key("execution.attached").booleanType().defaultValue(false); public static final ConfigOption lineageOption = ConfigOptions.key("openlineage.executed").booleanType().defaultValue(false); + public static final ConfigOption lineageJobUUID = + ConfigOptions.key("openlineage.uuid").stringType().noDefaultValue(); + public static final ConfigOption linageJobName = + ConfigOptions.key("openlineage.jobname").stringType().noDefaultValue(); + public static final ConfigOption linageJobNamespace = + ConfigOptions.key("openlineage.jobnamespace").stringType().noDefaultValue(); public static final ConfigOption KUBE_CLUSTER_ID = key("kubernetes.cluster-id") From d86dab60c96f8550e8d9e3b497feab6664fa520b Mon Sep 17 00:00:00 2001 From: maosen <738159199@qq.com> Date: Mon, 30 Sep 2024 17:56:44 +0800 Subject: [PATCH 03/33] extract job name for batch schedule change job runstate from running to completed --- .../apache/flink/lakesoul/entry/KafkaCdc.java | 7 ++-- .../lakesoul/entry/LakeSoulKafkaAvroSink.java | 20 +++++----- .../flink/lakesoul/entry/SyncDatabase.java | 39 ++++++++++--------- .../entry/sql/flink/FlinkSqlSubmitter.java | 4 +- .../flink/LakeSoulInAndOutputJobListener.java | 2 +- .../lakesoul/entry/sql/utils/FileUtil.java | 8 ++++ 6 files changed, 46 insertions(+), 34 deletions(-) diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/KafkaCdc.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/KafkaCdc.java index fd8a1a162..8f9c81847 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/KafkaCdc.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/KafkaCdc.java @@ -27,6 +27,7 @@ import org.apache.flink.connector.kafka.source.KafkaSourceBuilder; import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; import org.apache.flink.lakesoul.entry.sql.flink.LakeSoulInAndOutputJobListener; +import org.apache.flink.lakesoul.entry.sql.utils.FileUtil; import org.apache.flink.lakesoul.sink.LakeSoulMultiTableSinkStreamBuilder; import org.apache.flink.lakesoul.tool.JobOptions; import org.apache.flink.lakesoul.tool.LakeSoulSinkOptions; @@ -174,10 +175,10 @@ public static void main(String[] args) throws Exception { if (lineageUrl != null) { conf.set(JobOptions.transportTypeOption, "http"); conf.set(JobOptions.urlOption, lineageUrl); - conf.set(JobOptions.execAttach, true); + conf.set(JobOptions.execAttach, false); conf.set(lineageOption, true); env = StreamExecutionEnvironment.getExecutionEnvironment(conf); - appName = env.getConfiguration().get(JobOptions.KUBE_CLUSTER_ID); + appName = FileUtil.getSubNameFromBatch(env.getConfiguration().get(JobOptions.KUBE_CLUSTER_ID)); namespace = System.getenv("LAKESOUL_CURRENT_DOMAIN"); if (namespace == null) { namespace = "public"; @@ -237,7 +238,7 @@ public static void main(String[] args) throws Exception { confs.put(linageJobName.key(), appName); confs.put(linageJobNamespace.key(), namespace); confs.put(lineageJobUUID.key(), listener.getRunId()); - confs.put(lineageOption.key(),"true"); + confs.put(lineageOption.key(), "true"); context.conf = Configuration.fromMap(confs); } else { context.conf = (Configuration) env.getConfiguration(); diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/LakeSoulKafkaAvroSink.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/LakeSoulKafkaAvroSink.java index edf8ac869..a15c9949d 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/LakeSoulKafkaAvroSink.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/LakeSoulKafkaAvroSink.java @@ -25,6 +25,7 @@ import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroSerializationSchema; import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; import org.apache.flink.lakesoul.entry.sql.flink.LakeSoulInAndOutputJobListener; +import org.apache.flink.lakesoul.entry.sql.utils.FileUtil; import org.apache.flink.lakesoul.source.arrow.LakeSoulArrowSource; import org.apache.flink.lakesoul.tool.JobOptions; import org.apache.flink.lakesoul.tool.LakeSoulKeyGen; @@ -147,17 +148,17 @@ public static void main(String[] args) throws Exception { conf.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); conf.set(JobOptions.transportTypeOption, "http"); conf.set(JobOptions.urlOption, lineageUrl); - conf.set(JobOptions.execAttach, true); - conf.set(lineageOption,true); + conf.set(JobOptions.execAttach, false); + conf.set(lineageOption, true); env = StreamExecutionEnvironment.getExecutionEnvironment(conf); - String appName = env.getConfiguration().get(JobOptions.KUBE_CLUSTER_ID); + String appName = FileUtil.getSubNameFromBatch(env.getConfiguration().get(JobOptions.KUBE_CLUSTER_ID)); String namespace = System.getenv("LAKESOUL_CURRENT_DOMAIN"); if (namespace == null) { namespace = "public"; } listener = new LakeSoulInAndOutputJobListener(lineageUrl); listener.jobName(appName, namespace); - listener.outputFacets("Kafka."+kafkaTopic,"kafka-public",null,null); + listener.outputFacets("Kafka." + kafkaTopic, "kafka-public", null, null); DBManager lakesoulDBManager = new DBManager(); TableInfo tableInfo = lakesoulDBManager.getTableInfoByNameAndNamespace(lakeSoulTableName, lakeSoulDBName); String tableSchema = tableInfo.getTableSchema(); @@ -170,7 +171,7 @@ public static void main(String[] args) throws Exception { colNames[i] = field.getName(); colTypes[i] = field.getType().toString(); } - listener.inputFacets("lakesoul." + lakeSoulDBName + "." + lakeSoulTableName, tableInfo.getDomain(), colNames,colTypes); + listener.inputFacets("lakesoul." + lakeSoulDBName + "." + lakeSoulTableName, tableInfo.getDomain(), colNames, colTypes); env.registerJobListener(listener); } else { env = StreamExecutionEnvironment.getExecutionEnvironment(conf); @@ -221,7 +222,7 @@ public static void main(String[] args) throws Exception { .build(); - Tuple4 keyInfo = getKeyInfo(lakeSoulDBName, + Tuple4 keyInfo = getKeyInfo(lakeSoulDBName, lakeSoulTableName, kafkaTopic, schemaRegistryUrl, props); ConfluentRegistryAvroSerializationSchema keySerialization; RowDataToAvroConverter keyRowDataToAvroConverter; @@ -229,7 +230,7 @@ public static void main(String[] args) throws Exception { FieldGetter[] keyFieldGetters; if (keyInfo != null) { keySerialization = keyInfo.f0; - keyRowDataToAvroConverter = keyInfo.f1 ; + keyRowDataToAvroConverter = keyInfo.f1; keyRowType = keyInfo.f2; keyFieldGetters = keyInfo.f3; } else { @@ -259,7 +260,7 @@ public void flatMap(LakeSoulArrowWrapper lakeSoulArrowWrapper, Collector 0) { try { diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/SyncDatabase.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/SyncDatabase.java index b49c37655..3382df6f9 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/SyncDatabase.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/SyncDatabase.java @@ -15,6 +15,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestOptions; import org.apache.flink.lakesoul.entry.sql.flink.LakeSoulInAndOutputJobListener; +import org.apache.flink.lakesoul.entry.sql.utils.FileUtil; import org.apache.flink.lakesoul.metadata.LakeSoulCatalog; import org.apache.flink.lakesoul.tool.JobOptions; import org.apache.flink.streaming.api.CheckpointingMode; @@ -69,20 +70,20 @@ public static void main(String[] args) throws Exception { targetTableName = parameter.get(TARGET_DB_TABLE_NAME.key()).toLowerCase(); url = parameter.get(TARGET_DB_URL.key()); checkpointInterval = parameter.getInt(JOB_CHECKPOINT_INTERVAL.key(), JOB_CHECKPOINT_INTERVAL.defaultValue()); - if (dbType.equals("mysql") || dbType.equals("postgresql") || dbType.equals("doris")){ + if (dbType.equals("mysql") || dbType.equals("postgresql") || dbType.equals("doris")) { for (int i = 0; i < args.length; i++) { - if ( args[i].startsWith("--D")){ + if (args[i].startsWith("--D")) { connectorOptions.append("'") .append(args[i].substring(3)) .append("'") .append("=") .append("'") - .append(args[i+1]) + .append(args[i + 1]) .append("'") .append(","); } } - if (connectorOptions.length()>0){ + if (connectorOptions.length() > 0) { jdbcOrDorisOptions = connectorOptions.substring(0, connectorOptions.length() - 1); } } @@ -100,9 +101,9 @@ public static void main(String[] args) throws Exception { conf.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); conf.set(JobOptions.transportTypeOption, "http"); conf.set(JobOptions.urlOption, lineageUrl); - conf.set(JobOptions.execAttach, true); + conf.set(JobOptions.execAttach, false); env = StreamExecutionEnvironment.getExecutionEnvironment(conf); - String appName = env.getConfiguration().get(JobOptions.KUBE_CLUSTER_ID); + String appName = FileUtil.getSubNameFromBatch(env.getConfiguration().get(JobOptions.KUBE_CLUSTER_ID)); String namespace = System.getenv("LAKESOUL_CURRENT_DOMAIN"); if (namespace == null) { namespace = "public"; @@ -224,7 +225,7 @@ public static String[] getDorisFieldTypes(DataType[] fieldTypes) { stringFieldTypes[i] = "TIMESTAMP"; } else if (fieldTypes[i].getLogicalType() instanceof VarCharType) { stringFieldTypes[i] = "VARCHAR"; - } else if (fieldTypes[i].getLogicalType() instanceof LocalZonedTimestampType ) { + } else if (fieldTypes[i].getLogicalType() instanceof LocalZonedTimestampType) { stringFieldTypes[i] = "TIMESTAMP"; } else { stringFieldTypes[i] = fieldTypes[i].toString(); @@ -300,7 +301,7 @@ public static void xsyncToPg(StreamExecutionEnvironment env) throws SQLException } } String sql; - if (jdbcOrDorisOptions==null){ + if (jdbcOrDorisOptions == null) { if (tablePk != null) { sql = String.format( "create table %s(%s ,PRIMARY KEY (%s) NOT ENFORCED) with ('connector' = '%s', 'url' = '%s', 'table-name' = '%s', 'username' = '%s', 'password' = '%s', 'sink.parallelism' = '%s')", @@ -309,7 +310,7 @@ public static void xsyncToPg(StreamExecutionEnvironment env) throws SQLException sql = String.format("create table %s(%s) with ('connector' = '%s', 'url' = '%s', 'table-name' = '%s', 'username' = '%s', 'password' = '%s', 'sink.parallelism' = '%s')", targetTableName, coulmns, "jdbc", jdbcUrl, targetTableName, username, password, sinkParallelism); } - }else { + } else { if (tablePk != null) { sql = String.format( "create table %s(%s ,PRIMARY KEY (%s) NOT ENFORCED) with ('connector' = '%s', 'url' = '%s', 'table-name' = '%s', 'username' = '%s', 'password' = '%s', 'sink.parallelism' = '%s', %s)", @@ -364,7 +365,7 @@ public static void xsyncToMysql(StreamExecutionEnvironment env) throws SQLExcept } } String sql; - if (jdbcOrDorisOptions==null){ + if (jdbcOrDorisOptions == null) { if (tablePk != null) { sql = String.format( "create table %s(%s ,PRIMARY KEY (%s) NOT ENFORCED) with ('connector' = '%s', 'url' = '%s', 'table-name' = '%s', 'username' = '%s', 'password' = '%s', 'sink.parallelism' = '%s')", @@ -373,7 +374,7 @@ public static void xsyncToMysql(StreamExecutionEnvironment env) throws SQLExcept sql = String.format("create table %s(%s) with ('connector' = '%s', 'url' = '%s', 'table-name' = '%s', 'username' = '%s', 'password' = '%s', 'sink.parallelism' = '%s')", targetTableName, coulmns, "jdbc", jdbcUrl, targetTableName, username, password, sinkParallelism); } - }else { + } else { if (tablePk != null) { sql = String.format( "create table %s(%s ,PRIMARY KEY (%s) NOT ENFORCED) with ('connector' = '%s', 'url' = '%s', 'table-name' = '%s', 'username' = '%s', 'password' = '%s', 'sink.parallelism' = '%s', %s)", @@ -409,11 +410,11 @@ public static void xsyncToDoris(StreamExecutionEnvironment env, String fenodes) String[] dorisFieldTypes = getDorisFieldTypes(fieldDataTypes); if (lineageUrl != null) { String inputName = "lakeSoul." + sourceDatabase + "." + sourceTableName; - String inputnNamespace = getTableDomain(sourceDatabase,sourceTableName); + String inputnNamespace = getTableDomain(sourceDatabase, sourceTableName); String[] inputTypes = Arrays.stream(fieldDataTypes).map(type -> type.toString()).collect(Collectors.toList()).toArray(new String[0]); - listener.inputFacets(inputName,inputnNamespace,fieldNames,inputTypes); + listener.inputFacets(inputName, inputnNamespace, fieldNames, inputTypes); String targetName = "doris." + targetDatabase + "." + targetTableName; - listener.outputFacets(targetName,"lake-public",fieldNames,dorisFieldTypes); + listener.outputFacets(targetName, "lake-public", fieldNames, dorisFieldTypes); } StringBuilder coulmns = new StringBuilder(); for (int i = 0; i < fieldDataTypes.length; i++) { @@ -423,7 +424,7 @@ public static void xsyncToDoris(StreamExecutionEnvironment env, String fenodes) } } String sql; - if (jdbcOrDorisOptions == null){ + if (jdbcOrDorisOptions == null) { sql = String.format( "create table %s(%s) with ('connector' = '%s'," + " 'jdbc-url' = '%s'," + @@ -434,7 +435,7 @@ public static void xsyncToDoris(StreamExecutionEnvironment env, String fenodes) " 'sink.properties.format' = 'json'," + " 'sink.properties.read_json_by_line' = 'true')", targetTableName, coulmns, "doris", jdbcUrl, fenodes, targetDatabase + "." + targetTableName, username, password); - }else { + } else { sql = String.format( "create table %s(%s) with ('connector' = '%s'," + " 'jdbc-url' = '%s'," + @@ -449,13 +450,13 @@ public static void xsyncToDoris(StreamExecutionEnvironment env, String fenodes) } tEnvs.executeSql(sql); - if (lineageUrl != null){ + if (lineageUrl != null) { String insertsql = "insert into " + targetTableName + " select * from lakeSoul.`" + sourceDatabase + "`." + sourceTableName; - StreamStatementSet statements = tEnvs.createStatementSet(); + StreamStatementSet statements = tEnvs.createStatementSet(); statements.addInsertSql(insertsql); statements.attachAsDataStream(); env.execute(); - }else{ + } else { tEnvs.executeSql("insert into " + targetTableName + " select * from lakeSoul.`" + sourceDatabase + "`." + sourceTableName); } diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/FlinkSqlSubmitter.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/FlinkSqlSubmitter.java index 15e40aa2c..6540ff6d3 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/FlinkSqlSubmitter.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/FlinkSqlSubmitter.java @@ -41,7 +41,7 @@ public void submit() throws Exception { if (lineageUrl != null) { conf.set(JobOptions.transportTypeOption, "http"); conf.set(JobOptions.urlOption, lineageUrl); - conf.set(JobOptions.execAttach, true); + conf.set(JobOptions.execAttach, false); } EnvironmentSettings settings = null; StreamTableEnvironment tEnv = null; @@ -63,7 +63,7 @@ public void submit() throws Exception { this.setCheckpoint(env); } if (lineageUrl != null) { - String appName = env.getConfiguration().get(JobOptions.KUBE_CLUSTER_ID); + String appName = FileUtil.getSubNameFromBatch(env.getConfiguration().get(JobOptions.KUBE_CLUSTER_ID)); String namespace = System.getenv("LAKESOUL_CURRENT_DOMAIN"); if (namespace == null) { namespace = "lake-public"; diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/LakeSoulInAndOutputJobListener.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/LakeSoulInAndOutputJobListener.java index c3d70c18a..d984d47cc 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/LakeSoulInAndOutputJobListener.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/LakeSoulInAndOutputJobListener.java @@ -136,7 +136,7 @@ public void onJobSubmitted(@Nullable JobClient jobClient, @Nullable Throwable th OpenLineage.RunEvent runStateUpdate = openLineage.newRunEventBuilder() - .eventType(OpenLineage.RunEvent.EventType.RUNNING) + .eventType(OpenLineage.RunEvent.EventType.COMPLETE) .eventTime(ZonedDateTime.now()) .run(this.run) .job(this.job) diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/utils/FileUtil.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/utils/FileUtil.java index 0aa5dde7c..ce60ad505 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/utils/FileUtil.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/utils/FileUtil.java @@ -52,4 +52,12 @@ public static String replaceDefaultKeywordFromZeppelin(String text) { String replacedStr = pc.matcher(text).replaceAll(replaceText); return replacedStr; } + public static String getSubNameFromBatch(String batchName) { + if (batchName.matches(".*-(\\d+)-(batch$)")) { + int index = batchName.lastIndexOf("-",batchName.lastIndexOf("-")-1); + return batchName.substring(0,index); + }else{ + return batchName; + } + } } From dc7abf2e14f114162c33c9ed7f902d3d59a7fa55 Mon Sep 17 00:00:00 2001 From: Xu Chen Date: Thu, 26 Sep 2024 13:46:55 +0800 Subject: [PATCH 04/33] [Flink] Fix arrow sink config (#546) * fix arrow sink config Signed-off-by: chenxu * fix ut Signed-off-by: chenxu --------- Signed-off-by: chenxu Co-authored-by: chenxu --- .../lakesoul/sink/LakeSoulMultiTableSinkStreamBuilder.java | 4 +--- .../lakesoul/test/connector/LakeSoulArrowConnectorCase.java | 3 +++ 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTableSinkStreamBuilder.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTableSinkStreamBuilder.java index 7cb06761b..4ea8d2607 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTableSinkStreamBuilder.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTableSinkStreamBuilder.java @@ -84,9 +84,7 @@ public static DataStreamSink buildArrowSink(Context contex DataStream stream, int parallelism ) { - if (!context.conf.contains(AUTO_SCHEMA_CHANGE)) { - context.conf.set(AUTO_SCHEMA_CHANGE, true); - } + context.conf.set(DYNAMIC_BUCKETING, true); LakeSoulRollingPolicyImpl rollingPolicy = new LakeSoulRollingPolicyImpl<>( context.conf.getLong(FILE_ROLLING_SIZE), context.conf.getLong(FILE_ROLLING_TIME)); OutputFileConfig fileNameConfig = OutputFileConfig.builder() diff --git a/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/connector/LakeSoulArrowConnectorCase.java b/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/connector/LakeSoulArrowConnectorCase.java index 04ebac55d..34d36e438 100644 --- a/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/connector/LakeSoulArrowConnectorCase.java +++ b/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/connector/LakeSoulArrowConnectorCase.java @@ -41,6 +41,7 @@ import java.util.Collections; import java.util.List; +import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.AUTO_SCHEMA_CHANGE; import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.FILE_ROLLING_SIZE; import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.INFERRING_SCHEMA; import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.MAX_ROW_GROUP_SIZE; @@ -60,6 +61,7 @@ public void test() throws Exception { Configuration conf = new Configuration(); conf.set(LakeSoulSinkOptions.BUCKET_PARALLELISM, parallelism); + conf.set(AUTO_SCHEMA_CHANGE, true); LakeSoulMultiTableSinkStreamBuilder.Context context = new LakeSoulMultiTableSinkStreamBuilder.Context(); context.env = execEnv; @@ -172,6 +174,7 @@ public void testManualArrowBatch() throws Exception { LakeSoulMultiTableSinkStreamBuilder.Context context = new LakeSoulMultiTableSinkStreamBuilder.Context(); context.env = env; context.conf = (Configuration) env.getConfiguration(); + context.conf.set(AUTO_SCHEMA_CHANGE, true); LakeSoulMultiTableSinkStreamBuilder.buildArrowSink(context, source); From 4cd17332b337bcfcfb8dd8b8725b95284861cf76 Mon Sep 17 00:00:00 2001 From: hw_jnj_syl <1001aabad99246419c6343979aaca886@example.com> Date: Thu, 26 Sep 2024 18:35:20 +0800 Subject: [PATCH 05/33] add scheduleTime replacer in flink sql submitter --- .../lakesoul/entry/sql/common/SubmitOption.java | 6 ++++++ .../entry/sql/flink/FlinkSqlSubmitter.java | 14 +++++++++++++- 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/common/SubmitOption.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/common/SubmitOption.java index cf9095f73..ce7bfabac 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/common/SubmitOption.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/common/SubmitOption.java @@ -14,6 +14,7 @@ public class SubmitOption { private final String jobType; private final String language; private final String sqlFilePath; + private final Long scheduleTime; private FlinkOption flinkOption; public SubmitOption(ParameterTool params) { @@ -21,12 +22,17 @@ public SubmitOption(ParameterTool params) { this.jobType = params.get("job_type"); this.language = params.get("language"); this.sqlFilePath = params.get("sql_file_path"); + this.scheduleTime = params.getLong("scheduleTime", 0L); this.checkParam(); if (SubmitType.getSubmitType(submitType) == SubmitType.FLINK) { setFlinkOption(params, this); } } + public Long getScheduleTime() { + return scheduleTime; + } + public String getSqlFilePath() { return sqlFilePath; } diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/FlinkSqlSubmitter.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/FlinkSqlSubmitter.java index 6540ff6d3..0ac4c3bdc 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/FlinkSqlSubmitter.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/FlinkSqlSubmitter.java @@ -28,6 +28,7 @@ public class FlinkSqlSubmitter extends Submitter { private static final Logger LOG = LoggerFactory.getLogger(FlinkSqlSubmitter.class); + private static final String SCHEDULE_TIME_REPLACE = "\\$\\{scheduleTime}"; public FlinkSqlSubmitter(SubmitOption submitOption) { super(submitOption); @@ -79,9 +80,16 @@ public void submit() throws Exception { tEnv = StreamTableEnvironment.create(env, settings); String sql = FileUtil.readHDFSFile(submitOption.getSqlFilePath()); - System.out.println( + LOG.info( MessageFormatter.format("\n======SQL Script Content from file {}:\n{}", submitOption.getSqlFilePath(), sql).getMessage()); + Long scheduleTime = submitOption.getScheduleTime(); + if (scheduleTime != 0) { + sql = replaceSchedulerTime(sql, scheduleTime); + } + LOG.info("scheduleTime: " + scheduleTime); + LOG.info("replace ${scheduleTime} sql: " + sql); + ExecuteSql.executeSqlFileContent(sql, tEnv, env); } @@ -104,4 +112,8 @@ private void setCheckpoint(StreamExecutionEnvironment env) { Time.of(20, TimeUnit.SECONDS) // delay )); } + + public static String replaceSchedulerTime(String sqlText, Long scheduleTime) { + return sqlText.replaceAll(SCHEDULE_TIME_REPLACE, scheduleTime.toString()); + } } From 5e71a736a720cfbd0ea783655c60d3b454ab9048 Mon Sep 17 00:00:00 2001 From: fphantam Date: Thu, 26 Sep 2024 21:08:51 +0800 Subject: [PATCH 06/33] fix clean task bug case path is null Signed-off-by: fphantam --- .../lakesoul/spark/clean/CleanOldCompaction.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanOldCompaction.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanOldCompaction.scala index 87389547e..6a28eee93 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanOldCompaction.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanOldCompaction.scala @@ -81,10 +81,12 @@ object CleanOldCompaction { }) pathSet.foreach(p => { - val path = new Path(p) - val sessionHadoopConf = spark.sessionState.newHadoopConf() - val fs = path.getFileSystem(sessionHadoopConf) - fs.delete(path, true) + if (p != null && p != "") { + val path = new Path(p) + val sessionHadoopConf = spark.sessionState.newHadoopConf() + val fs = path.getFileSystem(sessionHadoopConf) + fs.delete(path, true) + } }) } From d08f49f168639c8b96c2802d1276651ec2d79698 Mon Sep 17 00:00:00 2001 From: Ceng <441651826@qq.com> Date: Fri, 27 Sep 2024 11:49:59 +0800 Subject: [PATCH 07/33] [NativeIO/Fix] Add error info of native writer && fix case of aux_sort_cols (#547) * add error info of native writer && fix case of aux_sort_cols Signed-off-by: zenghua * fix clippy Signed-off-by: zenghua * do cargo fmt Signed-off-by: zenghua --------- Signed-off-by: zenghua Co-authored-by: zenghua --- rust/lakesoul-io/src/async_writer/mod.rs | 1 - .../src/async_writer/multipart_writer.rs | 9 +- .../src/async_writer/partitioning_writer.rs | 37 ++-- .../src/async_writer/sort_writer.rs | 8 +- rust/lakesoul-io/src/filter/parser.rs | 7 +- rust/lakesoul-io/src/helpers.rs | 37 ++-- rust/lakesoul-io/src/lakesoul_io_config.rs | 4 +- rust/lakesoul-io/src/lakesoul_writer.rs | 201 ++++++++++-------- rust/lakesoul-io/src/transform.rs | 42 ++-- 9 files changed, 178 insertions(+), 168 deletions(-) diff --git a/rust/lakesoul-io/src/async_writer/mod.rs b/rust/lakesoul-io/src/async_writer/mod.rs index bbbe8b000..cb9d164f7 100644 --- a/rust/lakesoul-io/src/async_writer/mod.rs +++ b/rust/lakesoul-io/src/async_writer/mod.rs @@ -32,7 +32,6 @@ use datafusion::{ use datafusion_common::{DataFusionError, Result}; use parquet::format::FileMetaData; - // The result of a flush operation with format (partition_desc, file_path, file_meta) pub type WriterFlushResult = Result>; diff --git a/rust/lakesoul-io/src/async_writer/multipart_writer.rs b/rust/lakesoul-io/src/async_writer/multipart_writer.rs index f249809b3..646a47f76 100644 --- a/rust/lakesoul-io/src/async_writer/multipart_writer.rs +++ b/rust/lakesoul-io/src/async_writer/multipart_writer.rs @@ -15,10 +15,13 @@ use tokio::io::{AsyncWrite, AsyncWriteExt}; use url::Url; use crate::{ - constant::TBD_PARTITION_DESC, helpers::get_batch_memory_size, lakesoul_io_config::{create_session_context, LakeSoulIOConfig}, transform::{uniform_record_batch, uniform_schema} + constant::TBD_PARTITION_DESC, + helpers::get_batch_memory_size, + lakesoul_io_config::{create_session_context, LakeSoulIOConfig}, + transform::{uniform_record_batch, uniform_schema}, }; -use super::{AsyncBatchWriter, WriterFlushResult, InMemBuf}; +use super::{AsyncBatchWriter, InMemBuf, WriterFlushResult}; /// An async writer using object_store's multi-part upload feature for cloud storage. /// This writer uses a `VecDeque` as `std::io::Write` for arrow-rs's ArrowWriter. @@ -169,7 +172,6 @@ impl MultiPartAsyncWriter { #[async_trait::async_trait] impl AsyncBatchWriter for MultiPartAsyncWriter { - async fn write_record_batch(&mut self, batch: RecordBatch) -> Result<()> { let batch = uniform_record_batch(batch)?; self.num_rows += batch.num_rows() as u64; @@ -213,5 +215,4 @@ impl AsyncBatchWriter for MultiPartAsyncWriter { fn buffered_size(&self) -> u64 { self.buffered_size } - } diff --git a/rust/lakesoul-io/src/async_writer/partitioning_writer.rs b/rust/lakesoul-io/src/async_writer/partitioning_writer.rs index 288d213ab..461e634aa 100644 --- a/rust/lakesoul-io/src/async_writer/partitioning_writer.rs +++ b/rust/lakesoul-io/src/async_writer/partitioning_writer.rs @@ -13,29 +13,26 @@ use datafusion::{ PhysicalSortExpr, }, physical_plan::{ - projection::ProjectionExec, - sorts::sort::SortExec, - stream::RecordBatchReceiverStream, - ExecutionPlan, Partitioning, PhysicalExpr, + projection::ProjectionExec, sorts::sort::SortExec, stream::RecordBatchReceiverStream, ExecutionPlan, + Partitioning, PhysicalExpr, }, }; use datafusion_common::{DataFusionError, Result}; use rand::distributions::DistString; -use tokio::{ - sync::mpsc::Sender, - task::JoinHandle, -}; +use tokio::{sync::mpsc::Sender, task::JoinHandle}; use tokio_stream::StreamExt; use tracing::debug; use crate::{ - helpers::{columnar_values_to_partition_desc, columnar_values_to_sub_path, get_batch_memory_size, get_columnar_values}, + helpers::{ + columnar_values_to_partition_desc, columnar_values_to_sub_path, get_batch_memory_size, get_columnar_values, + }, lakesoul_io_config::{create_session_context, LakeSoulIOConfig, LakeSoulIOConfigBuilder}, repartition::RepartitionByRangeAndHashExec, }; -use super::{AsyncBatchWriter, WriterFlushResult, MultiPartAsyncWriter, ReceiverStreamExec}; +use super::{AsyncBatchWriter, MultiPartAsyncWriter, ReceiverStreamExec, WriterFlushResult}; // type PartitionedWriterInfo = Arc>>>; @@ -75,7 +72,7 @@ impl PartitioningAsyncWriter { task_context.clone(), config.clone().into(), Arc::new(config.range_partitions.clone()), - write_id.clone() + write_id.clone(), )); // // In a separate task, wait for each input to be done // // (and pass along any errors, including panic!s) @@ -198,7 +195,6 @@ impl PartitioningAsyncWriter { let mut err = None; - let mut partitioned_writer = HashMap::>::new(); let mut flush_join_handle_list = Vec::new(); // let mut partitioned_flush_result_locked = partitioned_flush_result.lock().await; @@ -230,7 +226,6 @@ impl PartitioningAsyncWriter { // row_count += batch_excluding_range.num_rows(); async_writer.write_record_batch(batch_excluding_range).await?; } - } // received abort signal Err(e) => { @@ -256,19 +251,13 @@ impl PartitioningAsyncWriter { } Ok(flush_join_handle_list) } else { - for (partition_desc, writer) in partitioned_writer.into_iter() { - let flush_result = tokio::spawn(async move { - let writer_flush_results =writer.flush_and_close().await?; - Ok( - writer_flush_results.into_iter().map( - |(_, path, file_metadata)| - { - (partition_desc.clone(), path, file_metadata) - } - ).collect::>() - ) + let writer_flush_results = writer.flush_and_close().await?; + Ok(writer_flush_results + .into_iter() + .map(|(_, path, file_metadata)| (partition_desc.clone(), path, file_metadata)) + .collect::>()) }); flush_join_handle_list.push(flush_result); } diff --git a/rust/lakesoul-io/src/async_writer/sort_writer.rs b/rust/lakesoul-io/src/async_writer/sort_writer.rs index c10fa4ce1..de4d513e7 100644 --- a/rust/lakesoul-io/src/async_writer/sort_writer.rs +++ b/rust/lakesoul-io/src/async_writer/sort_writer.rs @@ -12,10 +12,8 @@ use datafusion::{ PhysicalSortExpr, }, physical_plan::{ - projection::ProjectionExec, - sorts::sort::SortExec, - stream::RecordBatchReceiverStream, - ExecutionPlan, PhysicalExpr, + projection::ProjectionExec, sorts::sort::SortExec, stream::RecordBatchReceiverStream, ExecutionPlan, + PhysicalExpr, }, }; use datafusion_common::{DataFusionError, Result}; @@ -24,7 +22,7 @@ use tokio_stream::StreamExt; use crate::{helpers::get_batch_memory_size, lakesoul_io_config::LakeSoulIOConfig}; -use super::{AsyncBatchWriter, WriterFlushResult, MultiPartAsyncWriter, ReceiverStreamExec}; +use super::{AsyncBatchWriter, MultiPartAsyncWriter, ReceiverStreamExec, WriterFlushResult}; /// Wrap the above async writer with a SortExec to /// sort the batches before write to async writer diff --git a/rust/lakesoul-io/src/filter/parser.rs b/rust/lakesoul-io/src/filter/parser.rs index 0d3369e0c..6ab559938 100644 --- a/rust/lakesoul-io/src/filter/parser.rs +++ b/rust/lakesoul-io/src/filter/parser.rs @@ -229,7 +229,6 @@ impl Parser { } pub(crate) fn parse_proto(plan: &Plan, df_schema: &DFSchema) -> Result { - let function_extension = plan .extensions .iter() @@ -733,7 +732,10 @@ fn _from_nullability(nullability: Nullability) -> bool { mod tests { use std::result::Result; - use datafusion::{logical_expr::{LogicalPlan, TableScan}, prelude::{ParquetReadOptions, SessionContext}}; + use datafusion::{ + logical_expr::{LogicalPlan, TableScan}, + prelude::{ParquetReadOptions, SessionContext}, + }; use prost::Message; use super::*; @@ -750,7 +752,6 @@ mod tests { #[tokio::test] async fn tt() { - let ctx = SessionContext::new(); let options = ParquetReadOptions::default(); let table_path = "/var/folders/_b/qyl87wbn1119cvw8kts6fqtw0000gn/T/lakeSource/type/part-00000-97db3149-f99e-404a-aa9a-2af4ab3f7a44_00000.c000.parquet"; diff --git a/rust/lakesoul-io/src/helpers.rs b/rust/lakesoul-io/src/helpers.rs index 8999a5d16..398ef72ce 100644 --- a/rust/lakesoul-io/src/helpers.rs +++ b/rust/lakesoul-io/src/helpers.rs @@ -32,9 +32,9 @@ use url::Url; use crate::{ constant::{ - DATE32_FORMAT, FLINK_TIMESTAMP_FORMAT, LAKESOUL_EMPTY_STRING, LAKESOUL_NULL_STRING, - TIMESTAMP_MICROSECOND_FORMAT, TIMESTAMP_MILLSECOND_FORMAT, TIMESTAMP_NANOSECOND_FORMAT, - TIMESTAMP_SECOND_FORMAT, LAKESOUL_COMMA, LAKESOUL_EQ + DATE32_FORMAT, FLINK_TIMESTAMP_FORMAT, LAKESOUL_COMMA, LAKESOUL_EMPTY_STRING, LAKESOUL_EQ, + LAKESOUL_NULL_STRING, TIMESTAMP_MICROSECOND_FORMAT, TIMESTAMP_MILLSECOND_FORMAT, TIMESTAMP_NANOSECOND_FORMAT, + TIMESTAMP_SECOND_FORMAT, }, filter::parser::Parser, lakesoul_io_config::LakeSoulIOConfig, @@ -169,12 +169,10 @@ pub fn format_scalar_value(v: &ScalarValue) -> String { } ScalarValue::Decimal128(Some(s), _, _) => format!("{}", s), ScalarValue::Decimal256(Some(s), _, _) => format!("{}", s), - ScalarValue::Binary(e) - | ScalarValue::FixedSizeBinary(_, e) - | ScalarValue::LargeBinary(e) => match e { - Some(bytes) => hex::encode(bytes), - None => LAKESOUL_NULL_STRING.to_string(), - } + ScalarValue::Binary(e) | ScalarValue::FixedSizeBinary(_, e) | ScalarValue::LargeBinary(e) => match e { + Some(bytes) => hex::encode(bytes), + None => LAKESOUL_NULL_STRING.to_string(), + }, other => other.to_string(), } } @@ -192,7 +190,7 @@ pub fn into_scalar_value(val: &str, data_type: &DataType) -> Result }, DataType::Decimal128(p, s) => Ok(ScalarValue::Decimal128(None, *p, *s)), DataType::Decimal256(p, s) => Ok(ScalarValue::Decimal256(None, *p, *s)), - DataType::Binary=> Ok(ScalarValue::Binary(None)), + DataType::Binary => Ok(ScalarValue::Binary(None)), DataType::FixedSizeBinary(size) => Ok(ScalarValue::FixedSizeBinary(*size, None)), DataType::LargeBinary => Ok(ScalarValue::LargeBinary(None)), _ => Ok(ScalarValue::Null), @@ -204,7 +202,9 @@ pub fn into_scalar_value(val: &str, data_type: &DataType) -> Result if val.eq(LAKESOUL_EMPTY_STRING) { Ok(ScalarValue::Utf8(Some("".to_string()))) } else { - Ok(ScalarValue::Utf8(Some(val.replace(LAKESOUL_EQ, "=").replace(LAKESOUL_COMMA, ",")))) + Ok(ScalarValue::Utf8(Some( + val.replace(LAKESOUL_EQ, "=").replace(LAKESOUL_COMMA, ","), + ))) } } DataType::Timestamp(unit, timezone) => match unit { @@ -264,7 +264,7 @@ pub fn into_scalar_value(val: &str, data_type: &DataType) -> Result }, DataType::Decimal128(p, s) => Ok(ScalarValue::Decimal128(Some(val.parse::().unwrap()), *p, *s)), DataType::Decimal256(p, s) => Ok(ScalarValue::Decimal256(Some(i256::from_string(val).unwrap()), *p, *s)), - DataType::Binary=> Ok(ScalarValue::Binary(Some(hex::decode(val).unwrap()))), + DataType::Binary => Ok(ScalarValue::Binary(Some(hex::decode(val).unwrap()))), DataType::FixedSizeBinary(size) => Ok(ScalarValue::FixedSizeBinary(*size, Some(hex::decode(val).unwrap()))), DataType::LargeBinary => Ok(ScalarValue::LargeBinary(Some(hex::decode(val).unwrap()))), _ => ScalarValue::try_from_string(val.to_string(), data_type), @@ -526,7 +526,11 @@ pub fn timestamp_str_to_unix_time(value: &str, fmt: &str) -> Result { Ok(datetime.signed_duration_since(epoch_time.naive_utc())) } -pub fn column_with_name_and_name2index<'a>(schema: &'a SchemaRef, name: &str, name_to_index: &Option>) -> Option<(usize, &'a Field)> { +pub fn column_with_name_and_name2index<'a>( + schema: &'a SchemaRef, + name: &str, + name_to_index: &Option>, +) -> Option<(usize, &'a Field)> { if let Some(name_to_index) = name_to_index { name_to_index.get(name).map(|index| (*index, schema.field(*index))) } else { @@ -535,12 +539,11 @@ pub fn column_with_name_and_name2index<'a>(schema: &'a SchemaRef, name: &str, na } pub fn get_batch_memory_size(batch: &RecordBatch) -> Result { - Ok( - batch.columns() + Ok(batch + .columns() .iter() .map(|array| array.to_data().get_slice_memory_size()) .collect::, ArrowError>>()? .into_iter() - .sum() - ) + .sum()) } diff --git a/rust/lakesoul-io/src/lakesoul_io_config.rs b/rust/lakesoul-io/src/lakesoul_io_config.rs index 1514400a7..0fe6fb07b 100644 --- a/rust/lakesoul-io/src/lakesoul_io_config.rs +++ b/rust/lakesoul-io/src/lakesoul_io_config.rs @@ -46,11 +46,9 @@ pub static OPTION_DEFAULT_VALUE_KEEP_ORDERS: &str = "false"; pub static OPTION_KEY_MEM_LIMIT: &str = "mem_limit"; pub static OPTION_KEY_POOL_SIZE: &str = "pool_size"; -pub static OPTION_KEY_HASH_BUCKET_ID : &str = "hash_bucket_id"; +pub static OPTION_KEY_HASH_BUCKET_ID: &str = "hash_bucket_id"; pub static OPTION_KEY_MAX_FILE_SIZE: &str = "max_file_size"; - - #[derive(Debug, Derivative)] #[derivative(Default, Clone)] pub struct LakeSoulIOConfig { diff --git a/rust/lakesoul-io/src/lakesoul_writer.rs b/rust/lakesoul-io/src/lakesoul_writer.rs index d77667d16..38704aee5 100644 --- a/rust/lakesoul-io/src/lakesoul_writer.rs +++ b/rust/lakesoul-io/src/lakesoul_writer.rs @@ -20,7 +20,6 @@ use crate::helpers::get_batch_memory_size; use crate::lakesoul_io_config::{IOSchema, LakeSoulIOConfig}; use crate::transform::uniform_schema; - pub type SendableWriter = Box; // inner is sort writer @@ -38,24 +37,9 @@ impl SyncSendableMutableLakeSoulWriter { pub fn try_new(config: LakeSoulIOConfig, runtime: Runtime) -> Result { let runtime = Arc::new(runtime); runtime.clone().block_on(async move { - // if aux sort cols exist, we need to adjust the schema of final writer - // to exclude all aux sort cols - let writer_schema: SchemaRef = if !config.aux_sort_cols.is_empty() { - let schema = config.target_schema.0.clone(); - // O(nm), n = number of target schema fields, m = number of aux sort cols - let proj_indices = schema - .fields - .iter() - .filter(|f| !config.aux_sort_cols.contains(f.name())) - .map(|f| schema.index_of(f.name().as_str()).map_err(DataFusionError::ArrowError)) - .collect::>>()?; - Arc::new(schema.project(proj_indices.borrow())?) - } else { - config.target_schema.0.clone() - }; let writer_config = config.clone(); let mut config = config.clone(); - let writer = Self::create_writer(writer_schema, writer_config).await?; + let writer = Self::create_writer(writer_config).await?; let schema = writer.schema(); if let Some(mem_limit) = config.mem_limit() { @@ -63,7 +47,7 @@ impl SyncSendableMutableLakeSoulWriter { config.max_file_size = Some((mem_limit as f64 * 0.15) as u64); } else if !config.primary_keys.is_empty() && !config.keep_ordering() { config.max_file_size = Some((mem_limit as f64 * 0.2) as u64); - } + } } Ok(SyncSendableMutableLakeSoulWriter { @@ -76,9 +60,25 @@ impl SyncSendableMutableLakeSoulWriter { }) } - async fn create_writer(writer_schema: SchemaRef, config: LakeSoulIOConfig) -> Result> { + async fn create_writer(config: LakeSoulIOConfig) -> Result> { + // if aux sort cols exist, we need to adjust the schema of final writer + // to exclude all aux sort cols + let writer_schema: SchemaRef = if !config.aux_sort_cols.is_empty() { + let schema = config.target_schema.0.clone(); + // O(nm), n = number of target schema fields, m = number of aux sort cols + let proj_indices = schema + .fields + .iter() + .filter(|f| !config.aux_sort_cols.contains(f.name())) + .map(|f| schema.index_of(f.name().as_str()).map_err(DataFusionError::ArrowError)) + .collect::>>()?; + Arc::new(schema.project(proj_indices.borrow())?) + } else { + config.target_schema.0.clone() + }; + let mut writer_config = config.clone(); - let writer : Box = if config.use_dynamic_partition { + let writer: Box = if config.use_dynamic_partition { Box::new(PartitioningAsyncWriter::try_new(writer_config)?) } else if !writer_config.primary_keys.is_empty() && !writer_config.keep_ordering() { // sort primary key table @@ -113,7 +113,7 @@ impl SyncSendableMutableLakeSoulWriter { pub fn schema(&self) -> SchemaRef { self.schema.clone() } - + pub fn config(&self) -> &LakeSoulIOConfig { &self.config } @@ -123,85 +123,99 @@ impl SyncSendableMutableLakeSoulWriter { // and upload concurrently in background, we only need blocking method here // for ffi callers pub fn write_batch(&mut self, record_batch: RecordBatch) -> Result<()> { - let runtime = self.runtime.clone(); - runtime.block_on(async move { - self.write_batch_async(record_batch, false).await - }) + runtime.block_on(async move { self.write_batch_async(record_batch, false).await }) } #[async_recursion::async_recursion(?Send)] async fn write_batch_async(&mut self, record_batch: RecordBatch, do_spill: bool) -> Result<()> { debug!(record_batch_row=?record_batch.num_rows(), do_spill=?do_spill, "write_batch_async"); - let schema = self.schema(); - let config = self.config().clone(); - if let Some(max_file_size) = self.config().max_file_size { - // if max_file_size is set, we need to split batch into multiple files - let in_progress_writer = match &mut self.in_progress { - Some(writer) => writer, - x => - x.insert( - Arc::new(Mutex::new( - Self::create_writer(schema, config).await? - )) - ) - }; - let mut guard = in_progress_writer.lock().await; - - let batch_memory_size = get_batch_memory_size(&record_batch)? as u64; - let batch_rows = record_batch.num_rows() as u64; - // If would exceed max_file_size, split batch - if !do_spill && guard.buffered_size() + batch_memory_size > max_file_size { - let to_write = (batch_rows * (max_file_size - guard.buffered_size())) / batch_memory_size; - if to_write + 1 < batch_rows { - let to_write = to_write as usize + 1; - let a = record_batch.slice(0, to_write); - let b = record_batch.slice(to_write, record_batch.num_rows() - to_write); - drop(guard); - self.write_batch_async(a, true).await?; - return self.write_batch_async(b, false).await; - } - } - guard.write_record_batch(record_batch).await?; - - if do_spill { - dbg!(format!("spilling writer with size: {}", guard.buffered_size())); + let config = self.config().clone(); + if let Some(max_file_size) = self.config().max_file_size { + // if max_file_size is set, we need to split batch into multiple files + let in_progress_writer = match &mut self.in_progress { + Some(writer) => writer, + x => x.insert(Arc::new(Mutex::new(Self::create_writer(config).await?))), + }; + let mut guard = in_progress_writer.lock().await; + + let batch_memory_size = get_batch_memory_size(&record_batch)? as u64; + let batch_rows = record_batch.num_rows() as u64; + // If would exceed max_file_size, split batch + if !do_spill && guard.buffered_size() + batch_memory_size > max_file_size { + let to_write = (batch_rows * (max_file_size - guard.buffered_size())) / batch_memory_size; + if to_write + 1 < batch_rows { + let to_write = to_write as usize + 1; + let a = record_batch.slice(0, to_write); + let b = record_batch.slice(to_write, record_batch.num_rows() - to_write); drop(guard); - if let Some(writer) = self.in_progress.take() { - let inner_writer = match Arc::try_unwrap(writer) { - Ok(inner) => inner, - Err(_) => { - return Err(DataFusionError::Internal("Cannot get ownership of inner writer".to_string())) - }, - }; - let writer = inner_writer.into_inner(); - let results = writer.flush_and_close().await?; - self.flush_results.extend(results); - } + self.write_batch_async(a, true).await?; + return self.write_batch_async(b, false).await; + } + } + let rb_schema = record_batch.schema(); + guard.write_record_batch(record_batch).await.map_err(|e| { + DataFusionError::Internal(format!( + "err={}, config={:?}, batch_schema={:?}", + e, + self.config.clone(), + rb_schema + )) + })?; + + if do_spill { + dbg!(format!("spilling writer with size: {}", guard.buffered_size())); + drop(guard); + if let Some(writer) = self.in_progress.take() { + let inner_writer = match Arc::try_unwrap(writer) { + Ok(inner) => inner, + Err(_) => { + return Err(DataFusionError::Internal( + "Cannot get ownership of inner writer".to_string(), + )) + } + }; + let writer = inner_writer.into_inner(); + let results = writer.flush_and_close().await.map_err(|e| { + DataFusionError::Internal(format!( + "err={}, config={:?}, batch_schema={:?}", + e, + self.config.clone(), + rb_schema + )) + })?; + self.flush_results.extend(results); } - Ok(()) - } else if let Some(inner_writer) = &self.in_progress { - let inner_writer = inner_writer.clone(); - let mut writer = inner_writer.lock().await; - writer.write_record_batch(record_batch).await - } else { - Err(DataFusionError::Internal("Invalid state of inner writer".to_string())) } - + Ok(()) + } else if let Some(inner_writer) = &self.in_progress { + let inner_writer = inner_writer.clone(); + let mut writer = inner_writer.lock().await; + writer.write_record_batch(record_batch).await + } else { + Err(DataFusionError::Internal("Invalid state of inner writer".to_string())) + } } pub fn flush_and_close(self) -> Result> { if let Some(inner_writer) = self.in_progress { let inner_writer = match Arc::try_unwrap(inner_writer) { Ok(inner) => inner, - Err(_) => return Err(DataFusionError::Internal("Cannot get ownership of inner writer".to_string())), + Err(_) => { + return Err(DataFusionError::Internal( + "Cannot get ownership of inner writer".to_string(), + )) + } }; let runtime = self.runtime; runtime.block_on(async move { let writer = inner_writer.into_inner(); - + let mut grouped_results: HashMap> = HashMap::new(); - let results = writer.flush_and_close().await?; + let results = writer + .flush_and_close() + .await + .map_err(|e| DataFusionError::Internal(format!("err={}, config={:?}", e, self.config.clone())))?; for (partition_desc, file, _) in self.flush_results.into_iter().chain(results) { match grouped_results.get_mut(&partition_desc) { Some(files) => { @@ -220,7 +234,6 @@ impl SyncSendableMutableLakeSoulWriter { summary += files.join("\x02").as_str(); } Ok(summary.into_bytes()) - }) } else { Ok(vec![]) @@ -231,7 +244,11 @@ impl SyncSendableMutableLakeSoulWriter { if let Some(inner_writer) = self.in_progress { let inner_writer = match Arc::try_unwrap(inner_writer) { Ok(inner) => inner, - Err(_) => return Err(DataFusionError::Internal("Cannot get ownership of inner writer".to_string())), + Err(_) => { + return Err(DataFusionError::Internal( + "Cannot get ownership of inner writer".to_string(), + )) + } }; let runtime = self.runtime; runtime.block_on(async move { @@ -264,9 +281,9 @@ mod tests { use datafusion::error::Result; use parquet::arrow::arrow_reader::ParquetRecordBatchReader; use rand::{distributions::DistString, Rng}; - use tracing_subscriber::layer::SubscriberExt; use std::{fs::File, sync::Arc}; use tokio::{runtime::Builder, time::Instant}; + use tracing_subscriber::layer::SubscriberExt; use super::SortAsyncWriter; @@ -546,7 +563,10 @@ mod tests { Arc::new(StringArray::from( (0..num_rows) .into_iter() - .map(|_| rand::distributions::Alphanumeric.sample_string(&mut rng, len_rng.gen_range(str_len..str_len * 3))) + .map(|_| { + rand::distributions::Alphanumeric + .sample_string(&mut rng, len_rng.gen_range(str_len..str_len * 3)) + }) .collect::>(), )) as ArrayRef, true, @@ -621,7 +641,6 @@ mod tests { #[cfg(feature = "dhat-heap")] #[global_allocator] static ALLOC: dhat::Alloc = dhat::Alloc; - #[tracing::instrument] #[test] @@ -629,10 +648,14 @@ mod tests { use tracing_subscriber::fmt; tracing_subscriber::fmt::init(); - - let subscriber = fmt::layer() - .event_format(fmt::format::Format::default().with_level(true).with_source_location(true).with_file(true)); - // .with_max_level(Level::TRACE); + + let subscriber = fmt::layer().event_format( + fmt::format::Format::default() + .with_level(true) + .with_source_location(true) + .with_file(true), + ); + // .with_max_level(Level::TRACE); tracing_subscriber::registry().with(subscriber); #[cfg(feature = "dhat-heap")] diff --git a/rust/lakesoul-io/src/transform.rs b/rust/lakesoul-io/src/transform.rs index e7a2d235f..6f8b40549 100644 --- a/rust/lakesoul-io/src/transform.rs +++ b/rust/lakesoul-io/src/transform.rs @@ -19,7 +19,9 @@ use crate::constant::{ ARROW_CAST_OPTIONS, FLINK_TIMESTAMP_FORMAT, LAKESOUL_EMPTY_STRING, LAKESOUL_NULL_STRING, TIMESTAMP_MICROSECOND_FORMAT, TIMESTAMP_MILLSECOND_FORMAT, TIMESTAMP_NANOSECOND_FORMAT, TIMESTAMP_SECOND_FORMAT, }; -use crate::helpers::{column_with_name_and_name2index, date_str_to_epoch_days, timestamp_str_to_unix_time, into_scalar_value}; +use crate::helpers::{ + column_with_name_and_name2index, date_str_to_epoch_days, into_scalar_value, timestamp_str_to_unix_time, +}; /// adjust time zone to UTC pub fn uniform_field(orig_field: &FieldRef) -> FieldRef { @@ -83,18 +85,17 @@ pub fn transform_record_batch( ) -> Result { let num_rows = batch.num_rows(); let orig_schema = batch.schema(); - let name_to_index = - if orig_schema.fields().len() > crate::constant::NUM_COLUMN_OPTIMIZE_THRESHOLD { - Some(HashMap::::from_iter( - orig_schema + let name_to_index = if orig_schema.fields().len() > crate::constant::NUM_COLUMN_OPTIMIZE_THRESHOLD { + Some(HashMap::::from_iter( + orig_schema .fields() .iter() .enumerate() - .map(|(idx, field)| (field.name().clone(), idx)) - )) - } else { - None - }; + .map(|(idx, field)| (field.name().clone(), idx)), + )) + } else { + None + }; let mut transform_arrays = Vec::new(); let mut fields = vec![]; // O(nm) n = orig_schema.fields().len(), m = target_schema.fields().len() @@ -349,18 +350,15 @@ pub fn make_default_array(datatype: &DataType, value: &String, num_rows: usize) .map_err(|e| External(Box::new(e)))?; num_rows ])), - data_type => { - match into_scalar_value(value, data_type) { - Ok(scalar) => scalar.to_array_of_size(num_rows)?, - Err(_) => { - println!( - "make_default_array() datatype not match, datatype={:?}, value={:?}", - datatype, value - ); - new_null_array(datatype, num_rows) - } + data_type => match into_scalar_value(value, data_type) { + Ok(scalar) => scalar.to_array_of_size(num_rows)?, + Err(_) => { + println!( + "make_default_array() datatype not match, datatype={:?}, value={:?}", + datatype, value + ); + new_null_array(datatype, num_rows) } - - } + }, }) } From 48e0609260b5310387e63486266a79ab4213aa3d Mon Sep 17 00:00:00 2001 From: zenghua Date: Sun, 29 Sep 2024 17:12:51 +0800 Subject: [PATCH 08/33] add LakeSoulLocalJavaWriter Signed-off-by: zenghua --- .../dmetasoul/lakesoul/meta/DBManager.java | 9 +- .../sink/writer/NativeParquetWriter.java | 6 +- .../NativeLakeSoulArrowWrapperWriter.java | 6 +- .../lakesoul/lakesoul/io/NativeIOWriter.java | 80 +++- .../local/LakeSoulLocalJavaWriter.java | 431 ++++++++++++++++++ .../local/arrow/ArrowBatchWriter.java | 160 +++++++ .../local/arrow/writers/ArrayWriter.java | 123 +++++ .../local/arrow/writers/ArrowFieldWriter.java | 86 ++++ .../local/arrow/writers/BigIntWriter.java | 74 +++ .../local/arrow/writers/BinaryWriter.java | 74 +++ .../local/arrow/writers/BooleanWriter.java | 76 +++ .../local/arrow/writers/DateWriter.java | 74 +++ .../local/arrow/writers/DecimalWriter.java | 117 +++++ .../local/arrow/writers/DoubleWriter.java | 73 +++ .../local/arrow/writers/FloatWriter.java | 74 +++ .../local/arrow/writers/IntWriter.java | 75 +++ .../local/arrow/writers/MapWriter.java | 135 ++++++ .../local/arrow/writers/NullWriter.java | 35 ++ .../local/arrow/writers/RowWriter.java | 132 ++++++ .../local/arrow/writers/SmallIntWriter.java | 73 +++ .../local/arrow/writers/TimeWriter.java | 87 ++++ .../local/arrow/writers/TimestampWriter.java | 122 +++++ .../local/arrow/writers/TinyIntWriter.java | 74 +++ .../local/arrow/writers/VarBinaryWriter.java | 74 +++ .../local/arrow/writers/VarCharWriter.java | 75 +++ rust/lakesoul-io/src/async_writer/mod.rs | 7 +- .../src/async_writer/multipart_writer.rs | 10 +- .../src/async_writer/partitioning_writer.rs | 12 +- .../src/async_writer/sort_writer.rs | 4 +- rust/lakesoul-io/src/helpers.rs | 21 + rust/lakesoul-io/src/lakesoul_writer.rs | 14 +- 31 files changed, 2374 insertions(+), 39 deletions(-) create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/LakeSoulLocalJavaWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/ArrowBatchWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/ArrayWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/ArrowFieldWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/BigIntWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/BinaryWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/BooleanWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/DateWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/DecimalWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/DoubleWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/FloatWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/IntWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/MapWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/NullWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/RowWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/SmallIntWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/TimeWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/TimestampWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/TinyIntWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/VarBinaryWriter.java create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/VarCharWriter.java diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBManager.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBManager.java index 6d6961d68..f8f613f0f 100644 --- a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBManager.java +++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBManager.java @@ -100,6 +100,12 @@ public TableInfo getTableInfoByNameAndNamespace(String tableName, String namespa public void createNewTable(String tableId, String namespace, String tableName, String tablePath, String tableSchema, JSONObject properties, String partitions) { + properties.put(DBConfig.TableInfoProperty.LAST_TABLE_SCHEMA_CHANGE_TIME, String.valueOf(System.currentTimeMillis())); + createTable(tableId, namespace, tableName, tablePath, tableSchema, properties.toJSONString(), partitions); + } + + public void createTable(String tableId, String namespace, String tableName, String tablePath, String tableSchema, + String properties, String partitions) { TableInfo.Builder tableInfo = TableInfo.newBuilder(); tableInfo.setTableId(tableId); @@ -108,8 +114,7 @@ public void createNewTable(String tableId, String namespace, String tableName, S tableInfo.setTablePath(tablePath); tableInfo.setTableSchema(tableSchema); tableInfo.setPartitions(partitions); - properties.put(DBConfig.TableInfoProperty.LAST_TABLE_SCHEMA_CHANGE_TIME, String.valueOf(System.currentTimeMillis())); - tableInfo.setProperties(properties.toJSONString()); + tableInfo.setProperties(properties); String domain = getNameSpaceDomain(namespace); diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/NativeParquetWriter.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/NativeParquetWriter.java index 9026a50c2..fbdc5563e 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/NativeParquetWriter.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/NativeParquetWriter.java @@ -230,14 +230,14 @@ public Map> closeForCommitWithRecoverableMa if (this.batch.getRowCount() > 0) { this.nativeWriter.write(this.batch); } - HashMap> partitionDescAndFilesMap = this.nativeWriter.flush(); - for (Map.Entry> entry : partitionDescAndFilesMap.entrySet()) { + HashMap> partitionDescAndFilesMap = this.nativeWriter.flush(); + for (Map.Entry> entry : partitionDescAndFilesMap.entrySet()) { String key = isDynamicBucket ? entry.getKey() : bucketID; recoverableMap.put( key, entry.getValue() .stream() - .map(path -> new NativeParquetWriter.NativeWriterPendingFileRecoverable(path, + .map(result -> new NativeParquetWriter.NativeWriterPendingFileRecoverable(result.getFilePath(), creationTime)) .collect(Collectors.toList()) ); diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/NativeLakeSoulArrowWrapperWriter.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/NativeLakeSoulArrowWrapperWriter.java index 34472045d..6ae83db76 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/NativeLakeSoulArrowWrapperWriter.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/NativeLakeSoulArrowWrapperWriter.java @@ -114,13 +114,13 @@ public Map> closeForCommitWithRecoverableMa long timer = System.currentTimeMillis(); Map> recoverableMap = new HashMap<>(); - HashMap> partitionDescAndFilesMap = this.nativeWriter.flush(); - for (Map.Entry> entry : partitionDescAndFilesMap.entrySet()) { + HashMap> partitionDescAndFilesMap = this.nativeWriter.flush(); + for (Map.Entry> entry : partitionDescAndFilesMap.entrySet()) { recoverableMap.put( entry.getKey(), entry.getValue() .stream() - .map(path -> new NativeParquetWriter.NativeWriterPendingFileRecoverable(path, creationTime)) + .map(result -> new NativeParquetWriter.NativeWriterPendingFileRecoverable(result.getFilePath(), creationTime)) .collect(Collectors.toList()) ); } diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java index 9f3f636b9..078be7bd7 100644 --- a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java @@ -4,25 +4,27 @@ package com.dmetasoul.lakesoul.lakesoul.io; -import jnr.ffi.Memory; +import com.dmetasoul.lakesoul.meta.DBUtil; +import com.dmetasoul.lakesoul.meta.entity.TableInfo; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import jnr.ffi.Pointer; import jnr.ffi.Runtime; import org.apache.arrow.c.ArrowArray; import org.apache.arrow.c.ArrowSchema; import org.apache.arrow.c.Data; -import com.dmetasoul.lakesoul.lakesoul.io.jnr.LibLakeSoulIO; +import org.apache.arrow.util.Preconditions; import org.apache.arrow.vector.VectorSchemaRoot; import org.apache.arrow.vector.ipc.ArrowStreamReader; import org.apache.arrow.vector.types.pojo.Schema; import java.io.ByteArrayInputStream; import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; +import java.util.*; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +import static com.dmetasoul.lakesoul.meta.DBConfig.TableInfoProperty.HASH_BUCKET_NUM; public class NativeIOWriter extends NativeIOBase implements AutoCloseable { @@ -33,6 +35,31 @@ public NativeIOWriter(Schema schema) { setSchema(schema); } + public NativeIOWriter(TableInfo tableInfo) { + super("NativeWriter"); + try { + setSchema(Schema.fromJSON(tableInfo.getTableSchema())); + } catch (IOException e) { + throw new RuntimeException(e); + } + DBUtil.TablePartitionKeys partitionKeys = DBUtil.parseTableInfoPartitions(tableInfo.getPartitions()); + setPrimaryKeys(partitionKeys.primaryKeys); + setRangePartitions(partitionKeys.rangeKeys); + useDynamicPartition(true); + + ObjectMapper mapper = new ObjectMapper(); + + try { + Map properties = mapper.readValue(tableInfo.getProperties(), Map.class); + setHashBucketNum(Integer.parseInt(properties.get(HASH_BUCKET_NUM).toString())); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + + withPrefix(tableInfo.getTablePath()); + + } + public void setAuxSortColumns(Iterable auxSortColumns) { for (String col : auxSortColumns) { @@ -112,7 +139,39 @@ public void write(VectorSchemaRoot batch) throws IOException { } } - public HashMap> flush() throws IOException { + public static class FlushResult { + final String filePath; + final Long fileSize; + + final String fileExistCols; + + FlushResult(String filePath, Long fileSize, String fileExistCols) { + this.filePath = filePath; + this.fileSize = fileSize; + this.fileExistCols = fileExistCols; + } + + public Long getFileSize() { + return fileSize; + } + + public String getFilePath() { + return filePath; + } + + public String getFileExistCols() { + return fileExistCols; + } + } + + public static FlushResult decodeFlushResult(String encoded) { + String[] fields = encoded.split("\u0003"); + + Preconditions.checkArgument(fields.length == 3); + return new FlushResult(fields[0], Long.parseLong(fields[1]), fields[2]); + } + + public HashMap> flush() throws IOException { AtomicReference errMsg = new AtomicReference<>(); AtomicReference lenResult = new AtomicReference<>(); IntegerCallback nativeIntegerCallback = new IntegerCallback((len, err) -> { @@ -158,11 +217,12 @@ public HashMap> flush() throws IOException { if (partitionNum != splits.length - 1) { throw new IOException("Dynamic Partitions Result [" + decodedResult + "] encode error: partition number mismatch " + partitionNum + "!=" + (splits.length - 1)); } - HashMap> partitionDescAndFilesMap = new HashMap<>(); + HashMap> partitionDescAndFilesMap = new HashMap<>(); for (int i = 1; i < splits.length; i++) { String[] partitionDescAndFiles = splits[i].split("\u0002"); List list = new ArrayList<>(Arrays.asList(partitionDescAndFiles).subList(1, partitionDescAndFiles.length)); - partitionDescAndFilesMap.put(partitionDescAndFiles[0], list); + List result = list.stream().map(NativeIOWriter::decodeFlushResult).collect(Collectors.toList()); + partitionDescAndFilesMap.put(partitionDescAndFiles[0], result); } return partitionDescAndFilesMap; diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/LakeSoulLocalJavaWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/LakeSoulLocalJavaWriter.java new file mode 100644 index 000000000..4d1da9f04 --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/LakeSoulLocalJavaWriter.java @@ -0,0 +1,431 @@ +package com.dmetasoul.lakesoul.lakesoul.local; + +import com.dmetasoul.lakesoul.lakesoul.local.arrow.ArrowBatchWriter; +import com.dmetasoul.lakesoul.lakesoul.io.NativeIOBase; +import com.dmetasoul.lakesoul.lakesoul.io.NativeIOWriter; +import com.dmetasoul.lakesoul.meta.DBConfig; +import com.dmetasoul.lakesoul.meta.DBManager; +import com.dmetasoul.lakesoul.meta.DBUtil; +import com.dmetasoul.lakesoul.meta.entity.*; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.arrow.util.Preconditions; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.types.DateUnit; +import org.apache.arrow.vector.types.TimeUnit; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.arrow.vector.types.pojo.Schema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.util.*; + +import static com.dmetasoul.lakesoul.meta.DBConfig.TableInfoProperty; + +/** + * LakeSoulLocalJavaWriter is responsible for writing LakeSoul data in a local environment. + *

+ * This class implements the AutoCloseable interface, supporting automatic resource management. + *

+ * Key functionalities include: + * 1. Initializing and configuring the LakeSoul writer + * 2. Managing Arrow batch data + * 3. Handling table information and metadata + * 4. Setting up file system and S3-related configurations + * 5. Supporting writing of various data types + *

+ * When using this class, it's important to correctly set database connection parameters, + * table information, and file system configurations. + */ +public class LakeSoulLocalJavaWriter implements AutoCloseable { + + private static final Logger LOG = LoggerFactory.getLogger(LakeSoulLocalJavaWriter.class); + + // Key for table name + public static final String TABLE_NAME = "table_name"; + // Key for PostgreSQL URL + public static final String PG_URL_KEY = DBUtil.urlKey; + // Key for username + public static final String USERNAME_KEY = DBUtil.usernameKey; + // Key for password + public static final String PWD_KEY = DBUtil.passwordKey; + + // Key for default file system + public static final String DEFAULT_FS = "fs.defaultFS"; + + // Key for S3 access key + public static final String S3_ACCESS_KEY = "s3.access-key"; + // Key for S3 secret key + public static final String S3_SECRET_KEY = "s3.secret-key"; + // Key for S3 endpoint + public static final String S3_ENDPOINT = "s3.endpoint"; + // Key for S3 path style access + public static final String S3_PATH_STYLE_ACCESS = "s3.path.style.access"; + // Key for S3 bucket + public static final String S3_BUCKET = "s3.bucket"; + + // Key for memory limit of native writer + public static final String MEM_LIMIT = "lakesoul.native_writer.mem_limit"; + + // Key for keeping orders of native writer + public static final String KEEP_ORDERS = "lakesoul.native_writer.keep_orders"; + + public static final List NATIVE_OPTION_LIST = Arrays.asList(MEM_LIMIT, KEEP_ORDERS); + + private NativeIOWriter nativeWriter; + private VectorSchemaRoot batch; + private ArrowBatchWriter arrowWriter; + private int rowsInBatch; + private long totalRows = 0; + private TableInfo tableInfo; + private DBManager dbManager; + private Map params; + + public static void setIOConfigs(Map conf, NativeIOBase io) { + + if (conf.containsKey(DEFAULT_FS)) { + setFSConf(conf, DEFAULT_FS, DEFAULT_FS, io); + } + + // try hadoop's s3 configs + setFSConf(conf, "fs.s3a.access.key", "fs.s3a.access.key", io); + setFSConf(conf, "fs.s3a.secret.key", "fs.s3a.secret.key", io); + setFSConf(conf, "fs.s3a.endpoint", "fs.s3a.endpoint", io); + setFSConf(conf, "fs.s3a.endpoint.region", "fs.s3a.endpoint.region", io); + setFSConf(conf, "fs.s3a.path.style.access", "fs.s3a.path.style.access", io); + // try flink's s3 credential configs + setFSConf(conf, S3_ACCESS_KEY, "fs.s3a.access.key", io); + setFSConf(conf, S3_SECRET_KEY, "fs.s3a.secret.key", io); + setFSConf(conf, S3_ENDPOINT, "fs.s3a.endpoint", io); + setFSConf(conf, "s3.endpoint.region", "fs.s3a.endpoint.region", io); + setFSConf(conf, S3_PATH_STYLE_ACCESS, "fs.s3a.path.style.access", io); + setFSConf(conf, S3_BUCKET, "fs.s3a.bucket", io); + + // try other native options + for (String option : NATIVE_OPTION_LIST) { + String value = conf.get(option); + if (value != null) { + int lastDot = option.lastIndexOf('.'); + String key = lastDot == -1 ? option : option.substring(lastDot + 1); + io.setOption(key, value); + } + } + } + + public static void setFSConf(Map conf, String confKey, String fsConfKey, NativeIOBase io) { + String value = conf.getOrDefault(confKey, ""); + if (!value.isEmpty()) { + LOG.info("Set native object store option {}={}", fsConfKey, value); + io.setObjectStoreOption(fsConfKey, value); + } + } + + + public void init(Map params) throws IOException { + this.params = params; + Preconditions.checkArgument(params.containsKey(PG_URL_KEY)); + Preconditions.checkArgument(params.containsKey(USERNAME_KEY)); + Preconditions.checkArgument(params.containsKey(PWD_KEY)); + System.setProperty(DBUtil.urlKey, params.get(PG_URL_KEY)); + System.setProperty(DBUtil.usernameKey, params.get(USERNAME_KEY)); + System.setProperty(DBUtil.passwordKey, params.get(PWD_KEY)); + dbManager = new DBManager(); + + tableInfo = dbManager.getTableInfoByName(params.get(TABLE_NAME)); + + initNativeWriter(); + + } + + private void initNativeWriter() throws IOException { + nativeWriter = new NativeIOWriter(tableInfo); + + Schema arrowSchema = Schema.fromJSON(tableInfo.getTableSchema()); + + batch = VectorSchemaRoot.create(arrowSchema, nativeWriter.getAllocator()); + arrowWriter = ArrowBatchWriter.createWriter(batch); + + setIOConfigs(params, nativeWriter); + nativeWriter.initializeWriter(); + } + + void write(Object[] row) { + this.arrowWriter.write(row); + this.rowsInBatch++; + this.totalRows++; + } + + + void commit() throws Exception { + this.arrowWriter.finish(); + this.nativeWriter.write(this.batch); + + List commitInfoList = new ArrayList<>(); + HashMap> partitionDescAndFilesMap = this.nativeWriter.flush(); + for (Map.Entry> entry : partitionDescAndFilesMap.entrySet()) { + commitInfoList.add(createDataCommitInfo(entry.getKey(), entry.getValue())); + } + + for (DataCommitInfo commitInfo : commitInfoList) { + dbManager.commitDataCommitInfo(commitInfo, Collections.emptyList()); + } + + recreateWriter(); + this.batch.clear(); + this.arrowWriter.reset(); + this.rowsInBatch = 0; + } + + private void recreateWriter() throws Exception { + close(); + initNativeWriter(); + } + + private DataCommitInfo createDataCommitInfo(String partitionDesc, List fileList) { + DataCommitInfo.Builder builder = DataCommitInfo.newBuilder() + .setTableId(tableInfo.getTableId()) + .setPartitionDesc(partitionDesc) + .setCommitId(DBUtil.toProtoUuid(UUID.randomUUID())) + .setCommitted(false) + .setCommitOp(CommitOp.AppendCommit) + .setTimestamp(System.currentTimeMillis()) + .setDomain("public"); + for (NativeIOWriter.FlushResult file : fileList) { + DataFileOp.Builder fileOp = DataFileOp.newBuilder() + .setFileOp(FileOp.add) + .setPath(file.getFilePath()) + .setSize(file.getFileSize()) + .setFileExistCols(file.getFileExistCols()); + builder.addFileOps(fileOp.build()); + } + + return builder.build(); + } + + + @Override + public void close() throws Exception { + if (batch != null) { + batch.close(); + batch = null; + } + if (nativeWriter != null) { + nativeWriter.close(); + nativeWriter = null; + } + } + + private static class ArrowTypeMockDataGenerator + implements ArrowType.ArrowTypeVisitor { + + long count = 0; + + static long mod = 511; + + public static final ArrowTypeMockDataGenerator INSTANCE = new ArrowTypeMockDataGenerator(); + + @Override + public Object visit(ArrowType.Null aNull) { + return null; + } + + @Override + public Object visit(ArrowType.Struct struct) { + return null; + } + + @Override + public Object visit(ArrowType.List list) { + return null; + } + + @Override + public Object visit(ArrowType.LargeList largeList) { + return null; + } + + @Override + public Object visit(ArrowType.FixedSizeList fixedSizeList) { + return null; + } + + @Override + public Object visit(ArrowType.Union union) { + return null; + } + + @Override + public Object visit(ArrowType.Map map) { + return null; + } + + @Override + public Object visit(ArrowType.Int type) { + int bitWidth = type.getBitWidth(); + long value = (count++) % mod; + if (bitWidth <= 8) return (byte) value; + if (bitWidth <= 2 * 8) return (short) value; + if (bitWidth <= 4 * 8) return (int) value; + return value; + } + + @Override + public Object visit(ArrowType.FloatingPoint type) { + double value = ((double) (count++)) / mod; + switch (type.getPrecision()) { + case HALF: + case SINGLE: + return (float) value; + } + return value; + } + + @Override + public Object visit(ArrowType.Utf8 utf8) { + return String.valueOf((count++) % mod); + } + + @Override + public Object visit(ArrowType.LargeUtf8 largeUtf8) { + return String.valueOf((count++) % mod); + } + + @Override + public Object visit(ArrowType.Binary binary) { + return String.valueOf((count++) % mod).getBytes(); + } + + @Override + public Object visit(ArrowType.LargeBinary largeBinary) { + return String.valueOf((count++) % mod).getBytes(); + } + + @Override + public Object visit(ArrowType.FixedSizeBinary fixedSizeBinary) { + return String.valueOf((count++) % mod).getBytes(); + } + + @Override + public Object visit(ArrowType.Bool bool) { + return (count++) % 2 == 0; + } + + @Override + public Object visit(ArrowType.Decimal decimal) { + return new BigDecimal(((double) (count++)) / mod).setScale(decimal.getScale(), BigDecimal.ROUND_UP); + } + + @Override + public Object visit(ArrowType.Date date) { + return (int) ((count++) % mod); + } + + @Override + public Object visit(ArrowType.Time time) { + long value = (count++) % mod; + return new Timestamp(value * 1_000_000_000); + } + + @Override + public Object visit(ArrowType.Timestamp timestamp) { + long value = (count++) % mod; + return new java.sql.Timestamp(value * 1000); // 将秒转换为毫秒 + } + + @Override + public Object visit(ArrowType.Interval interval) { + return null; + } + + @Override + public Object visit(ArrowType.Duration duration) { + return null; + } + } + + public static void main(String[] args) throws Exception { + + DBManager meta = new DBManager(); + meta.cleanMeta(); + String tableId = "table_" + UUID.randomUUID(); + List primaryKeys = Arrays.asList("id"); + List partitionKeys = Arrays.asList("range"); +// List partitionKeys = Collections.emptyList(); + String partition = DBUtil.formatTableInfoPartitionsField( + primaryKeys, + partitionKeys + ); + List fields = Arrays.asList( + new Field("id", FieldType.nullable(new ArrowType.Int(32, true)), null) + , new Field("range", FieldType.nullable(new ArrowType.Int(32, true)), null) + , new Field("int", FieldType.nullable(new ArrowType.Int(32, true)), null) + , new Field("utf8", FieldType.nullable(new ArrowType.Utf8()), null) + , new Field("decimal", FieldType.nullable(ArrowType.Decimal.createDecimal(10, 3, null)), null) + , new Field("boolean", FieldType.nullable(new ArrowType.Bool()), null) + , new Field("date", FieldType.nullable(new ArrowType.Date(DateUnit.DAY)), null) + , new Field("datetimeSec", FieldType.nullable(new ArrowType.Timestamp(TimeUnit.SECOND, ZoneId.of("UTC").toString())), null) + , new Field("datetimeMilli", FieldType.nullable(new ArrowType.Timestamp(TimeUnit.MILLISECOND, ZoneId.of("UTC").toString())), null) + + ); + Schema schema = new Schema(fields); + + ObjectMapper objectMapper = new ObjectMapper(); + + Map properties = new HashMap<>(); + if (!primaryKeys.isEmpty()) { + properties.put(TableInfoProperty.HASH_BUCKET_NUM, "4"); + properties.put("hashPartitions", + String.join(DBConfig.LAKESOUL_HASH_PARTITION_SPLITTER, primaryKeys)); + } + + meta.createTable( + tableId, + "default", + "test_local_java_table", + "file:/tmp/test_local_java_table", + schema.toJson(), + objectMapper.writeValueAsString(properties), + partition); + + LakeSoulLocalJavaWriter localJavaWriter = new LakeSoulLocalJavaWriter(); + + Map params = new HashMap<>(); + params.put("lakesoul.pg.url", "jdbc:postgresql://127.0.0.1:5433/test_lakesoul_meta?stringtype=unspecified"); + params.put("lakesoul.pg.username", "yugabyte"); + params.put("lakesoul.pg.password", "yugabyte"); + params.put(TABLE_NAME, "test_local_java_table"); + + localJavaWriter.init(params); + int times = 8; + int ranges = 13; + for (int t = 0; t < times; t++) { + int numRows = 1024; + int numCols = fields.size(); + for (int i = 0; i < numRows; i++) { + Object[] row = new Object[numCols]; + for (int j = 0; j < numCols; j++) { + if (fields.get(j).getName().contains("range")) { + row[j] = i % ranges; + } else { + row[j] = fields.get(j).getType().accept(ArrowTypeMockDataGenerator.INSTANCE); + } + } + localJavaWriter.write(row); + } + localJavaWriter.commit(); + } + + + assert meta.getAllPartitionInfo(tableId).size() == times; + + System.out.println("data commit DONE"); + localJavaWriter.close(); + } +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/ArrowBatchWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/ArrowBatchWriter.java new file mode 100644 index 000000000..9901fc563 --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/ArrowBatchWriter.java @@ -0,0 +1,160 @@ +package com.dmetasoul.lakesoul.lakesoul.local.arrow; + +import com.dmetasoul.lakesoul.lakesoul.local.arrow.writers.*; +import org.apache.arrow.util.Preconditions; +import org.apache.arrow.vector.*; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; + +import java.util.List; + +import static com.dmetasoul.lakesoul.lakesoul.local.arrow.writers.DecimalWriter.getPrecision; + +public class ArrowBatchWriter { + + /** + * Container that holds a set of vectors for the rows to be sent to the Python worker. + */ + private final VectorSchemaRoot root; + + /** + * An array of writers which are responsible for the serialization of each column of the rows. + */ + private final ArrowFieldWriter[] fieldWriters; + + public ArrowBatchWriter(VectorSchemaRoot root, ArrowFieldWriter[] fieldWriters) { + this.root = Preconditions.checkNotNull(root); + this.fieldWriters = Preconditions.checkNotNull(fieldWriters); + } + + /** + * Gets the field writers. + */ + public ArrowFieldWriter[] getFieldWriters() { + return fieldWriters; + } + + /** + * Writes the specified row which is serialized into Arrow format. + */ + public void write(IN row) { + for (int i = 0; i < fieldWriters.length; i++) { + fieldWriters[i].write(row, i); + } + } + + /** + * Finishes the writing of the current row batch. + */ + public void finish() { + root.setRowCount(fieldWriters[0].getCount()); + for (ArrowFieldWriter fieldWriter : fieldWriters) { + fieldWriter.finish(); + } + } + + /** + * Resets the state of the writer to write the next batch of rows. + */ + public void reset() { + root.setRowCount(0); + for (ArrowFieldWriter fieldWriter : fieldWriters) { + fieldWriter.reset(); + } + } + + public static ArrowBatchWriter createWriter(VectorSchemaRoot root) { + ArrowFieldWriter[] fieldWriters = + new ArrowFieldWriter[root.getFieldVectors().size()]; + List vectors = root.getFieldVectors(); + for (int i = 0; i < vectors.size(); i++) { + FieldVector vector = vectors.get(i); + vector.allocateNew(); + fieldWriters[i] = createArrowFieldWriterForObject(vector, vector.getField()); + } + + return new ArrowBatchWriter<>(root, fieldWriters); + } + + private static ArrowFieldWriter createArrowFieldWriterForObject( + ValueVector vector, Field field) { + if (vector instanceof TinyIntVector) { + return TinyIntWriter.forObject((TinyIntVector) vector); + } else if (vector instanceof SmallIntVector) { + return SmallIntWriter.forObject((SmallIntVector) vector); + } else if (vector instanceof IntVector) { + return IntWriter.forObject((IntVector) vector); + } else if (vector instanceof BigIntVector) { + return BigIntWriter.forObject((BigIntVector) vector); + } else if (vector instanceof BitVector) { + return BooleanWriter.forObject((BitVector) vector); + } else if (vector instanceof Float4Vector) { + return FloatWriter.forObject((Float4Vector) vector); + } else if (vector instanceof Float8Vector) { + return DoubleWriter.forObject((Float8Vector) vector); + } else if (vector instanceof VarCharVector) { + return VarCharWriter.forObject((VarCharVector) vector); + } else if (vector instanceof FixedSizeBinaryVector) { + return BinaryWriter.forObject((FixedSizeBinaryVector) vector); + } else if (vector instanceof VarBinaryVector) { + return VarBinaryWriter.forObject((VarBinaryVector) vector); + } else if (vector instanceof DecimalVector) { + DecimalVector decimalVector = (DecimalVector) vector; + return DecimalWriter.forObject( + decimalVector, getPrecision(decimalVector), decimalVector.getScale()); + } else if (vector instanceof DateDayVector) { + return DateWriter.forObject((DateDayVector) vector); + } else if (vector instanceof TimeSecVector + || vector instanceof TimeMilliVector + || vector instanceof TimeMicroVector + || vector instanceof TimeNanoVector) { + return TimeWriter.forObject(vector); + } else if (vector instanceof TimeStampVector) { + int precision = 0; + switch (((ArrowType.Timestamp) field.getType()).getUnit()) { + case MILLISECOND: + precision = 3; + break; + case MICROSECOND: + precision = 6; + break; + case NANOSECOND: + precision = 9; + break; + } + return TimestampWriter.forObject(vector, precision); +// } else if (vector instanceof MapVector) { +// MapVector mapVector = (MapVector) vector; +// LogicalType keyType = ((MapType) field).getKeyType(); +// LogicalType valueType = ((MapType) field).getValueType(); +// StructVector structVector = (StructVector) mapVector.getDataVector(); +// return MapWriter.forObject( +// mapVector, +// createArrowFieldWriterForArray( +// structVector.getChild(MapVector.KEY_NAME), keyType), +// createArrowFieldWriterForArray( +// structVector.getChild(MapVector.VALUE_NAME), valueType)); +// } else if (vector instanceof ListVector) { +// ListVector listVector = (ListVector) vector; +// LogicalType elementType = ((ArrayType) field).getElementType(); +// return ArrayWriter.forObject( +// listVector, +// createArrowFieldWriterForArray(listVector.getDataVector(), elementType)); +// } else if (vector instanceof StructVector) { +// RowType rowType = (RowType) field; +// ArrowFieldWriter[] fieldsWriters = +// new ArrowFieldWriter[rowType.getFieldCount()]; +// for (int i = 0; i < fieldsWriters.length; i++) { +// fieldsWriters[i] = +// createArrowFieldWriterforObject( +// ((StructVector) vector).getVectorById(i), rowType.getTypeAt(i)); +// } +// return RowWriter.forObject((StructVector) vector, fieldsWriters); + } else if (vector instanceof NullVector) { + return new NullWriter<>((NullVector) vector); + } else { + throw new UnsupportedOperationException( + String.format("Unsupported type %s.", field)); + } + } +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/ArrayWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/ArrayWriter.java new file mode 100644 index 000000000..52a2115f2 --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/ArrayWriter.java @@ -0,0 +1,123 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; +// +//import org.apache.arrow.vector.complex.ListVector; +//import org.apache.flink.annotation.Internal; +//import org.apache.flink.table.data.ArrayData; +//import org.apache.flink.table.data.RowData; +//import org.apache.flink.util.Preconditions; +// +///** +// * {@link ArrowFieldWriter} for Array. +// */ +//@Internal +//public abstract class ArrayWriter extends ArrowFieldWriter { +// +// public static ArrayWriter forRow( +// ListVector listVector, ArrowFieldWriter elementWriter) { +// return new ArrayWriterForRow(listVector, elementWriter); +// } +// +// public static ArrayWriter forArray( +// ListVector listVector, ArrowFieldWriter elementWriter) { +// return new ArrayWriterForArray(listVector, elementWriter); +// } +// +// // ------------------------------------------------------------------------------------------ +// +// private final ArrowFieldWriter elementWriter; +// +// private ArrayWriter(ListVector listVector, ArrowFieldWriter elementWriter) { +// super(listVector); +// this.elementWriter = Preconditions.checkNotNull(elementWriter); +// } +// +// abstract boolean isNullAt(T in, int ordinal); +// +// abstract ArrayData readArray(T in, int ordinal); +// +// @Override +// public void doWrite(T in, int ordinal) { +// if (!isNullAt(in, ordinal)) { +// ((ListVector) getValueVector()).startNewValue(getCount()); +// ArrayData array = readArray(in, ordinal); +// for (int i = 0; i < array.size(); i++) { +// elementWriter.write(array, i); +// } +// ((ListVector) getValueVector()).endValue(getCount(), array.size()); +// } +// } +// +// @Override +// public void finish() { +// super.finish(); +// elementWriter.finish(); +// } +// +// @Override +// public void reset() { +// super.reset(); +// elementWriter.reset(); +// } +// +// // ------------------------------------------------------------------------------------------ +// +// /** +// * {@link ArrayWriter} for {@link RowData} input. +// */ +// public static final class ArrayWriterForRow extends ArrayWriter { +// +// private ArrayWriterForRow( +// ListVector listVector, ArrowFieldWriter elementWriter) { +// super(listVector, elementWriter); +// } +// +// @Override +// boolean isNullAt(RowData in, int ordinal) { +// return in.isNullAt(ordinal); +// } +// +// @Override +// ArrayData readArray(RowData in, int ordinal) { +// return in.getArray(ordinal); +// } +// } +// +// /** +// * {@link ArrayWriter} for {@link ArrayData} input. +// */ +// public static final class ArrayWriterForArray extends ArrayWriter { +// +// private ArrayWriterForArray( +// ListVector listVector, ArrowFieldWriter elementWriter) { +// super(listVector, elementWriter); +// } +// +// @Override +// boolean isNullAt(ArrayData in, int ordinal) { +// return in.isNullAt(ordinal); +// } +// +// @Override +// ArrayData readArray(ArrayData in, int ordinal) { +// return in.getArray(ordinal); +// } +// } +//} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/ArrowFieldWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/ArrowFieldWriter.java new file mode 100644 index 000000000..3b6bf15f6 --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/ArrowFieldWriter.java @@ -0,0 +1,86 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.util.Preconditions; +import org.apache.arrow.vector.ValueVector; + +/** + * Base class for arrow field writer which is used to convert a field to an Arrow format. + * + * @param Type of the row to write. + */ +public abstract class ArrowFieldWriter { + + /** + * Container which is used to store the written sequence of values of a column. + */ + private final ValueVector valueVector; + + /** + * The current count of elements written. + */ + private int count = 0; + + public ArrowFieldWriter(ValueVector valueVector) { + this.valueVector = Preconditions.checkNotNull(valueVector); + } + + /** + * Returns the underlying container which stores the sequence of values of a column. + */ + public ValueVector getValueVector() { + return valueVector; + } + + /** + * Returns the current count of elements written. + */ + public int getCount() { + return count; + } + + /** + * Sets the field value as the field at the specified ordinal of the specified rwow. + */ + public abstract void doWrite(IN row, int ordinal); + + /** + * Writes the specified ordinal of the specified row. + */ + public void write(IN row, int ordinal) { + doWrite(row, ordinal); + count += 1; + } + + /** + * Finishes the writing of the current row batch. + */ + public void finish() { + valueVector.setValueCount(count); + } + + /** + * Resets the state of the writer to write the next batch of fields. + */ + public void reset() { + valueVector.reset(); + count = 0; + } +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/BigIntWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/BigIntWriter.java new file mode 100644 index 000000000..d87b5d01d --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/BigIntWriter.java @@ -0,0 +1,74 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.vector.BigIntVector; + +/** + * {@link ArrowFieldWriter} for BigInt. + */ +public abstract class BigIntWriter extends ArrowFieldWriter { + + public static BigIntWriter forObject(BigIntVector bigIntVector) { + return new BigIntWriterforObject(bigIntVector); + } + + + // ------------------------------------------------------------------------------------------ + + private BigIntWriter(BigIntVector bigIntVector) { + super(bigIntVector); + } + + abstract boolean isNullAt(T in, int ordinal); + + abstract long readLong(T in, int ordinal); + + @Override + public void doWrite(T in, int ordinal) { + if (isNullAt(in, ordinal)) { + ((BigIntVector) getValueVector()).setNull(getCount()); + } else { + ((BigIntVector) getValueVector()).setSafe(getCount(), readLong(in, ordinal)); + } + } + + // ------------------------------------------------------------------------------------------ + + /** + * {@link BigIntWriter} for {@link Object[]} input. + */ + public static final class BigIntWriterforObject extends BigIntWriter { + + private BigIntWriterforObject(BigIntVector bigIntVector) { + super(bigIntVector); + } + + @Override + boolean isNullAt(Object[] in, int ordinal) { + return in[ordinal] == null; + } + + @Override + long readLong(Object[] in, int ordinal) { + return (long) in[ordinal]; + } + } + +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/BinaryWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/BinaryWriter.java new file mode 100644 index 000000000..6c0b56323 --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/BinaryWriter.java @@ -0,0 +1,74 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.vector.FixedSizeBinaryVector; + +/** + * {@link ArrowFieldWriter} for Binary. + */ +public abstract class BinaryWriter extends ArrowFieldWriter { + + public static BinaryWriter forObject(FixedSizeBinaryVector fixedSizeBinaryVector) { + return new BinaryWriterForObject(fixedSizeBinaryVector); + } + + + // ------------------------------------------------------------------------------------------ + + private BinaryWriter(FixedSizeBinaryVector fixedSizeBinaryVector) { + super(fixedSizeBinaryVector); + } + + abstract boolean isNullAt(T in, int ordinal); + + abstract byte[] readBinary(T in, int ordinal); + + @Override + public void doWrite(T in, int ordinal) { + if (isNullAt(in, ordinal)) { + ((FixedSizeBinaryVector) getValueVector()).setNull(getCount()); + } else { + ((FixedSizeBinaryVector) getValueVector()).setSafe(getCount(), readBinary(in, ordinal)); + } + } + + // ------------------------------------------------------------------------------------------ + + /** + * {@link BinaryWriter} for {@link Object[]} input. + */ + public static final class BinaryWriterForObject extends BinaryWriter { + + private BinaryWriterForObject(FixedSizeBinaryVector fixedSizeBinaryVector) { + super(fixedSizeBinaryVector); + } + + @Override + boolean isNullAt(Object[] in, int ordinal) { + return in[ordinal] == null; + } + + @Override + byte[] readBinary(Object[] in, int ordinal) { + return (byte[]) in[ordinal]; + } + } + +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/BooleanWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/BooleanWriter.java new file mode 100644 index 000000000..676174833 --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/BooleanWriter.java @@ -0,0 +1,76 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.vector.BitVector; + +/** + * {@link ArrowFieldWriter} for Boolean. + */ +public abstract class BooleanWriter extends ArrowFieldWriter { + + public static BooleanWriter forObject(BitVector bitVector) { + return new BooleanWriterForObject(bitVector); + } + + + // ------------------------------------------------------------------------------------------ + + private BooleanWriter(BitVector bitVector) { + super(bitVector); + } + + abstract boolean isNullAt(T in, int ordinal); + + abstract boolean readBoolean(T in, int ordinal); + + @Override + public void doWrite(T in, int ordinal) { + if (isNullAt(in, ordinal)) { + ((BitVector) getValueVector()).setNull(getCount()); + } else if (readBoolean(in, ordinal)) { + ((BitVector) getValueVector()).setSafe(getCount(), 1); + } else { + ((BitVector) getValueVector()).setSafe(getCount(), 0); + } + } + + // ------------------------------------------------------------------------------------------ + + /** + * {@link BooleanWriter} for {@link Object[]} input. + */ + public static final class BooleanWriterForObject extends BooleanWriter { + + private BooleanWriterForObject(BitVector bitVector) { + super(bitVector); + } + + @Override + boolean isNullAt(Object[] in, int ordinal) { + return in[ordinal] == null; + } + + @Override + boolean readBoolean(Object[] in, int ordinal) { + return (boolean) in[ordinal]; + } + } + +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/DateWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/DateWriter.java new file mode 100644 index 000000000..f82e49b1a --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/DateWriter.java @@ -0,0 +1,74 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.vector.DateDayVector; + +/** + * {@link ArrowFieldWriter} for Date. + */ +public abstract class DateWriter extends ArrowFieldWriter { + + public static DateWriter forObject(DateDayVector dateDayVector) { + return new DateWriterForObject(dateDayVector); + } + + + // ------------------------------------------------------------------------------------------ + + private DateWriter(DateDayVector dateDayVector) { + super(dateDayVector); + } + + abstract boolean isNullAt(T in, int ordinal); + + abstract int readDate(T in, int ordinal); + + @Override + public void doWrite(T in, int ordinal) { + if (isNullAt(in, ordinal)) { + ((DateDayVector) getValueVector()).setNull(getCount()); + } else { + ((DateDayVector) getValueVector()).setSafe(getCount(), readDate(in, ordinal)); + } + } + + // ------------------------------------------------------------------------------------------ + + /** + * {@link DateWriter} for {@link Object[]} input. + */ + public static final class DateWriterForObject extends DateWriter { + + private DateWriterForObject(DateDayVector dateDayVector) { + super(dateDayVector); + } + + @Override + boolean isNullAt(Object[] in, int ordinal) { + return in[ordinal] == null; + } + + @Override + int readDate(Object[] in, int ordinal) { + return (int) in[ordinal]; + } + } + +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/DecimalWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/DecimalWriter.java new file mode 100644 index 000000000..4326d206e --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/DecimalWriter.java @@ -0,0 +1,117 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.vector.DecimalVector; + +import java.math.BigDecimal; +import java.math.RoundingMode; + +/** + * {@link ArrowFieldWriter} for Decimal. + */ +public abstract class DecimalWriter extends ArrowFieldWriter { + + public static DecimalWriter forObject( + DecimalVector decimalVector, int precision, int scale) { + return new DecimalWriterForObject(decimalVector, precision, scale); + } + + // ------------------------------------------------------------------------------------------ + + protected final int precision; + protected final int scale; + + private DecimalWriter(DecimalVector decimalVector, int precision, int scale) { + super(decimalVector); + this.precision = precision; + this.scale = scale; + } + + abstract boolean isNullAt(T in, int ordinal); + + abstract BigDecimal readDecimal(T in, int ordinal); + + @Override + public void doWrite(T in, int ordinal) { + if (isNullAt(in, ordinal)) { + ((DecimalVector) getValueVector()).setNull(getCount()); + } else { + BigDecimal bigDecimal = readDecimal(in, ordinal); + bigDecimal = fromBigDecimal(bigDecimal, precision, scale); + if (bigDecimal == null) { + ((DecimalVector) getValueVector()).setNull(getCount()); + } else { + ((DecimalVector) getValueVector()).setSafe(getCount(), bigDecimal); + } + } + } + + // ------------------------------------------------------------------------------------------ + + /** + * {@link DecimalWriter} for {@link Object[]} input. + */ + public static final class DecimalWriterForObject extends DecimalWriter { + + private DecimalWriterForObject(DecimalVector decimalVector, int precision, int scale) { + super(decimalVector, precision, scale); + } + + @Override + boolean isNullAt(Object[] in, int ordinal) { + return in[ordinal] == null; + } + + @Override + BigDecimal readDecimal(Object[] in, int ordinal) { + return (BigDecimal) in[ordinal]; + } + } + + /** + * Convert the specified bigDecimal according to the specified precision and scale. The + * specified bigDecimal may be rounded to have the specified scale and then the specified + * precision is checked. If precision overflow, it will return `null`. + */ + public static BigDecimal fromBigDecimal(BigDecimal bigDecimal, int precision, int scale) { + if (bigDecimal.scale() != scale || bigDecimal.precision() > precision) { + // need adjust the precision and scale + bigDecimal = bigDecimal.setScale(scale, RoundingMode.HALF_UP); + if (bigDecimal.precision() > precision) { + return null; + } + } + return bigDecimal; + } + + public static int getPrecision(DecimalVector decimalVector) { + int precision = -1; + try { + java.lang.reflect.Field precisionField = + decimalVector.getClass().getDeclaredField("precision"); + precisionField.setAccessible(true); + precision = (int) precisionField.get(decimalVector); + } catch (NoSuchFieldException | IllegalAccessException e) { + // should not happen, ignore + } + return precision; + } + +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/DoubleWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/DoubleWriter.java new file mode 100644 index 000000000..486d822cf --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/DoubleWriter.java @@ -0,0 +1,73 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.vector.Float8Vector; + +/** + * {@link ArrowFieldWriter} for Double. + */ +public abstract class DoubleWriter extends ArrowFieldWriter { + + public static DoubleWriter forObject(Float8Vector doubleVector) { + return new DoubleWriterForObject(doubleVector); + } + + // ------------------------------------------------------------------------------------------ + + private DoubleWriter(Float8Vector doubleVector) { + super(doubleVector); + } + + abstract boolean isNullAt(T in, int ordinal); + + abstract double readDouble(T in, int ordinal); + + @Override + public void doWrite(T in, int ordinal) { + if (isNullAt(in, ordinal)) { + ((Float8Vector) getValueVector()).setNull(getCount()); + } else { + ((Float8Vector) getValueVector()).setSafe(getCount(), readDouble(in, ordinal)); + } + } + + // ------------------------------------------------------------------------------------------ + + /** + * {@link DoubleWriter} for {@link Object[]} input. + */ + public static final class DoubleWriterForObject extends DoubleWriter { + + private DoubleWriterForObject(Float8Vector doubleVector) { + super(doubleVector); + } + + @Override + boolean isNullAt(Object[] in, int ordinal) { + return in[ordinal] == null; + } + + @Override + double readDouble(Object[] in, int ordinal) { + return (double) in[ordinal]; + } + } + +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/FloatWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/FloatWriter.java new file mode 100644 index 000000000..1fb33b552 --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/FloatWriter.java @@ -0,0 +1,74 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.vector.Float4Vector; + +/** + * {@link ArrowFieldWriter} for Float. + */ +public abstract class FloatWriter extends ArrowFieldWriter { + + public static FloatWriter forObject(Float4Vector floatVector) { + return new FloatWriterForObject(floatVector); + } + + + // ------------------------------------------------------------------------------------------ + + private FloatWriter(Float4Vector floatVector) { + super(floatVector); + } + + abstract boolean isNullAt(T in, int ordinal); + + abstract float readFloat(T in, int ordinal); + + @Override + public void doWrite(T in, int ordinal) { + if (isNullAt(in, ordinal)) { + ((Float4Vector) getValueVector()).setNull(getCount()); + } else { + ((Float4Vector) getValueVector()).setSafe(getCount(), readFloat(in, ordinal)); + } + } + + // ------------------------------------------------------------------------------------------ + + /** + * {@link FloatWriter} for {@link Object[]} input. + */ + public static final class FloatWriterForObject extends FloatWriter { + + private FloatWriterForObject(Float4Vector floatVector) { + super(floatVector); + } + + @Override + boolean isNullAt(Object[] in, int ordinal) { + return in[ordinal] == null; + } + + @Override + float readFloat(Object[] in, int ordinal) { + return (float) in[ordinal]; + } + } + +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/IntWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/IntWriter.java new file mode 100644 index 000000000..fbedd2257 --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/IntWriter.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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.vector.IntVector; + +/** + * {@link ArrowFieldWriter} for Int. + */ +public abstract class IntWriter extends ArrowFieldWriter { + + public static IntWriter forObject(IntVector intVector) { + return new IntWriterforObject(intVector); + } + + + // ------------------------------------------------------------------------------------------ + + private IntWriter(IntVector intVector) { + super(intVector); + } + + abstract boolean isNullAt(T in, int ordinal); + + abstract int readInt(T in, int ordinal); + + @Override + public void doWrite(T in, int ordinal) { + if (isNullAt(in, ordinal)) { + ((IntVector) getValueVector()).setNull(getCount()); + } else { + ((IntVector) getValueVector()).setSafe(getCount(), readInt(in, ordinal)); + } + } + + // ------------------------------------------------------------------------------------------ + + /** + * {@link IntWriter} for {@link Object[]} input. + */ + public static final class IntWriterforObject extends IntWriter { + + private IntWriterforObject(IntVector intVector) { + super(intVector); + } + + @Override + boolean isNullAt(Object[] in, int ordinal) { + return in[ordinal] == null; + } + + @Override + int readInt(Object[] in, int ordinal) { + return (int) in[ordinal]; + } + } + + +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/MapWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/MapWriter.java new file mode 100644 index 000000000..ba709caf7 --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/MapWriter.java @@ -0,0 +1,135 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; +// +//import org.apache.arrow.vector.complex.MapVector; +//import org.apache.arrow.vector.complex.StructVector; +//import org.apache.flink.annotation.Internal; +//import org.apache.flink.table.data.ArrayData; +//import org.apache.flink.table.data.MapData; +//import org.apache.flink.table.data.RowData; +//import org.apache.flink.util.Preconditions; +// +///** +// * {@link ArrowFieldWriter} for Map. +// */ +//@Internal +//public abstract class MapWriter extends ArrowFieldWriter { +// +// public static MapWriter forRow( +// MapVector mapVector, +// ArrowFieldWriter keyWriter, +// ArrowFieldWriter valueWriter) { +// return new MapWriterForRow(mapVector, keyWriter, valueWriter); +// } +// +// public static MapWriter forArray( +// MapVector mapVector, +// ArrowFieldWriter keyWriter, +// ArrowFieldWriter valueWriter) { +// return new MapWriterForArray(mapVector, keyWriter, valueWriter); +// } +// +// // ------------------------------------------------------------------------------------------ +// +// private final ArrowFieldWriter keyWriter; +// +// private final ArrowFieldWriter valueWriter; +// +// private MapWriter( +// MapVector mapVector, +// ArrowFieldWriter keyWriter, +// ArrowFieldWriter valueWriter) { +// super(mapVector); +// this.keyWriter = Preconditions.checkNotNull(keyWriter); +// this.valueWriter = Preconditions.checkNotNull(valueWriter); +// } +// +// abstract boolean isNullAt(T in, int ordinal); +// +// abstract MapData readMap(T in, int ordinal); +// +// @Override +// public void doWrite(T in, int ordinal) { +// if (!isNullAt(in, ordinal)) { +// ((MapVector) getValueVector()).startNewValue(getCount()); +// +// StructVector structVector = +// (StructVector) ((MapVector) getValueVector()).getDataVector(); +// MapData map = readMap(in, ordinal); +// ArrayData keys = map.keyArray(); +// ArrayData values = map.valueArray(); +// for (int i = 0; i < map.size(); i++) { +// structVector.setIndexDefined(keyWriter.getCount()); +// keyWriter.write(keys, i); +// valueWriter.write(values, i); +// } +// +// ((MapVector) getValueVector()).endValue(getCount(), map.size()); +// } +// } +// +// // ------------------------------------------------------------------------------------------ +// +// /** +// * {@link MapWriter} for {@link RowData} input. +// */ +// public static final class MapWriterForRow extends MapWriter { +// +// private MapWriterForRow( +// MapVector mapVector, +// ArrowFieldWriter keyWriter, +// ArrowFieldWriter valueWriter) { +// super(mapVector, keyWriter, valueWriter); +// } +// +// @Override +// boolean isNullAt(RowData in, int ordinal) { +// return in.isNullAt(ordinal); +// } +// +// @Override +// MapData readMap(RowData in, int ordinal) { +// return in.getMap(ordinal); +// } +// } +// +// /** +// * {@link MapWriter} for {@link ArrayData} input. +// */ +// public static final class MapWriterForArray extends MapWriter { +// +// private MapWriterForArray( +// MapVector mapVector, +// ArrowFieldWriter keyWriter, +// ArrowFieldWriter valueWriter) { +// super(mapVector, keyWriter, valueWriter); +// } +// +// @Override +// boolean isNullAt(ArrayData in, int ordinal) { +// return in.isNullAt(ordinal); +// } +// +// @Override +// MapData readMap(ArrayData in, int ordinal) { +// return in.getMap(ordinal); +// } +// } +//} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/NullWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/NullWriter.java new file mode 100644 index 000000000..f71ef44c9 --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/NullWriter.java @@ -0,0 +1,35 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.vector.NullVector; + +/** + * {@link ArrowFieldWriter} for Null. + */ +public class NullWriter extends ArrowFieldWriter { + + public NullWriter(NullVector nullVector) { + super(nullVector); + } + + @Override + public void doWrite(T in, int ordinal) { + } +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/RowWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/RowWriter.java new file mode 100644 index 000000000..b345679ae --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/RowWriter.java @@ -0,0 +1,132 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +//import org.apache.arrow.vector.complex.StructVector; +//import org.apache.flink.annotation.Internal; +//import org.apache.flink.table.data.ArrayData; +//import org.apache.flink.table.data.GenericRowData; +//import org.apache.flink.table.data.RowData; +// +///** +// * {@link ArrowFieldWriter} for Row. +// */ +//@Internal +//public abstract class RowWriter extends ArrowFieldWriter { +// +// public static RowWriter forRow( +// StructVector structVector, ArrowFieldWriter[] fieldsWriters) { +// return new RowWriterForRow(structVector, fieldsWriters); +// } +// +// public static RowWriter forArray( +// StructVector structVector, ArrowFieldWriter[] fieldsWriters) { +// return new RowWriterForArray(structVector, fieldsWriters); +// } +// +// // ------------------------------------------------------------------------------------------ +// +// protected final ArrowFieldWriter[] fieldsWriters; +// private final GenericRowData nullRow; +// +// private RowWriter(StructVector structVector, ArrowFieldWriter[] fieldsWriters) { +// super(structVector); +// this.fieldsWriters = fieldsWriters; +// this.nullRow = new GenericRowData(fieldsWriters.length); +// } +// +// abstract boolean isNullAt(T in, int ordinal); +// +// abstract RowData readRow(T in, int ordinal); +// +// @Override +// public void doWrite(T in, int ordinal) { +// RowData row; +// if (isNullAt(in, ordinal)) { +// row = nullRow; +// ((StructVector) getValueVector()).setNull(getCount()); +// } else { +// row = readRow(in, ordinal); +// ((StructVector) getValueVector()).setIndexDefined(getCount()); +// } +// for (int i = 0; i < fieldsWriters.length; i++) { +// fieldsWriters[i].write(row, i); +// } +// } +// +// @Override +// public void finish() { +// super.finish(); +// for (ArrowFieldWriter fieldsWriter : fieldsWriters) { +// fieldsWriter.finish(); +// } +// } +// +// @Override +// public void reset() { +// super.reset(); +// for (ArrowFieldWriter fieldsWriter : fieldsWriters) { +// fieldsWriter.reset(); +// } +// } +// +// // ------------------------------------------------------------------------------------------ +// +// /** +// * {@link RowWriter} for {@link RowData} input. +// */ +// public static final class RowWriterForRow extends RowWriter { +// +// private RowWriterForRow( +// StructVector structVector, ArrowFieldWriter[] fieldsWriters) { +// super(structVector, fieldsWriters); +// } +// +// @Override +// boolean isNullAt(RowData in, int ordinal) { +// return in.isNullAt(ordinal); +// } +// +// @Override +// RowData readRow(RowData in, int ordinal) { +// return in.getRow(ordinal, fieldsWriters.length); +// } +// } +// +// /** +// * {@link RowWriter} for {@link ArrayData} input. +// */ +// public static final class RowWriterForArray extends RowWriter { +// +// private RowWriterForArray( +// StructVector structVector, ArrowFieldWriter[] fieldsWriters) { +// super(structVector, fieldsWriters); +// } +// +// @Override +// boolean isNullAt(ArrayData in, int ordinal) { +// return in.isNullAt(ordinal); +// } +// +// @Override +// RowData readRow(ArrayData in, int ordinal) { +// return in.getRow(ordinal, fieldsWriters.length); +// } +// } +//} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/SmallIntWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/SmallIntWriter.java new file mode 100644 index 000000000..68f77d3ac --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/SmallIntWriter.java @@ -0,0 +1,73 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.vector.SmallIntVector; + +/** + * {@link ArrowFieldWriter} for SmallInt. + */ +public abstract class SmallIntWriter extends ArrowFieldWriter { + + public static SmallIntWriter forObject(SmallIntVector intVector) { + return new SmallIntWriterforObject(intVector); + } + + // ------------------------------------------------------------------------------------------ + + private SmallIntWriter(SmallIntVector intVector) { + super(intVector); + } + + abstract boolean isNullAt(T in, int ordinal); + + abstract short readShort(T in, int ordinal); + + @Override + public void doWrite(T in, int ordinal) { + if (isNullAt(in, ordinal)) { + ((SmallIntVector) getValueVector()).setNull(getCount()); + } else { + ((SmallIntVector) getValueVector()).setSafe(getCount(), readShort(in, ordinal)); + } + } + + // ------------------------------------------------------------------------------------------ + + /** + * {@link SmallIntWriter} for {@link Object[]} input. + */ + public static final class SmallIntWriterforObject extends SmallIntWriter { + + private SmallIntWriterforObject(SmallIntVector intVector) { + super(intVector); + } + + @Override + boolean isNullAt(Object[] in, int ordinal) { + return in[ordinal] == null; + } + + @Override + short readShort(Object[] in, int ordinal) { + return (short) in[ordinal]; + } + } + +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/TimeWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/TimeWriter.java new file mode 100644 index 000000000..e1bd82e94 --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/TimeWriter.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.util.Preconditions; +import org.apache.arrow.vector.*; + +/** + * {@link ArrowFieldWriter} for Time. + */ +public abstract class TimeWriter extends ArrowFieldWriter { + + public static TimeWriter forObject(ValueVector valueVector) { + return new TimeWriterForObject(valueVector); + } + + + // ------------------------------------------------------------------------------------------ + + private TimeWriter(ValueVector valueVector) { + super(valueVector); + Preconditions.checkState( + valueVector instanceof TimeSecVector + || valueVector instanceof TimeMilliVector + || valueVector instanceof TimeMicroVector + || valueVector instanceof TimeNanoVector); + } + + abstract boolean isNullAt(T in, int ordinal); + + abstract int readTime(T in, int ordinal); + + @Override + public void doWrite(T in, int ordinal) { + ValueVector valueVector = getValueVector(); + if (isNullAt(in, ordinal)) { + ((BaseFixedWidthVector) valueVector).setNull(getCount()); + } else if (valueVector instanceof TimeSecVector) { + ((TimeSecVector) valueVector).setSafe(getCount(), readTime(in, ordinal) / 1000); + } else if (valueVector instanceof TimeMilliVector) { + ((TimeMilliVector) valueVector).setSafe(getCount(), readTime(in, ordinal)); + } else if (valueVector instanceof TimeMicroVector) { + ((TimeMicroVector) valueVector).setSafe(getCount(), readTime(in, ordinal) * 1000L); + } else { + ((TimeNanoVector) valueVector).setSafe(getCount(), readTime(in, ordinal) * 1000000L); + } + } + + // ------------------------------------------------------------------------------------------ + + /** + * {@link TimeWriter} for {@link Object[]} input. + */ + public static final class TimeWriterForObject extends TimeWriter { + + private TimeWriterForObject(ValueVector valueVector) { + super(valueVector); + } + + @Override + boolean isNullAt(Object[] in, int ordinal) { + return in[ordinal] == null; + } + + @Override + int readTime(Object[] in, int ordinal) { + return (int) in[ordinal]; + } + } + +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/TimestampWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/TimestampWriter.java new file mode 100644 index 000000000..134293e07 --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/TimestampWriter.java @@ -0,0 +1,122 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.util.Preconditions; +import org.apache.arrow.vector.*; + +import java.sql.Timestamp; + +/** + * {@link ArrowFieldWriter} for Timestamp. + */ +public abstract class TimestampWriter extends ArrowFieldWriter { + + public static TimestampWriter forObject(ValueVector valueVector, int precision) { + return new TimestampWriterforObject(valueVector, precision); + } + + + // ------------------------------------------------------------------------------------------ + + protected final int precision; + + private TimestampWriter(ValueVector valueVector, int precision) { + super(valueVector); + Preconditions.checkState( + valueVector instanceof TimeStampVector + ); + this.precision = precision; + } + + abstract boolean isNullAt(T in, int ordinal); + + abstract Timestamp readTimestamp(T in, int ordinal); + + @Override + public void doWrite(T in, int ordinal) { + ValueVector valueVector = getValueVector(); + if (isNullAt(in, ordinal)) { + ((TimeStampVector) valueVector).setNull(getCount()); + } else { + Timestamp timestamp = readTimestamp(in, ordinal); + + if (valueVector instanceof TimeStampSecTZVector) { + ((TimeStampSecTZVector) valueVector) + .setSafe(getCount(), timestamp.getTime() / 1000); + } else if (valueVector instanceof TimeStampSecVector) { + ((TimeStampSecVector) valueVector) + .setSafe(getCount(), timestamp.getTime() / 1000); + } else if (valueVector instanceof TimeStampMilliTZVector) { + ((TimeStampMilliTZVector) valueVector) + .setSafe(getCount(), timestamp.getTime()); + } else if (valueVector instanceof TimeStampMilliVector) { + ((TimeStampMilliVector) valueVector) + .setSafe(getCount(), timestamp.getTime()); + } else if (valueVector instanceof TimeStampMicroTZVector) { + ((TimeStampMicroTZVector) valueVector) + .setSafe( + getCount(), + timestamp.getTime() * 1000 + + timestamp.getNanos() / 1000); + } else if (valueVector instanceof TimeStampMicroVector) { + ((TimeStampMicroVector) valueVector) + .setSafe( + getCount(), + timestamp.getTime() * 1000 + + timestamp.getNanos() / 1000); + } else if (valueVector instanceof TimeStampNanoTZVector) { + ((TimeStampNanoTZVector) valueVector) + .setSafe( + getCount(), + timestamp.getTime() * 1_000_000 + + timestamp.getNanos()); + } else { + ((TimeStampNanoVector) valueVector) + .setSafe( + getCount(), + timestamp.getTime() * 1_000_000 + + timestamp.getNanos()); + } + } + } + + // ------------------------------------------------------------------------------------------ + + /** + * {@link TimestampWriter} for {@link Object[]} input. + */ + public static final class TimestampWriterforObject extends TimestampWriter { + + private TimestampWriterforObject(ValueVector valueVector, int precision) { + super(valueVector, precision); + } + + @Override + boolean isNullAt(Object[] in, int ordinal) { + return in[ordinal] == null; + } + + @Override + Timestamp readTimestamp(Object[] in, int ordinal) { + return (Timestamp) in[ordinal]; + } + } + +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/TinyIntWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/TinyIntWriter.java new file mode 100644 index 000000000..bbb84d030 --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/TinyIntWriter.java @@ -0,0 +1,74 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.vector.TinyIntVector; + + +/** + * {@link ArrowFieldWriter} for TinyInt. + */ +public abstract class TinyIntWriter extends ArrowFieldWriter { + + public static TinyIntWriter forObject(TinyIntVector tinyIntVector) { + return new TinyIntWriterForObject(tinyIntVector); + } + + // ------------------------------------------------------------------------------------------ + + private TinyIntWriter(TinyIntVector tinyIntVector) { + super(tinyIntVector); + } + + abstract boolean isNullAt(T in, int ordinal); + + abstract byte readByte(T in, int ordinal); + + @Override + public void doWrite(T in, int ordinal) { + if (isNullAt(in, ordinal)) { + ((TinyIntVector) getValueVector()).setNull(getCount()); + } else { + ((TinyIntVector) getValueVector()).setSafe(getCount(), readByte(in, ordinal)); + } + } + + // ------------------------------------------------------------------------------------------ + + /** + * {@link TinyIntWriter} for {@link Object} input. + */ + public static final class TinyIntWriterForObject extends TinyIntWriter { + + private TinyIntWriterForObject(TinyIntVector tinyIntVector) { + super(tinyIntVector); + } + + @Override + boolean isNullAt(Object[] in, int ordinal) { + return in[ordinal] == null; + } + + @Override + byte readByte(Object[] in, int ordinal) { + return (byte) in[ordinal]; + } + } + +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/VarBinaryWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/VarBinaryWriter.java new file mode 100644 index 000000000..f89fb84a5 --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/VarBinaryWriter.java @@ -0,0 +1,74 @@ +/* + * 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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.vector.VarBinaryVector; + +/** + * {@link ArrowFieldWriter} for VarBinary. + */ +public abstract class VarBinaryWriter extends ArrowFieldWriter { + + public static VarBinaryWriter forObject(VarBinaryVector varBinaryVector) { + return new VarBinaryWriterForObject(varBinaryVector); + } + + + // ------------------------------------------------------------------------------------------ + + private VarBinaryWriter(VarBinaryVector varBinaryVector) { + super(varBinaryVector); + } + + abstract boolean isNullAt(T in, int ordinal); + + abstract byte[] readBinary(T in, int ordinal); + + @Override + public void doWrite(T in, int ordinal) { + if (isNullAt(in, ordinal)) { + ((VarBinaryVector) getValueVector()).setNull(getCount()); + } else { + ((VarBinaryVector) getValueVector()).setSafe(getCount(), readBinary(in, ordinal)); + } + } + + // ------------------------------------------------------------------------------------------ + + /** + * {@link VarBinaryWriter} for {@link Object[]} input. + */ + public static final class VarBinaryWriterForObject extends VarBinaryWriter { + + private VarBinaryWriterForObject(VarBinaryVector varBinaryVector) { + super(varBinaryVector); + } + + @Override + boolean isNullAt(Object[] in, int ordinal) { + return in[ordinal] == null; + } + + @Override + byte[] readBinary(Object[] in, int ordinal) { + return (byte[]) in[ordinal]; + } + } + +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/VarCharWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/VarCharWriter.java new file mode 100644 index 000000000..97f0b57cc --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/arrow/writers/VarCharWriter.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 com.dmetasoul.lakesoul.lakesoul.local.arrow.writers; + +import org.apache.arrow.vector.VarCharVector; + +/** + * {@link ArrowFieldWriter} for VarChar. + */ +public abstract class VarCharWriter extends ArrowFieldWriter { + + public static VarCharWriter forObject(VarCharVector varCharVector) { + return new VarCharWriterForObject(varCharVector); + } + + + // ------------------------------------------------------------------------------------------ + + private VarCharWriter(VarCharVector varCharVector) { + super(varCharVector); + } + + abstract boolean isNullAt(T in, int ordinal); + + abstract String readString(T in, int ordinal); + + @Override + public void doWrite(T in, int ordinal) { + if (isNullAt(in, ordinal)) { + ((VarCharVector) getValueVector()).setNull(getCount()); + } else { + ((VarCharVector) getValueVector()) + .setSafe(getCount(), readString(in, ordinal).getBytes()); + } + } + + // ------------------------------------------------------------------------------------------ + + /** + * {@link VarCharWriter} for {@link Object[]} input. + */ + public static final class VarCharWriterForObject extends VarCharWriter { + + private VarCharWriterForObject(VarCharVector varCharVector) { + super(varCharVector); + } + + @Override + boolean isNullAt(Object[] in, int ordinal) { + return in[ordinal] == null; + } + + @Override + String readString(Object[] in, int ordinal) { + return (String) in[ordinal]; + } + } + +} diff --git a/rust/lakesoul-io/src/async_writer/mod.rs b/rust/lakesoul-io/src/async_writer/mod.rs index cb9d164f7..4fcea7b13 100644 --- a/rust/lakesoul-io/src/async_writer/mod.rs +++ b/rust/lakesoul-io/src/async_writer/mod.rs @@ -6,6 +6,7 @@ mod multipart_writer; pub use multipart_writer::MultiPartAsyncWriter; mod sort_writer; +use object_store::ObjectMeta; pub use sort_writer::SortAsyncWriter; mod partitioning_writer; @@ -32,14 +33,14 @@ use datafusion::{ use datafusion_common::{DataFusionError, Result}; use parquet::format::FileMetaData; -// The result of a flush operation with format (partition_desc, file_path, file_meta) -pub type WriterFlushResult = Result>; +// The result of a flush operation with format (partition_desc, file_path, object_meta, file_meta) +pub type WriterFlushResult = Vec<(String, String, ObjectMeta, FileMetaData)>; #[async_trait::async_trait] pub trait AsyncBatchWriter { async fn write_record_batch(&mut self, batch: RecordBatch) -> Result<()>; - async fn flush_and_close(self: Box) -> WriterFlushResult; + async fn flush_and_close(self: Box) -> Result; async fn abort_and_close(self: Box) -> Result<()>; diff --git a/rust/lakesoul-io/src/async_writer/multipart_writer.rs b/rust/lakesoul-io/src/async_writer/multipart_writer.rs index 646a47f76..57855e35b 100644 --- a/rust/lakesoul-io/src/async_writer/multipart_writer.rs +++ b/rust/lakesoul-io/src/async_writer/multipart_writer.rs @@ -7,7 +7,7 @@ use std::{collections::VecDeque, sync::Arc}; use arrow_array::RecordBatch; use arrow_schema::SchemaRef; use atomic_refcell::AtomicRefCell; -use datafusion::execution::{object_store::ObjectStoreUrl, TaskContext}; +use datafusion::{datasource::listing::ListingTableUrl, execution::{object_store::ObjectStoreUrl, TaskContext}}; use datafusion_common::{project_schema, DataFusionError, Result}; use object_store::{path::Path, MultipartId, ObjectStore}; use parquet::{arrow::ArrowWriter, basic::Compression, file::properties::WriterProperties}; @@ -179,7 +179,7 @@ impl AsyncBatchWriter for MultiPartAsyncWriter { MultiPartAsyncWriter::write_batch(batch, &mut self.arrow_writer, &mut self.in_mem_buf, &mut self.writer).await } - async fn flush_and_close(self: Box) -> WriterFlushResult { + async fn flush_and_close(self: Box) -> Result { // close arrow writer to flush remaining rows let mut this = *self; let arrow_writer = this.arrow_writer; @@ -196,7 +196,11 @@ impl AsyncBatchWriter for MultiPartAsyncWriter { // shutdown multi-part async writer to complete the upload this.writer.flush().await?; this.writer.shutdown().await?; - Ok(vec![(TBD_PARTITION_DESC.to_string(), file_path, metadata)]) + let path = Path::from_url_path( + >::as_ref(&ListingTableUrl::parse(&file_path)?).path(), + )?; + let object_meta = this.object_store.head(&path).await?; + Ok(vec![(TBD_PARTITION_DESC.to_string(), file_path, object_meta, metadata)]) } async fn abort_and_close(self: Box) -> Result<()> { diff --git a/rust/lakesoul-io/src/async_writer/partitioning_writer.rs b/rust/lakesoul-io/src/async_writer/partitioning_writer.rs index 461e634aa..d5602e253 100644 --- a/rust/lakesoul-io/src/async_writer/partitioning_writer.rs +++ b/rust/lakesoul-io/src/async_writer/partitioning_writer.rs @@ -42,7 +42,7 @@ pub struct PartitioningAsyncWriter { schema: SchemaRef, sorter_sender: Sender>, _partitioning_exec: Arc, - join_handle: Option>, + join_handle: Option>>, err: Option, buffered_size: u64, } @@ -179,7 +179,7 @@ impl PartitioningAsyncWriter { range_partitions: Arc>, write_id: String, // partitioned_flush_result: PartitionedWriterInfo, - ) -> Result>> { + ) -> Result>>> { let mut data = input.execute(partition, context.clone())?; // O(nm), n = number of data fields, m = number of range partitions let schema_projection_excluding_range = data @@ -256,7 +256,7 @@ impl PartitioningAsyncWriter { let writer_flush_results = writer.flush_and_close().await?; Ok(writer_flush_results .into_iter() - .map(|(_, path, file_metadata)| (partition_desc.clone(), path, file_metadata)) + .map(|(_, path, object_meta, file_metadata)| (partition_desc.clone(), path, object_meta, file_metadata)) .collect::>()) }); flush_join_handle_list.push(flush_result); @@ -266,9 +266,9 @@ impl PartitioningAsyncWriter { } async fn await_and_summary( - join_handles: Vec>>>>, + join_handles: Vec>>>>>, // partitioned_file_path_and_row_count: PartitionedWriterInfo, - ) -> WriterFlushResult { + ) -> Result { let mut flatten_results = Vec::new(); let results = futures::future::join_all(join_handles).await; for result in results { @@ -329,7 +329,7 @@ impl AsyncBatchWriter for PartitioningAsyncWriter { } } - async fn flush_and_close(self: Box) -> WriterFlushResult { + async fn flush_and_close(self: Box) -> Result { if let Some(join_handle) = self.join_handle { let sender = self.sorter_sender; drop(sender); diff --git a/rust/lakesoul-io/src/async_writer/sort_writer.rs b/rust/lakesoul-io/src/async_writer/sort_writer.rs index de4d513e7..c84c80189 100644 --- a/rust/lakesoul-io/src/async_writer/sort_writer.rs +++ b/rust/lakesoul-io/src/async_writer/sort_writer.rs @@ -30,7 +30,7 @@ pub struct SortAsyncWriter { schema: SchemaRef, sorter_sender: Sender>, _sort_exec: Arc, - join_handle: Option>, + join_handle: Option>>, err: Option, buffered_size: u64, } @@ -165,7 +165,7 @@ impl AsyncBatchWriter for SortAsyncWriter { } } - async fn flush_and_close(self: Box) -> WriterFlushResult { + async fn flush_and_close(self: Box) -> Result { if let Some(join_handle) = self.join_handle { let sender = self.sorter_sender; drop(sender); diff --git a/rust/lakesoul-io/src/helpers.rs b/rust/lakesoul-io/src/helpers.rs index 398ef72ce..71cb4887f 100644 --- a/rust/lakesoul-io/src/helpers.rs +++ b/rust/lakesoul-io/src/helpers.rs @@ -26,6 +26,7 @@ use datafusion_common::{cast::as_primitive_array, DFSchema, DataFusionError, Res use datafusion_substrait::substrait::proto::Plan; use object_store::path::Path; +use parquet::format::FileMetaData; use proto::proto::entity::JniWrapper; use rand::distributions::DistString; use url::Url; @@ -547,3 +548,23 @@ pub fn get_batch_memory_size(batch: &RecordBatch) -> Result { .into_iter() .sum()) } + +pub fn get_file_size(metadata: &FileMetaData) -> usize { + let footer_size= metadata.footer_signing_key_metadata.as_ref().map_or(0, |f| f.len()); + dbg!(&metadata); + let rg_size = metadata + .row_groups + .iter() + .map(|row_group| row_group.total_byte_size as usize) + .sum::(); + footer_size + rg_size +} + +pub fn get_file_exist_col(metadata: &FileMetaData) -> String { + metadata + .schema + .iter() + .map(|schema_element| schema_element.name.clone()) + .collect::>() + .join(",") +} \ No newline at end of file diff --git a/rust/lakesoul-io/src/lakesoul_writer.rs b/rust/lakesoul-io/src/lakesoul_writer.rs index 38704aee5..370a461d4 100644 --- a/rust/lakesoul-io/src/lakesoul_writer.rs +++ b/rust/lakesoul-io/src/lakesoul_writer.rs @@ -9,14 +9,13 @@ use std::sync::Arc; use arrow_array::RecordBatch; use arrow_schema::SchemaRef; use datafusion_common::{DataFusionError, Result}; -use parquet::format::FileMetaData; use rand::distributions::DistString; use tokio::runtime::Runtime; use tokio::sync::Mutex; use tracing::debug; -use crate::async_writer::{AsyncBatchWriter, MultiPartAsyncWriter, PartitioningAsyncWriter, SortAsyncWriter}; -use crate::helpers::get_batch_memory_size; +use crate::async_writer::{AsyncBatchWriter, MultiPartAsyncWriter, PartitioningAsyncWriter, SortAsyncWriter, WriterFlushResult}; +use crate::helpers::{get_batch_memory_size, get_file_exist_col}; use crate::lakesoul_io_config::{IOSchema, LakeSoulIOConfig}; use crate::transform::uniform_schema; @@ -30,7 +29,7 @@ pub struct SyncSendableMutableLakeSoulWriter { config: LakeSoulIOConfig, /// The in-progress file writer if any in_progress: Option>>, - flush_results: Vec<(String, String, FileMetaData)>, + flush_results: WriterFlushResult, } impl SyncSendableMutableLakeSoulWriter { @@ -216,13 +215,14 @@ impl SyncSendableMutableLakeSoulWriter { .flush_and_close() .await .map_err(|e| DataFusionError::Internal(format!("err={}, config={:?}", e, self.config.clone())))?; - for (partition_desc, file, _) in self.flush_results.into_iter().chain(results) { + for (partition_desc, file, object_meta, metadata) in self.flush_results.into_iter().chain(results) { + let encoded = format!("{}\x03{}\x03{}", file, object_meta.size, get_file_exist_col(&metadata)); match grouped_results.get_mut(&partition_desc) { Some(files) => { - files.push(file); + files.push(encoded); } None => { - grouped_results.insert(partition_desc, vec![file]); + grouped_results.insert(partition_desc, vec![encoded]); } } } From 2309bf22d42f1ba21e0307d4ff57fbad21294aa9 Mon Sep 17 00:00:00 2001 From: zenghua Date: Wed, 9 Oct 2024 13:27:46 +0800 Subject: [PATCH 09/33] fix case of native reader timestamp convert Signed-off-by: zenghua --- rust/lakesoul-io/src/lakesoul_reader.rs | 64 ++++++++++++++++++++++++- rust/lakesoul-io/src/transform.rs | 55 ++++++--------------- 2 files changed, 77 insertions(+), 42 deletions(-) diff --git a/rust/lakesoul-io/src/lakesoul_reader.rs b/rust/lakesoul-io/src/lakesoul_reader.rs index af28908a1..54a555b5f 100644 --- a/rust/lakesoul-io/src/lakesoul_reader.rs +++ b/rust/lakesoul-io/src/lakesoul_reader.rs @@ -154,6 +154,8 @@ impl SyncSendableMutableLakeSoulReader { #[cfg(test)] mod tests { use super::*; + use arrow::array::as_primitive_array; + use arrow_array::ArrayRef; use rand::prelude::*; use std::mem::ManuallyDrop; use std::ops::Not; @@ -161,7 +163,7 @@ mod tests { use std::time::Instant; use tokio::runtime::Builder; - use arrow::datatypes::{DataType, Field, Schema}; + use arrow::datatypes::{DataType, Field, Schema, TimestampSecondType}; use arrow::util::pretty::print_batches; #[tokio::test] @@ -622,4 +624,64 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_read_file() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, true), + Field::new("range", DataType::Int32, true), + Field::new("datetimeSec", DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, Some(Arc::from("UTC"))), true), + ])); + let partition_schema = Arc::new(Schema::new(vec![Field::new("range", DataType::Int32, true)])); + let reader_conf = LakeSoulIOConfigBuilder::new() + .with_files(vec!["file:/private/tmp/test_local_java_table/range=0/part-AX31Bzzu4jGi23qY_0000.parquet".to_string()]) + // .with_files(vec!["file:/var/folders/4c/34n9w2cd65n0pyjkc3n4q7pc0000gn/T/lakeSource/user1/order_id=4/part-59guLCg5R6v4oLUT_0000.parquet".to_string()]) + .with_thread_num(1) + .with_batch_size(8192) + .with_schema(schema) + .with_partition_schema(partition_schema) + .with_default_column_value("range".to_string(), "0".to_string()) + // .set_inferring_schema(true) + .build(); + let reader = LakeSoulReader::new(reader_conf)?; + let mut reader = ManuallyDrop::new(reader); + reader.start().await?; + static mut ROW_CNT: usize = 0; + + let start = Instant::now(); + while let Some(rb) = reader.next_rb().await { + dbg!(&rb); + let num_rows = &rb.unwrap().num_rows(); + unsafe { + ROW_CNT += num_rows; + println!("{}", ROW_CNT); + } + sleep(Duration::from_millis(20)).await; + } + println!("time cost: {:?}ms", start.elapsed().as_millis()); // ms + + Ok(()) + } + + #[tokio::test] + async fn test_as_primitive_array_timestamp_second_type() -> Result<()> { + use arrow_array::{Array, TimestampSecondArray}; + use arrow_schema::DataType; + use std::sync::Arc; + + // 创建一个 TimestampSecondArray + let data = vec![Some(1627846260), Some(1627846261), None, Some(1627846263)]; + let array = Arc::new(TimestampSecondArray::from(data)) as ArrayRef; + + // 调用 as_primitive_array::(&array) + let primitive_array = as_primitive_array::(&array); + + // 验证结果 + assert_eq!(primitive_array.value(0), 1627846260); + assert_eq!(primitive_array.value(1), 1627846261); + assert!(primitive_array.is_null(2)); + assert_eq!(primitive_array.value(3), 1627846263); + Ok(()) + } + } diff --git a/rust/lakesoul-io/src/transform.rs b/rust/lakesoul-io/src/transform.rs index 6f8b40549..64116a176 100644 --- a/rust/lakesoul-io/src/transform.rs +++ b/rust/lakesoul-io/src/transform.rs @@ -13,7 +13,7 @@ use arrow_array::{ }; use arrow_schema::{DataType, Field, FieldRef, Fields, Schema, SchemaBuilder, SchemaRef, TimeUnit}; use datafusion::error::Result; -use datafusion_common::DataFusionError::{ArrowError, External, Internal}; +use datafusion_common::DataFusionError::{self, ArrowError, External, Internal}; use crate::constant::{ ARROW_CAST_OPTIONS, FLINK_TIMESTAMP_FORMAT, LAKESOUL_EMPTY_STRING, LAKESOUL_NULL_STRING, @@ -148,46 +148,19 @@ pub fn transform_array( default_column_value: Arc>, ) -> Result { Ok(match target_datatype { - DataType::Timestamp(target_unit, Some(target_tz)) => make_array(match &target_unit { - TimeUnit::Second => as_primitive_array::(&array) - .clone() - .with_timezone_opt(Some(target_tz)) - .into_data(), - TimeUnit::Microsecond => { - // workaround compatibility issue for Spark reading Flink's timestamp(9) type - if array.as_any().is::>() { - as_primitive_array::(&array) - .clone() - .reinterpret_cast::() - .with_timezone_opt(Some(target_tz)) - .into_data() - } else { - as_primitive_array::(&array) - .clone() - .with_timezone_opt(Some(target_tz)) - .into_data() - } - }, - TimeUnit::Millisecond => as_primitive_array::(&array) - .clone() - .with_timezone_opt(Some(target_tz)) - .into_data(), - TimeUnit::Nanosecond => { - // workaround compatibility issue for Spark reading Flink's timestamp(9) type - if array.as_any().is::>() { - as_primitive_array::(&array) - .clone() - .reinterpret_cast::() - .with_timezone_opt(Some(target_tz)) - .into_data() - } else { - as_primitive_array::(&array) - .clone() - .with_timezone_opt(Some(target_tz)) - .into_data() - } - }, - }), + DataType::Timestamp(target_unit, Some(target_tz)) => { + let array = match array.data_type() { + DataType::Timestamp(TimeUnit::Second, _) => as_primitive_array::(&array).clone().into_data(), + DataType::Timestamp(TimeUnit::Millisecond, _) => as_primitive_array::(&array).clone().into_data(), + DataType::Timestamp(TimeUnit::Microsecond, _) => as_primitive_array::(&array).clone().into_data(), + DataType::Timestamp(TimeUnit::Nanosecond, _) => as_primitive_array::(&array).clone().into_data(), + _ => return Err(DataFusionError::Internal("Unsupported timestamp type".to_string())), + }; + + let casted_array = cast_with_options(&make_array(array), &DataType::Timestamp(target_unit.clone(), Some(target_tz.clone())), &ARROW_CAST_OPTIONS) + .map_err(ArrowError)?; + casted_array + }, DataType::Struct(target_child_fields) => { let orig_array = as_struct_array(&array); let mut child_array = vec![]; From 7cb33a882d66b665edfd28ebc6f173e70d7ee7f6 Mon Sep 17 00:00:00 2001 From: zenghua Date: Thu, 10 Oct 2024 11:40:42 +0800 Subject: [PATCH 10/33] support cdc table Signed-off-by: zenghua --- .../com/dmetasoul/lakesoul/meta/DBConfig.java | 14 +- .../apache/spark/sql/lakesoul/DDLSuite.scala | 10 +- .../lakesoul/lakesoul/LakeSoulArrowUtils.java | 32 +++++ .../lakesoul/lakesoul/io/NativeIOWriter.java | 25 ++-- .../local/LakeSoulLocalJavaWriter.java | 122 ++++++++++++++---- 5 files changed, 162 insertions(+), 41 deletions(-) create mode 100644 native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/LakeSoulArrowUtils.java diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBConfig.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBConfig.java index b511b6717..2f1e2d511 100644 --- a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBConfig.java +++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBConfig.java @@ -29,12 +29,18 @@ public abstract class DBConfig { public static String LAKESOUL_PARTITION_DESC_KV_DELIM = "="; public static class TableInfoProperty { - public static String HASH_BUCKET_NUM = "hashBucketNum"; + public static final String HASH_BUCKET_NUM = "hashBucketNum"; - public static String DROPPED_COLUMN = "droppedColumn"; + public static final String DROPPED_COLUMN = "droppedColumn"; - public static String DROPPED_COLUMN_SPLITTER = ","; + public static final String DROPPED_COLUMN_SPLITTER = ","; - public static String LAST_TABLE_SCHEMA_CHANGE_TIME = "last_schema_change_time"; + public static final String LAST_TABLE_SCHEMA_CHANGE_TIME = "last_schema_change_time"; + + public static final String USE_CDC = "use_cdc"; + + public static final String CDC_CHANGE_COLUMN = "lakesoul_cdc_change_column"; + + public static final String CDC_CHANGE_COLUMN_DEFAULT = "rowKinds"; } } diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/DDLSuite.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/DDLSuite.scala index b2ac0485a..218e43ff1 100644 --- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/DDLSuite.scala +++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/DDLSuite.scala @@ -417,9 +417,9 @@ abstract class DDLTestBase extends QueryTest with SQLTestUtils { test("Call Statement") { withTable("lakesoul_test") { - val call = spark.sessionState.sqlParser.parsePlan("CALL cat.system.func(c1 => 'name=name1', c2 => map('2',3), c3 => true,c4 => TIMESTAMP '2013-01-01',c5=>3L,c6=>1.0D,c7=>ARRAY(1,3))") - val s = call.asInstanceOf[CallStatement] - assert(s.args.length == 7) + val call = spark.sessionState.sqlParser.parsePlan("CALL cat.system.func(c1 => 'name=name1', c2 => map('2',3), c3 => true,c4 => TIMESTAMP '2013-01-01',c5=>3L,c6=>1.0D,c7=>ARRAY(1,3))") + val s = call.asInstanceOf[CallStatement] + assert(s.args.length == 7) } } @@ -485,4 +485,8 @@ abstract class DDLTestBase extends QueryTest with SQLTestUtils { } } } + + // test("read test table") { + // sql("select * from test_local_java_table where range=2").show(1000) + // } } diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/LakeSoulArrowUtils.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/LakeSoulArrowUtils.java new file mode 100644 index 000000000..1bae28c00 --- /dev/null +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/LakeSoulArrowUtils.java @@ -0,0 +1,32 @@ +package com.dmetasoul.lakesoul.lakesoul; + +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.Schema; + +import java.util.ArrayList; +import java.util.List; + +public class LakeSoulArrowUtils { + public static Schema cdcColumnAlignment(Schema schema, String cdcColumn) { + if (cdcColumn != null) { + // set cdc column as the last field + Field cdcField = null; + List fields = new ArrayList<>(schema.getFields().size() + 1); + for (Field field : schema.getFields()) { + if (!field.getName().equals(cdcColumn)) { + fields.add(field); + } else { + cdcField = field; + } + } + if (cdcField != null) { + fields.add(cdcField); + } else { + throw new RuntimeException(String.format("Invalid Schema of %s, CDC Column [%s] not found", schema, cdcColumn)); +// fields.add(new Field(cdcColumn, FieldType.notNullable(new ArrowType.Utf8()), null)); + } + return new Schema(fields); + } + return schema; + } +} diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java index 078be7bd7..235295543 100644 --- a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java @@ -4,6 +4,8 @@ package com.dmetasoul.lakesoul.lakesoul.io; +import com.dmetasoul.lakesoul.lakesoul.LakeSoulArrowUtils; +import com.dmetasoul.lakesoul.meta.DBConfig; import com.dmetasoul.lakesoul.meta.DBUtil; import com.dmetasoul.lakesoul.meta.entity.TableInfo; import com.fasterxml.jackson.core.JsonProcessingException; @@ -37,8 +39,19 @@ public NativeIOWriter(Schema schema) { public NativeIOWriter(TableInfo tableInfo) { super("NativeWriter"); + + String cdcColumn; + try { + ObjectMapper mapper = new ObjectMapper(); + Map properties = mapper.readValue(tableInfo.getProperties(), Map.class); + setHashBucketNum(Integer.parseInt(properties.get(HASH_BUCKET_NUM))); + cdcColumn = properties.get(DBConfig.TableInfoProperty.CDC_CHANGE_COLUMN); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } try { - setSchema(Schema.fromJSON(tableInfo.getTableSchema())); + Schema schema = Schema.fromJSON(tableInfo.getTableSchema()); + setSchema(LakeSoulArrowUtils.cdcColumnAlignment(schema, cdcColumn)); } catch (IOException e) { throw new RuntimeException(e); } @@ -47,14 +60,6 @@ public NativeIOWriter(TableInfo tableInfo) { setRangePartitions(partitionKeys.rangeKeys); useDynamicPartition(true); - ObjectMapper mapper = new ObjectMapper(); - - try { - Map properties = mapper.readValue(tableInfo.getProperties(), Map.class); - setHashBucketNum(Integer.parseInt(properties.get(HASH_BUCKET_NUM).toString())); - } catch (JsonProcessingException e) { - throw new RuntimeException(e); - } withPrefix(tableInfo.getTablePath()); @@ -166,7 +171,7 @@ public String getFileExistCols() { public static FlushResult decodeFlushResult(String encoded) { String[] fields = encoded.split("\u0003"); - + Preconditions.checkArgument(fields.length == 3); return new FlushResult(fields[0], Long.parseLong(fields[1]), fields[2]); } diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/LakeSoulLocalJavaWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/LakeSoulLocalJavaWriter.java index 4d1da9f04..cb69565ec 100644 --- a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/LakeSoulLocalJavaWriter.java +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/LakeSoulLocalJavaWriter.java @@ -1,12 +1,14 @@ package com.dmetasoul.lakesoul.lakesoul.local; -import com.dmetasoul.lakesoul.lakesoul.local.arrow.ArrowBatchWriter; +import com.dmetasoul.lakesoul.lakesoul.LakeSoulArrowUtils; import com.dmetasoul.lakesoul.lakesoul.io.NativeIOBase; import com.dmetasoul.lakesoul.lakesoul.io.NativeIOWriter; +import com.dmetasoul.lakesoul.lakesoul.local.arrow.ArrowBatchWriter; import com.dmetasoul.lakesoul.meta.DBConfig; import com.dmetasoul.lakesoul.meta.DBManager; import com.dmetasoul.lakesoul.meta.DBUtil; import com.dmetasoul.lakesoul.meta.entity.*; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.arrow.util.Preconditions; import org.apache.arrow.vector.VectorSchemaRoot; @@ -22,9 +24,7 @@ import java.io.IOException; import java.math.BigDecimal; import java.sql.Timestamp; -import java.time.LocalDateTime; import java.time.ZoneId; -import java.time.ZoneOffset; import java.util.*; import static com.dmetasoul.lakesoul.meta.DBConfig.TableInfoProperty; @@ -71,6 +71,11 @@ public class LakeSoulLocalJavaWriter implements AutoCloseable { // Key for S3 bucket public static final String S3_BUCKET = "s3.bucket"; + // Key for lakesoul cdc column + public static final String CDC_COLUMN_KEY = "lakesoul.cdc.column"; + // Default value for lakesoul cdc column + public static final String CDC_COLUMN_DEFAULT_VALUE = "rowKinds"; + // Key for memory limit of native writer public static final String MEM_LIMIT = "lakesoul.native_writer.mem_limit"; @@ -88,6 +93,8 @@ public class LakeSoulLocalJavaWriter implements AutoCloseable { private DBManager dbManager; private Map params; + String cdcColumn = null; + public static void setIOConfigs(Map conf, NativeIOBase io) { if (conf.containsKey(DEFAULT_FS)) { @@ -140,6 +147,14 @@ public void init(Map params) throws IOException { tableInfo = dbManager.getTableInfoByName(params.get(TABLE_NAME)); + try { + ObjectMapper mapper = new ObjectMapper(); + Map properties = mapper.readValue(tableInfo.getProperties(), Map.class); + cdcColumn = properties.get(DBConfig.TableInfoProperty.CDC_CHANGE_COLUMN); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + initNativeWriter(); } @@ -147,7 +162,7 @@ public void init(Map params) throws IOException { private void initNativeWriter() throws IOException { nativeWriter = new NativeIOWriter(tableInfo); - Schema arrowSchema = Schema.fromJSON(tableInfo.getTableSchema()); + Schema arrowSchema = LakeSoulArrowUtils.cdcColumnAlignment(Schema.fromJSON(tableInfo.getTableSchema()), cdcColumn); batch = VectorSchemaRoot.create(arrowSchema, nativeWriter.getAllocator()); arrowWriter = ArrowBatchWriter.createWriter(batch); @@ -162,8 +177,31 @@ void write(Object[] row) { this.totalRows++; } + public void writeAddRow(Object[] row) { + if (cdcColumn != null) { + Object[] addRow = new Object[row.length + 1]; + for (int i = 0; i < row.length; i++) { + addRow[i] = row[i]; + } + addRow[row.length] = "insert"; + write(addRow); + } else { + write(row); + } + } - void commit() throws Exception { + public void writeDeleteRow(Object[] row) { + Preconditions.checkArgument(cdcColumn != null, ""); + Object[] delRow = new Object[row.length + 1]; + for (int i = 0; i < row.length; i++) { + delRow[i] = row[i]; + } + delRow[row.length] = "delete"; + write(delRow); + } + + + public void commit() throws IOException { this.arrowWriter.finish(); this.nativeWriter.write(this.batch); @@ -183,8 +221,14 @@ void commit() throws Exception { this.rowsInBatch = 0; } - private void recreateWriter() throws Exception { - close(); + private void recreateWriter() throws IOException { + try { + close(); + } catch (IOException e) { + throw e; + } catch (Exception e) { + throw new RuntimeException(e); + } initNativeWriter(); } @@ -353,6 +397,8 @@ public Object visit(ArrowType.Duration duration) { public static void main(String[] args) throws Exception { DBManager meta = new DBManager(); + + boolean cdc = true; meta.cleanMeta(); String tableId = "table_" + UUID.randomUUID(); List primaryKeys = Arrays.asList("id"); @@ -362,18 +408,34 @@ public static void main(String[] args) throws Exception { primaryKeys, partitionKeys ); - List fields = Arrays.asList( - new Field("id", FieldType.nullable(new ArrowType.Int(32, true)), null) - , new Field("range", FieldType.nullable(new ArrowType.Int(32, true)), null) - , new Field("int", FieldType.nullable(new ArrowType.Int(32, true)), null) - , new Field("utf8", FieldType.nullable(new ArrowType.Utf8()), null) - , new Field("decimal", FieldType.nullable(ArrowType.Decimal.createDecimal(10, 3, null)), null) - , new Field("boolean", FieldType.nullable(new ArrowType.Bool()), null) - , new Field("date", FieldType.nullable(new ArrowType.Date(DateUnit.DAY)), null) - , new Field("datetimeSec", FieldType.nullable(new ArrowType.Timestamp(TimeUnit.SECOND, ZoneId.of("UTC").toString())), null) - , new Field("datetimeMilli", FieldType.nullable(new ArrowType.Timestamp(TimeUnit.MILLISECOND, ZoneId.of("UTC").toString())), null) - ); + List fields; + if (cdc) { + fields = Arrays.asList( + new Field("id", FieldType.nullable(new ArrowType.Int(32, true)), null) + , new Field("range", FieldType.nullable(new ArrowType.Int(32, true)), null) + , new Field("int", FieldType.nullable(new ArrowType.Int(32, true)), null) + , new Field("utf8", FieldType.nullable(new ArrowType.Utf8()), null) + , new Field("decimal", FieldType.nullable(ArrowType.Decimal.createDecimal(10, 3, null)), null) + , new Field("boolean", FieldType.nullable(new ArrowType.Bool()), null) + , new Field("date", FieldType.nullable(new ArrowType.Date(DateUnit.DAY)), null) + , new Field("datetimeSec", FieldType.nullable(new ArrowType.Timestamp(TimeUnit.SECOND, ZoneId.of("UTC").toString())), null) + , new Field(TableInfoProperty.CDC_CHANGE_COLUMN_DEFAULT, FieldType.notNullable(new ArrowType.Utf8()), null) + , new Field("datetimeMilli", FieldType.nullable(new ArrowType.Timestamp(TimeUnit.MILLISECOND, ZoneId.of("UTC").toString())), null) + ); + } else { + fields = Arrays.asList( + new Field("id", FieldType.nullable(new ArrowType.Int(32, true)), null) + , new Field("range", FieldType.nullable(new ArrowType.Int(32, true)), null) + , new Field("int", FieldType.nullable(new ArrowType.Int(32, true)), null) + , new Field("utf8", FieldType.nullable(new ArrowType.Utf8()), null) + , new Field("decimal", FieldType.nullable(ArrowType.Decimal.createDecimal(10, 3, null)), null) + , new Field("boolean", FieldType.nullable(new ArrowType.Bool()), null) + , new Field("date", FieldType.nullable(new ArrowType.Date(DateUnit.DAY)), null) + , new Field("datetimeSec", FieldType.nullable(new ArrowType.Timestamp(TimeUnit.SECOND, ZoneId.of("UTC").toString())), null) + , new Field("datetimeMilli", FieldType.nullable(new ArrowType.Timestamp(TimeUnit.MILLISECOND, ZoneId.of("UTC").toString())), null) + ); + } Schema schema = new Schema(fields); ObjectMapper objectMapper = new ObjectMapper(); @@ -383,6 +445,11 @@ public static void main(String[] args) throws Exception { properties.put(TableInfoProperty.HASH_BUCKET_NUM, "4"); properties.put("hashPartitions", String.join(DBConfig.LAKESOUL_HASH_PARTITION_SPLITTER, primaryKeys)); + if (cdc) { + properties.put(TableInfoProperty.USE_CDC, "true"); + properties.put(TableInfoProperty.CDC_CHANGE_COLUMN, + TableInfoProperty.CDC_CHANGE_COLUMN_DEFAULT); + } } meta.createTable( @@ -409,15 +476,22 @@ public static void main(String[] args) throws Exception { int numRows = 1024; int numCols = fields.size(); for (int i = 0; i < numRows; i++) { - Object[] row = new Object[numCols]; - for (int j = 0; j < numCols; j++) { - if (fields.get(j).getName().contains("range")) { - row[j] = i % ranges; + Object[] row = new Object[cdc ? numCols - 1 : numCols]; + for (int j = 0, k = 0; j < numCols; j++) { + if (fields.get(j).getName().contains(TableInfoProperty.CDC_CHANGE_COLUMN_DEFAULT)) { + continue; + } else if (fields.get(j).getName().contains("id")) { + row[k++] = i; + } else if (fields.get(j).getName().contains("range")) { + row[k++] = i % ranges; } else { - row[j] = fields.get(j).getType().accept(ArrowTypeMockDataGenerator.INSTANCE); + row[k++] = fields.get(j).getType().accept(ArrowTypeMockDataGenerator.INSTANCE); } } - localJavaWriter.write(row); + localJavaWriter.writeAddRow(row); + if (cdc && i % 7 == 0) { + localJavaWriter.writeDeleteRow(row); + } } localJavaWriter.commit(); } From 5346ec02dbf710f27a7def724a21b68b884b6412 Mon Sep 17 00:00:00 2001 From: zenghua Date: Thu, 10 Oct 2024 11:54:41 +0800 Subject: [PATCH 11/33] add debug log for LakeSoulLocalJavaWriter Signed-off-by: zenghua --- .../com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java | 1 - .../lakesoul/lakesoul/local/LakeSoulLocalJavaWriter.java | 7 ++++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java index 235295543..5af7f9023 100644 --- a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java @@ -132,7 +132,6 @@ public int writeIpc(byte[] encodedBatch) throws IOException { } public void write(VectorSchemaRoot batch) throws IOException { - System.out.println("writing batch: " + batch.getRowCount()); ArrowArray array = ArrowArray.allocateNew(allocator); ArrowSchema schema = ArrowSchema.allocateNew(allocator); Data.exportVectorSchemaRoot(allocator, batch, provider, array, schema); diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/LakeSoulLocalJavaWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/LakeSoulLocalJavaWriter.java index cb69565ec..84523d00d 100644 --- a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/LakeSoulLocalJavaWriter.java +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/local/LakeSoulLocalJavaWriter.java @@ -136,6 +136,8 @@ public static void setFSConf(Map conf, String confKey, String fs public void init(Map params) throws IOException { + LOG.info(String.format("LakeSoulLocalJavaWriter init with params=%s", params)); + this.params = params; Preconditions.checkArgument(params.containsKey(PG_URL_KEY)); Preconditions.checkArgument(params.containsKey(USERNAME_KEY)); @@ -160,6 +162,7 @@ public void init(Map params) throws IOException { } private void initNativeWriter() throws IOException { + LOG.info(String.format("LakeSoulLocalJavaWriter initNativeWriter with tableInfo=%s", tableInfo)); nativeWriter = new NativeIOWriter(tableInfo); Schema arrowSchema = LakeSoulArrowUtils.cdcColumnAlignment(Schema.fromJSON(tableInfo.getTableSchema()), cdcColumn); @@ -191,7 +194,7 @@ public void writeAddRow(Object[] row) { } public void writeDeleteRow(Object[] row) { - Preconditions.checkArgument(cdcColumn != null, ""); + Preconditions.checkArgument(cdcColumn != null, "DeleteRow is not support for Non Cdc Table"); Object[] delRow = new Object[row.length + 1]; for (int i = 0; i < row.length; i++) { delRow[i] = row[i]; @@ -202,6 +205,7 @@ public void writeDeleteRow(Object[] row) { public void commit() throws IOException { + LOG.info(String.format("LakeSoulLocalJavaWriter commit batch size = %s, batch schema=%s", batch.getRowCount(), batch.getSchema().toJson())); this.arrowWriter.finish(); this.nativeWriter.write(this.batch); @@ -211,6 +215,7 @@ public void commit() throws IOException { commitInfoList.add(createDataCommitInfo(entry.getKey(), entry.getValue())); } + LOG.info(String.format("Committing DataCommitInfo=%s", commitInfoList)); for (DataCommitInfo commitInfo : commitInfoList) { dbManager.commitDataCommitInfo(commitInfo, Collections.emptyList()); } From 4c2675a8fa49e237a77e4a0dde133daac9eb8eb5 Mon Sep 17 00:00:00 2001 From: zenghua Date: Wed, 16 Oct 2024 15:40:00 +0800 Subject: [PATCH 12/33] add compaction ut and update transactionCommit interface Signed-off-by: zenghua --- .../dmetasoul/lakesoul/meta/MetaCommit.scala | 21 +- .../lakesoul/tables/LakeSoulTable.scala | 27 +- .../execution/LakeSoulTableOperations.scala | 7 +- .../sql/lakesoul/TransactionCommit.scala | 225 +++++++++------ .../sql/lakesoul/TransactionalWrite.scala | 1 + .../lakesoul/commands/CompactionCommand.scala | 269 +++++++++++++++++- .../spark/sql/lakesoul/utils/MetaData.scala | 3 +- .../lakesoul/commands/CompactionSuite.scala | 65 ++++- 8 files changed, 498 insertions(+), 120 deletions(-) diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/MetaCommit.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/MetaCommit.scala index 1ec14acb1..97072ec0b 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/MetaCommit.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/meta/MetaCommit.scala @@ -5,13 +5,14 @@ package com.dmetasoul.lakesoul.meta import com.alibaba.fastjson.JSONObject -import com.dmetasoul.lakesoul.meta.entity.{FileOp, Uuid} +import com.dmetasoul.lakesoul.meta.entity.Uuid import org.apache.spark.internal.Logging import org.apache.spark.sql.lakesoul.exception.LakeSoulErrors import org.apache.spark.sql.lakesoul.utils._ import java.util import scala.collection.JavaConverters +import scala.collection.JavaConverters.asScalaBufferConverter object MetaCommit extends Logging { //meta commit process @@ -94,21 +95,21 @@ object MetaCommit extends Logging { for (dataCommitInfo <- dataCommitInfoArray) { val metaDataCommitInfo = entity.DataCommitInfo.newBuilder metaDataCommitInfo.setTableId(table_id) - metaDataCommitInfo.setPartitionDesc(dataCommitInfo.range_value) + metaDataCommitInfo.setPartitionDesc(dataCommitInfo.getPartitionDesc) metaDataCommitInfo.setCommitOp(entity.CommitOp.valueOf(commitType)) - val uuid = dataCommitInfo.commit_id - metaDataCommitInfo.setCommitId(Uuid.newBuilder.setHigh(uuid.getMostSignificantBits).setLow(uuid.getLeastSignificantBits).build) + val uuid = dataCommitInfo.getCommitId + metaDataCommitInfo.setCommitId(uuid) val fileOps = new util.ArrayList[entity.DataFileOp]() - for (file_info <- dataCommitInfo.file_ops) { + for (file_info <- dataCommitInfo.getFileOpsList.asScala) { val metaDataFileInfo = entity.DataFileOp.newBuilder - metaDataFileInfo.setPath(file_info.path) - metaDataFileInfo.setFileOp(FileOp.valueOf(file_info.file_op)) - metaDataFileInfo.setSize(file_info.size) - metaDataFileInfo.setFileExistCols(file_info.file_exist_cols) + metaDataFileInfo.setPath(file_info.getPath) + metaDataFileInfo.setFileOp(file_info.getFileOp) + metaDataFileInfo.setSize(file_info.getSize) + metaDataFileInfo.setFileExistCols(file_info.getFileExistCols) fileOps.add(metaDataFileInfo.build) } metaDataCommitInfo.addAllFileOps(fileOps) - metaDataCommitInfo.setTimestamp(dataCommitInfo.modification_time) + metaDataCommitInfo.setTimestamp(dataCommitInfo.getTimestamp) metaDataCommitInfoList.add(metaDataCommitInfo.build) } SparkMetaVersion.dbManager.batchCommitDataCommitInfo(metaDataCommitInfoList) diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala index 23748bb6e..4dfae78d5 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala @@ -307,55 +307,55 @@ class LakeSoulTable(df: => Dataset[Row], snapshotManagement: SnapshotManagement) //by default, force perform compaction on whole table def compaction(): Unit = { - compaction("", true, Map.empty[String, Any], "", "", false) + compaction("", true, Map.empty[String, Any], "", "", false, None) } def compaction(cleanOldCompaction: Boolean): Unit = { - compaction("", true, Map.empty[String, Any], "", "", cleanOldCompaction) + compaction("", true, Map.empty[String, Any], "", "", cleanOldCompaction, None) } def compaction(condition: String): Unit = { - compaction(condition, true, Map.empty[String, Any], "", "", false) + compaction(condition, true, Map.empty[String, Any], "", "", false, None) } def compaction(condition: String, cleanOldCompaction: Boolean): Unit = { - compaction(condition, true, Map.empty[String, Any], "", "", cleanOldCompaction) + compaction(condition, true, Map.empty[String, Any], "", "", cleanOldCompaction, None) } def compaction(mergeOperatorInfo: Map[String, Any]): Unit = { - compaction("", true, mergeOperatorInfo, "", "", false) + compaction("", true, mergeOperatorInfo, "", "", false, None) } def compaction(condition: String, mergeOperatorInfo: Map[String, Any]): Unit = { - compaction(condition, true, mergeOperatorInfo, "", "", false) + compaction(condition, true, mergeOperatorInfo, "", "", false, None) } def compaction(condition: String, hiveTableName: String): Unit = { - compaction(condition, true, Map.empty[String, Any], hiveTableName, "", false) + compaction(condition, true, Map.empty[String, Any], hiveTableName, "", false, None) } def compaction(condition: String, hiveTableName: String, hivePartitionName: String): Unit = { - compaction(condition, true, Map.empty[String, Any], hiveTableName, hivePartitionName, false) + compaction(condition, true, Map.empty[String, Any], hiveTableName, hivePartitionName, false, None) } def compaction(force: Boolean, mergeOperatorInfo: Map[String, Any] = Map.empty[String, Any], cleanOldCompaction: Boolean): Unit = { - compaction("", force, mergeOperatorInfo, "", "", cleanOldCompaction) + compaction("", force, mergeOperatorInfo, "", "", cleanOldCompaction, None) } def compaction(condition: String, force: Boolean, cleanOldCompaction: Boolean): Unit = { - compaction(condition, true, Map.empty[String, Any], "", "", cleanOldCompaction) + compaction(condition, true, Map.empty[String, Any], "", "", cleanOldCompaction, None) } def compaction(condition: String, force: Boolean, mergeOperatorInfo: java.util.Map[String, Any], cleanOldCompaction: Boolean): Unit = { - compaction(condition, force, mergeOperatorInfo.asScala.toMap, "", "", cleanOldCompaction) + compaction(condition, force, mergeOperatorInfo.asScala.toMap, "", "", cleanOldCompaction, None) } def compaction(condition: String, @@ -363,7 +363,8 @@ class LakeSoulTable(df: => Dataset[Row], snapshotManagement: SnapshotManagement) mergeOperatorInfo: Map[String, Any], hiveTableName: String, hivePartitionName: String, - cleanOldCompaction: Boolean): Unit = { + cleanOldCompaction: Boolean, + maxSnapshotsPerGroup: Option[Int]): Unit = { val newMergeOpInfo = mergeOperatorInfo.map(m => { val key = if (!m._1.startsWith(LakeSoulUtils.MERGE_OP_COL)) { @@ -379,7 +380,7 @@ class LakeSoulTable(df: => Dataset[Row], snapshotManagement: SnapshotManagement) (key, value) }) - executeCompaction(df, snapshotManagement, condition, force, newMergeOpInfo, hiveTableName, hivePartitionName, cleanOldCompaction) + executeCompaction(df, snapshotManagement, condition, force, newMergeOpInfo, hiveTableName, hivePartitionName, cleanOldCompaction, maxSnapshotsPerGroup) } def setCompactionTtl(days: Int): LakeSoulTable = { diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala index f416e34de..0646b2ac3 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala @@ -173,7 +173,8 @@ trait LakeSoulTableOperations extends AnalysisHelper { mergeOperatorInfo: Map[String, String], hiveTableName: String = "", hivePartitionName: String = "", - cleanOldCompaction: Boolean): Unit = { + cleanOldCompaction: Boolean, + maxSnapshotsPerGroup: Option[Int]): Unit = { toDataset(sparkSession, CompactionCommand( snapshotManagement, condition, @@ -181,7 +182,9 @@ trait LakeSoulTableOperations extends AnalysisHelper { mergeOperatorInfo, hiveTableName, hivePartitionName, - cleanOldCompaction)) + cleanOldCompaction, + // maxSnapshotsPerGroup + )) } protected def executeSetCompactionTtl(snapshotManagement: SnapshotManagement, days: Int): Unit = { diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionCommit.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionCommit.scala index c380708af..5f76ccb16 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionCommit.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionCommit.scala @@ -5,6 +5,7 @@ package org.apache.spark.sql.lakesoul import com.dmetasoul.lakesoul.meta._ +import com.dmetasoul.lakesoul.meta.entity.{CommitOp, DataCommitInfo, DataFileOp, FileOp, Uuid} import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession @@ -15,6 +16,7 @@ import org.apache.spark.sql.lakesoul.schema.SchemaUtils import org.apache.spark.sql.lakesoul.utils._ import java.util.{ConcurrentModificationException, UUID} +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -254,105 +256,144 @@ trait Transaction extends TransactionalWrite with Logging { ) } - val expireFilesWithDeleteOp = expireFiles.map(f => f.copy(file_op = "del")) - - val depend_files = readFiles.toSeq ++ addFiles ++ expireFilesWithDeleteOp - - //Gets all the partition names that need to be changed - val depend_partitions = depend_files - .groupBy(_.range_partitions).keys - .toSet - - val add_file_arr_buf = new ArrayBuffer[DataCommitInfo]() - - val add_partition_info_arr_buf = new ArrayBuffer[PartitionInfoScala]() - - val commit_type = commitType.getOrElse(CommitType("append")).name - if (commit_type.equals(CommitType("update").name)) { - val delete_file_set = new mutable.HashSet[String]() - expireFilesWithDeleteOp.foreach(file => { - delete_file_set.add(file.path) - }) - - val partition_list = snapshotManagement.snapshot.getPartitionInfoArray - depend_partitions.foreach(range_key => { - val filter_files = new ArrayBuffer[DataFileInfo]() - val partition_info = partition_list.filter(_.range_value.equalsIgnoreCase(range_key)) - if (partition_info.length > 0) { - val partition_files = DataOperation.getSinglePartitionDataInfo(partition_info.head) - partition_files.foreach(partition_file => { - if (!delete_file_set.contains(partition_file.path)) { - filter_files += partition_file - } - }) - } + val (add_file_arr_buf, add_partition_info_arr_buf) = createDataCommitInfo(addFiles, expireFiles, query_id, batch_id) + + commitDataCommitInfo(add_file_arr_buf, add_partition_info_arr_buf, query_id, batch_id, readPartitionInfo) + } + snapshotManagement.updateSnapshot() + } - val changeFiles = addFiles.union(expireFilesWithDeleteOp) - .filter(a => a.range_partitions.equalsIgnoreCase(range_key)) - - filter_files ++= changeFiles - - if (filter_files.nonEmpty) { - val addUUID = getCommitIdByBatchIdAndQueryId(batch_id, query_id) - add_file_arr_buf += DataCommitInfo( - tableInfo.table_id, - range_key, - addUUID, - commit_type, - System.currentTimeMillis(), - filter_files.toArray - ) - add_partition_info_arr_buf += PartitionInfoScala( - table_id = tableInfo.table_id, - range_value = range_key, - read_files = Array(addUUID) - ) + def createDataCommitInfo(addFiles: Seq[DataFileInfo], + expireFiles: Seq[DataFileInfo], + query_id: String, + batch_id: Long): (List[DataCommitInfo], List[PartitionInfoScala]) = { + val expireFilesWithDeleteOp = expireFiles.map(f => f.copy(file_op = "del")) + + val depend_files = readFiles.toSeq ++ addFiles ++ expireFilesWithDeleteOp + + //Gets all the partition names that need to be changed + val depend_partitions = depend_files + .groupBy(_.range_partitions).keys + .toSet + + val add_file_arr_buf = List.newBuilder[DataCommitInfo] + + val add_partition_info_arr_buf = List.newBuilder[PartitionInfoScala] + + val commit_type = commitType.getOrElse(CommitType("append")).name + if (commit_type.equals(CommitType("update").name)) { + val delete_file_set = new mutable.HashSet[String]() + expireFilesWithDeleteOp.foreach(file => { + delete_file_set.add(file.path) + }) + + val partition_list = snapshotManagement.snapshot.getPartitionInfoArray + depend_partitions.foreach(range_key => { + val filter_files = new ArrayBuffer[DataFileInfo]() + val partition_info = partition_list.filter(_.range_value.equalsIgnoreCase(range_key)) + if (partition_info.length > 0) { + val partition_files = DataOperation.getSinglePartitionDataInfo(partition_info.head) + partition_files.foreach(partition_file => { + if (!delete_file_set.contains(partition_file.path)) { + filter_files += partition_file + } + }) + } + + val changeFiles = addFiles.union(expireFilesWithDeleteOp) + .filter(a => a.range_partitions.equalsIgnoreCase(range_key)) + + filter_files ++= changeFiles + + if (filter_files.nonEmpty) { + val addUUID = getCommitIdByBatchIdAndQueryId(batch_id, query_id) + val fileOps = filter_files.map { file => + DataFileOp.newBuilder() + .setPath(file.path) + .setFileOp(if (file.file_op == "add") FileOp.add else FileOp.del) + .setSize(file.size) + .setFileExistCols(file.file_exist_cols) + .build() } - }) - } else { - depend_partitions.foreach(range_key => { - val changeFiles = addFiles.union(expireFilesWithDeleteOp) - .filter(a => a.range_partitions.equalsIgnoreCase(range_key)) - if (changeFiles.nonEmpty) { - val addUUID = getCommitIdByBatchIdAndQueryId(batch_id, query_id) - add_file_arr_buf += DataCommitInfo( - tableInfo.table_id, - range_key, - addUUID, - commit_type, - System.currentTimeMillis(), - changeFiles.toArray - ) - add_partition_info_arr_buf += PartitionInfoScala( - table_id = tableInfo.table_id, - range_value = range_key, - read_files = Array(addUUID) - ) + + add_file_arr_buf += DataCommitInfo.newBuilder() + .setTableId(tableInfo.table_id) + .setPartitionDesc(range_key) + .setCommitId(Uuid.newBuilder().setHigh(addUUID.getMostSignificantBits).setLow(addUUID.getLeastSignificantBits).build()) + .addAllFileOps(fileOps.toList.asJava) + .setCommitOp(CommitOp.valueOf(commit_type)) + .setTimestamp(System.currentTimeMillis()) + .setCommitted(false) + .build() + + add_partition_info_arr_buf += PartitionInfoScala( + table_id = tableInfo.table_id, + range_value = range_key, + read_files = Array(addUUID) + ) + } + }) + } else { + depend_partitions.foreach(range_key => { + val changeFiles = addFiles.union(expireFilesWithDeleteOp) + .filter(a => a.range_partitions.equalsIgnoreCase(range_key)) + if (changeFiles.nonEmpty) { + val addUUID = getCommitIdByBatchIdAndQueryId(batch_id, query_id) + val fileOps = changeFiles.map { file => + DataFileOp.newBuilder() + .setPath(file.path) + .setFileOp(if (file.file_op == "add") FileOp.add else FileOp.del) + .setSize(file.size) + .setFileExistCols(file.file_exist_cols) + .build() } - }) - } - val meta_info = MetaInfo( - table_info = tableInfo.copy(short_table_name = shortTableName), - dataCommitInfo = add_file_arr_buf.toArray, - partitionInfoArray = add_partition_info_arr_buf.toArray, - commit_type = commitType.getOrElse(CommitType("append")), - query_id = query_id, - batch_id = batch_id, - readPartitionInfo = readPartitionInfo - ) + add_file_arr_buf += DataCommitInfo.newBuilder() + .setTableId(tableInfo.table_id) + .setPartitionDesc(range_key) + .setCommitId(Uuid.newBuilder().setHigh(addUUID.getMostSignificantBits).setLow(addUUID.getLeastSignificantBits).build()) + .addAllFileOps(fileOps.toList.asJava) + .setCommitOp(CommitOp.valueOf(commit_type)) + .setTimestamp(System.currentTimeMillis()) + .setCommitted(false) + .build() + + add_partition_info_arr_buf += PartitionInfoScala( + table_id = tableInfo.table_id, + range_value = range_key, + read_files = Array(addUUID) + ) + } + }) + } - try { - val changeSchema = !isFirstCommit && newTableInfo.nonEmpty - MetaCommit.doMetaCommit(meta_info, changeSchema) - } catch { - case e: MetaRerunException => throw e - case e: Throwable => throw e - } + (add_file_arr_buf.result(), add_partition_info_arr_buf.result()) + } - committed = true + def commitDataCommitInfo(add_file_arr_buf: List[DataCommitInfo], + add_partition_info_arr_buf: List[PartitionInfoScala], + query_id: String, + batch_id: Long, + readPartitionInfo: Array[PartitionInfoScala]): Unit = { + val meta_info = MetaInfo( + table_info = tableInfo.copy(short_table_name = shortTableName), + dataCommitInfo = add_file_arr_buf.toArray, + partitionInfoArray = add_partition_info_arr_buf.toArray, + commit_type = commitType.getOrElse(CommitType("append")), + query_id = query_id, + batch_id = batch_id, + readPartitionInfo = readPartitionInfo + ) + + try { + val changeSchema = !isFirstCommit && newTableInfo.nonEmpty + MetaCommit.doMetaCommit(meta_info, changeSchema) + } catch { + case e: MetaRerunException => throw e + case e: Throwable => throw e } - snapshotManagement.updateSnapshot() + + committed = true } def getCommitIdByBatchIdAndQueryId(batch_id: Long, query_id: String): UUID = { diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala index 617545fe2..541007468 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala @@ -6,6 +6,7 @@ package org.apache.spark.sql.lakesoul import com.dmetasoul.lakesoul.meta.DBConfig.{LAKESOUL_EMPTY_STRING, LAKESOUL_NULL_STRING, LAKESOUL_RANGE_PARTITION_SPLITTER} import com.dmetasoul.lakesoul.meta.{CommitType, DataFileInfo} +import com.dmetasoul.lakesoul.meta.entity.DataCommitInfo import org.apache.hadoop.fs.Path import org.apache.spark.sql.Dataset import org.apache.spark.sql.catalyst.catalog.BucketSpec diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala index 3b1f6e272..01dff4541 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala @@ -209,4 +209,271 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, Seq.empty } -} \ No newline at end of file +} + +//// SPDX-FileCopyrightText: 2023 LakeSoul Contributors +//// +//// SPDX-License-Identifier: Apache-2.0 +// +//package org.apache.spark.sql.lakesoul.commands +// +//import com.dmetasoul.lakesoul.meta.entity.{DataCommitInfo, Uuid, CommitOp, DataFileOp, FileOp} +//import com.dmetasoul.lakesoul.meta.{DBUtil, DataFileInfo, PartitionInfoScala, SparkMetaVersion} +//import com.dmetasoul.lakesoul.spark.clean.CleanOldCompaction.cleanOldCommitOpDiskData +//import org.apache.hadoop.fs.Path +//import org.apache.spark.internal.Logging +//import org.apache.spark.sql.catalyst.expressions.PredicateHelper +//import org.apache.spark.sql.execution.command.LeafRunnableCommand +//import org.apache.spark.sql.execution.datasources.v2.merge.MergeDeltaParquetScan +//import org.apache.spark.sql.execution.datasources.v2.parquet.{NativeParquetScan, ParquetScan} +//import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} +//import org.apache.spark.sql.functions.expr +//import org.apache.spark.sql.lakesoul.catalog.LakeSoulTableV2 +//import org.apache.spark.sql.lakesoul.exception.LakeSoulErrors +//import org.apache.spark.sql.lakesoul.{BatchDataSoulFileIndexV2, LakeSoulOptions, SnapshotManagement, TransactionCommit} +//import org.apache.spark.sql.types.StructType +//import org.apache.spark.sql.util.CaseInsensitiveStringMap +//import org.apache.spark.sql.{Dataset, Row, SparkSession} +// +//import scala.collection.mutable +//import scala.collection.mutable.ArrayBuffer +//import scala.collection.JavaConverters._ +//import java.util.{UUID, ArrayList} +// +//case class CompactionCommand(snapshotManagement: SnapshotManagement, +// conditionString: String, +// force: Boolean, +// mergeOperatorInfo: Map[String, String], +// hiveTableName: String = "", +// hivePartitionName: String = "", +// cleanOldCompaction: Boolean, +// maxSnapshotsPerGroup: Option[Int] +// ) +// extends LeafRunnableCommand with PredicateHelper with Logging { +// +// def filterPartitionNeedCompact(spark: SparkSession, +// force: Boolean, +// partitionInfo: PartitionInfoScala): Boolean = { +// partitionInfo.read_files.length >= 1 +// } +// +// def executeCompaction(spark: SparkSession, tc: TransactionCommit, files: Seq[DataFileInfo], readPartitionInfo: Array[PartitionInfoScala]): DataCommitInfo = { +// +// val fileIndex = BatchDataSoulFileIndexV2(spark, snapshotManagement, files) +// val table = LakeSoulTableV2( +// spark, +// new Path(snapshotManagement.table_path), +// None, +// None, +// Option(fileIndex), +// Option(mergeOperatorInfo) +// ) +// val option = new CaseInsensitiveStringMap( +// Map("basePath" -> tc.tableInfo.table_path_s.get, "isCompaction" -> "true").asJava) +// +// val partitionNames = readPartitionInfo.head.range_value.split(',').map(p => { +// p.split('=').head +// }) +// +// val scan = table.newScanBuilder(option).build() +// val newReadFiles = if (scan.isInstanceOf[ParquetScan] || scan.isInstanceOf[NativeParquetScan]) { +// fileIndex.getFileInfo(Nil) +// } else { +// scan.asInstanceOf[MergeDeltaParquetScan].newFileIndex.getFileInfo(Nil) +// } +// +// val tableSchemaWithoutPartitions = StructType(table.schema().filter(f => { +// !partitionNames.contains(f.name) +// })) +// +// val v2Relation = DataSourceV2Relation( +// table, +// tableSchemaWithoutPartitions.toAttributes, +// None, +// None, +// option +// ) +// +// val compactDF = Dataset.ofRows( +// spark, +// DataSourceV2ScanRelation( +// v2Relation, +// scan, +// tableSchemaWithoutPartitions.toAttributes +// ) +// ) +// +// tc.setReadFiles(newReadFiles) +// val map = mutable.HashMap[String, String]() +// map.put("isCompaction", "true") +// if (readPartitionInfo.nonEmpty) { +// map.put("partValue", readPartitionInfo.head.range_value) +// } +// val (newFiles, path) = tc.writeFiles(compactDF, Some(new LakeSoulOptions(map.toMap, spark.sessionState.conf)), isCompaction = true) +// tc.createDataCommitInfo(newFiles, Seq.empty, readPartitionInfo) +// +// val fileOps = new ArrayList[DataFileOp]() +// newFiles.foreach { file => +// fileOps.add(DataFileOp.newBuilder() +// .setPath(file.path) +// .setFileOp(FileOp.add) +// .setSize(file.size) +// .setFileExistCols(file.file_exist_cols) +// .build()) +// } +// +// val commitId = DBUtil.toProtoUuid(UUID.randomUUID()) +// +// DataCommitInfo.newBuilder() +// .setTableId(tc.tableInfo.table_id) +// .setPartitionDesc(readPartitionInfo.head.range_value) +// .setCommitId(commitId) +// .addAllFileOps(fileOps) +// .setCommitOp(CommitOp.CompactionCommit) +// .setTimestamp(System.currentTimeMillis()) +// .setCommitted(false) +// .setDomain("") // 如果需要设置 domain,请替换为适当的值 +// .build() +// } +// +// def escapeSingleBackQuotedString(str: String): String = { +// val builder = mutable.StringBuilder.newBuilder +// +// str.foreach { +// case '\'' => "" +// case '`' => "" +// case ch => builder += ch +// } +// +// builder.toString() +// } +// +// override def run(sparkSession: SparkSession): Seq[Row] = { +// val condition = conditionString match { +// case "" => None +// case _: String => Option(expr(conditionString).expr) +// } +// //when condition is defined, only one partition need compaction, +// //else we will check whole table +// if (condition.isDefined) { +// val targetOnlyPredicates = +// splitConjunctivePredicates(condition.get) +// +// snapshotManagement.withNewTransaction(tc => { +// tc.setCommitType("compaction") +// val files = tc.filterFiles(targetOnlyPredicates) +// //ensure only one partition execute compaction command +// val partitionSet = files.map(_.range_partitions).toSet +// if (partitionSet.isEmpty) { +// throw LakeSoulErrors.partitionColumnNotFoundException(condition.get, 0) +// } else if (partitionSet.size > 1) { +// throw LakeSoulErrors.partitionColumnNotFoundException(condition.get, partitionSet.size) +// } +// +// lazy val hasNoDeltaFile = if (force) { +// false +// } else { +// files.groupBy(_.file_bucket_id).forall(_._2.size == 1) +// } +// +// val groupedFiles = if (maxSnapshotsPerGroup.isDefined) { +// val groupSize = maxSnapshotsPerGroup.get +// var groupedFiles = new ArrayBuffer[Seq[DataFileInfo]] +// for (i <- files.indices by groupSize) { +// groupedFiles += files.slice(i, i + groupSize) +// } +// groupedFiles.toSeq +// } else { +// Seq(files) +// } +// +// +// val allDataCommitInfo = groupedFiles.flatMap(files => { +// if (!hasNoDeltaFile) { +// val partitionInfo = SparkMetaVersion.getSinglePartitionInfo( +// snapshotManagement.getTableInfoOnly.table_id, +// partitionSet.head, +// "" +// ) +// if (partitionInfo.commit_op.equals("CompactionCommit") && partitionInfo.read_files.length == 1) { +// logInfo("=========== All Partitions Have Compacted, This Operation Will Cancel!!! ===========") +// None +// } else { +// Some(executeCompaction(sparkSession, tc, files, Array(partitionInfo))) +// } +// } else { +// logInfo("== Compaction: This partition has been compacted or has no delta file.") +// None +// } +// }) +// +// // 提交所有的 DataCommitInfo +// tc.commitDataCommitInfo(allDataCommitInfo.toList, Array.empty[PartitionInfoScala].toList, "", 0, Array.empty[PartitionInfoScala]) +// +// if (cleanOldCompaction) { +// val tablePath = snapshotManagement.table_path +// cleanOldCommitOpDiskData(tablePath, partitionSet.head, sparkSession) +// } +// +// }) +// } else { +// +// val allInfo = SparkMetaVersion.getAllPartitionInfo(snapshotManagement.getTableInfoOnly.table_id) +// val partitionsNeedCompact = allInfo +// .filter(filterPartitionNeedCompact(sparkSession, force, _)) +// +// partitionsNeedCompact.foreach(part => { +// snapshotManagement.withNewTransaction(tc => { +// tc.setCommitType("compaction") +// val files = tc.getCompactionPartitionFiles(part) +// lazy val hasNoDeltaFile = if (force) { +// false +// } else { +// files.groupBy(_.file_bucket_id).forall(_._2.size == 1) +// } +// +// val groupedFiles = if (maxSnapshotsPerGroup.isDefined) { +// val groupSize = maxSnapshotsPerGroup.get +// var groupedFiles = new ArrayBuffer[Seq[DataFileInfo]] +// for (i <- files.indices by groupSize) { +// groupedFiles += files.slice(i, i + groupSize) +// } +// groupedFiles.toSeq +// } else { +// Seq(files) +// } +// +// +// val allDataCommitInfo = groupedFiles.flatMap(files => { +// if (!hasNoDeltaFile) { +// val partitionInfo = SparkMetaVersion.getSinglePartitionInfo( +// snapshotManagement.getTableInfoOnly.table_id, +// part.range_value, +// "" +// ) +// if (partitionInfo.commit_op.equals("CompactionCommit") && partitionInfo.read_files.length == 1) { +// logInfo("=========== All Partitions Have Compacted, This Operation Will Cancel!!! ===========") +// None +// } else { +// Some(executeCompaction(sparkSession, tc, files, Array(partitionInfo))) +// } +// } else { +// logInfo(s"== Partition ${part.range_value} has no delta file.") +// None +// } +// }) +// +// // 提交所有的 DataCommitInfo +// tc.commitDataCommitInfo(allDataCommitInfo.toList, Array.empty[PartitionInfoScala].toList, "", 0, Array(part)) +// +// if (cleanOldCompaction) { +// val tablePath = snapshotManagement.table_path +// cleanOldCommitOpDiskData(tablePath, part.range_value, sparkSession) +// } +// }) +// }) +// } +// Seq.empty +// } +// +//} diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/utils/MetaData.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/utils/MetaData.scala index 6ab671518..4ea2dd9e3 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/utils/MetaData.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/utils/MetaData.scala @@ -6,6 +6,7 @@ package org.apache.spark.sql.lakesoul.utils import com.dmetasoul.lakesoul.meta.DBConfig.{LAKESOUL_HASH_PARTITION_SPLITTER, LAKESOUL_RANGE_PARTITION_SPLITTER} import com.dmetasoul.lakesoul.meta.dao.TableInfoDao +import com.dmetasoul.lakesoul.meta.entity import com.dmetasoul.lakesoul.meta.{CommitState, CommitType, DataFileInfo, PartitionInfoScala} import com.fasterxml.jackson.annotation.JsonIgnore import org.apache.arrow.vector.types.pojo.Schema @@ -17,7 +18,7 @@ import java.util.UUID case class MetaInfo(table_info: TableInfo, partitionInfoArray: Array[PartitionInfoScala], - dataCommitInfo: Array[DataCommitInfo], + dataCommitInfo: Array[entity.DataCommitInfo], commit_type: CommitType, commit_id: String = "", query_id: String = "", diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala index 627836bb5..6efe295cd 100644 --- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala +++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala @@ -4,6 +4,7 @@ package org.apache.spark.sql.lakesoul.commands +import com.dmetasoul.lakesoul.meta.SparkMetaVersion import com.dmetasoul.lakesoul.tables.LakeSoulTable import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf @@ -502,6 +503,68 @@ class CompactionSuite extends QueryTest } } + test("compaction with limited file group") { + withTempDir { tempDir => + val tablePath = tempDir.getCanonicalPath + val spark = SparkSession.active + + // 创建测试数据 + val df = Seq( + (1, "2023-01-01", 10, 1), + (2, "2023-01-02", 20, 1), + (3, "2023-01-03", 30, 1), + (4, "2023-01-04", 40, 1), + (5, "2023-01-05", 50, 1) + ).toDF("id", "date", "value", "range") + + // 写入初始数据 + df.write + .format("lakesoul") + .option("rangePartitions", "range") + .option("hashPartitions", "id") + .option("hashBucketNum", "4") + .save(tablePath) -} + val lakeSoulTable = LakeSoulTable.forPath(tablePath) + + // 模拟多次追加操作 + for (i <- 1 to 10) { + val appendDf = Seq( + (i * 10, s"2023-02-0$i", i * 100, 1) + ).toDF("id", "date", "value", "range") + lakeSoulTable.upsert(appendDf) + } + + // 获取初始的PartitionInfo数量 + val initialPartitionInfoCount = getSnapshotCount(tablePath) + + // 执行受限的Compaction操作 (每3个PartitionInfo一组) + lakeSoulTable.compaction(condition = "", force = true, mergeOperatorInfo = Map.empty, + hiveTableName = "", hivePartitionName = "", cleanOldCompaction = false, + maxSnapshotsPerGroup = None) + // 获取Compaction后的PartitionInfo数量 + val compactedPartitionInfoCount = getSnapshotCount(tablePath) + + // 验证结果 + assert(compactedPartitionInfoCount < initialPartitionInfoCount, + s"Compaction应该减少PartitionInfo的数量,但是从${initialPartitionInfoCount}变为$compactedPartitionInfoCount") + + assert(compactedPartitionInfoCount > 1, + s"Compaction应该产生多个PartitionInfo组,但只有${compactedPartitionInfoCount}个") + + // 验证数据完整性 + val compactedData = lakeSoulTable.toDF.orderBy("date", "id").collect() + assert(compactedData.length == 15, s"压缩后的数据应该有15行,但实际有${compactedData.length}行") + } + } + + // 辅助方法:获取Snapshot的数量 + def getSnapshotCount(tablePath: String): Int = { + val sm = SnapshotManagement(SparkUtil.makeQualifiedTablePath(new Path(tablePath)).toString) + var count = 0 + SparkMetaVersion.getAllPartitionInfo(sm.getTableInfoOnly.table_id).foreach(p => count += p.read_files.length) + count + } + +} From 3227c5e08447d1df3456a5972ca63bbd2142c063 Mon Sep 17 00:00:00 2001 From: zenghua Date: Thu, 17 Oct 2024 14:47:02 +0800 Subject: [PATCH 13/33] compact with file number limit Signed-off-by: zenghua --- .../test/flinkSource/BatchReadSuite.java | 5 + .../spark/compaction/CompactionTask.scala | 4 +- .../lakesoul/tables/LakeSoulTable.scala | 12 +- .../execution/LakeSoulTableOperations.scala | 4 +- .../datasources/LakeSoulFileWriter.scala | 52 +-- .../sql/lakesoul/TransactionalWrite.scala | 8 +- .../lakesoul/commands/CompactionCommand.scala | 417 ++++-------------- .../lakesoul/commands/CompactionSuite.scala | 88 ++-- 8 files changed, 204 insertions(+), 386 deletions(-) diff --git a/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/flinkSource/BatchReadSuite.java b/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/flinkSource/BatchReadSuite.java index ec39cc1d8..2e9507848 100644 --- a/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/flinkSource/BatchReadSuite.java +++ b/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/flinkSource/BatchReadSuite.java @@ -163,4 +163,9 @@ private void createLakeSoulSourceTableWithoutPK(TableEnvironment tEnvs) tEnvs.executeSql(createOrderSql); tEnvs.executeSql("INSERT INTO order_noPK VALUES (1,'apple',20), (2,'tomato',10), (3,'water',15)").await(); } + + public static void main(String[] args) { + TableEnvironment tEnv = TestUtils.createTableEnv(BATCH_TYPE); + tEnv.executeSql("select * from compaction_limit_table order by `id`, `date`").print(); + } } diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala index d2ea2854c..475f8aa62 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala @@ -96,14 +96,14 @@ object CompactionTask { println("------ " + threadName + " is compressing table path is: " + path + " ------") val table = LakeSoulTable.forPath(path) if (partitionDesc == "") { - table.compaction() + table.compaction(fileNumLimit = Some(100)) } else { val partitions = partitionDesc.split(",").map( partition => { partition.replace("=", "='") + "'" } ).mkString(" and ") - table.compaction(partitions, true) + table.compaction(partitions, true, fileNumLimit = Some(100)) } } catch { case e: Exception => { diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala index 4dfae78d5..ee9bc9c4c 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala @@ -310,6 +310,10 @@ class LakeSoulTable(df: => Dataset[Row], snapshotManagement: SnapshotManagement) compaction("", true, Map.empty[String, Any], "", "", false, None) } + def compaction(fileNumLimit: Option[Int]): Unit = { + compaction("", true, Map.empty[String, Any], "", "", false, fileNumLimit) + } + def compaction(cleanOldCompaction: Boolean): Unit = { compaction("", true, Map.empty[String, Any], "", "", cleanOldCompaction, None) } @@ -322,6 +326,10 @@ class LakeSoulTable(df: => Dataset[Row], snapshotManagement: SnapshotManagement) compaction(condition, true, Map.empty[String, Any], "", "", cleanOldCompaction, None) } + def compaction(condition: String, cleanOldCompaction: Boolean, fileNumLimit: Option[Int]): Unit = { + compaction(condition, true, Map.empty[String, Any], "", "", cleanOldCompaction, fileNumLimit) + } + def compaction(mergeOperatorInfo: Map[String, Any]): Unit = { compaction("", true, mergeOperatorInfo, "", "", false, None) } @@ -364,7 +372,7 @@ class LakeSoulTable(df: => Dataset[Row], snapshotManagement: SnapshotManagement) hiveTableName: String, hivePartitionName: String, cleanOldCompaction: Boolean, - maxSnapshotsPerGroup: Option[Int]): Unit = { + fileNumLimit: Option[Int]): Unit = { val newMergeOpInfo = mergeOperatorInfo.map(m => { val key = if (!m._1.startsWith(LakeSoulUtils.MERGE_OP_COL)) { @@ -380,7 +388,7 @@ class LakeSoulTable(df: => Dataset[Row], snapshotManagement: SnapshotManagement) (key, value) }) - executeCompaction(df, snapshotManagement, condition, force, newMergeOpInfo, hiveTableName, hivePartitionName, cleanOldCompaction, maxSnapshotsPerGroup) + executeCompaction(df, snapshotManagement, condition, force, newMergeOpInfo, hiveTableName, hivePartitionName, cleanOldCompaction, fileNumLimit) } def setCompactionTtl(days: Int): LakeSoulTable = { diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala index 0646b2ac3..155f5b212 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala @@ -174,7 +174,7 @@ trait LakeSoulTableOperations extends AnalysisHelper { hiveTableName: String = "", hivePartitionName: String = "", cleanOldCompaction: Boolean, - maxSnapshotsPerGroup: Option[Int]): Unit = { + fileNumLimit: Option[Int]): Unit = { toDataset(sparkSession, CompactionCommand( snapshotManagement, condition, @@ -183,7 +183,7 @@ trait LakeSoulTableOperations extends AnalysisHelper { hiveTableName, hivePartitionName, cleanOldCompaction, - // maxSnapshotsPerGroup + fileNumLimit )) } diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala index c6d305a50..63cd2f203 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala @@ -40,20 +40,20 @@ import java.util.{Date, UUID} /** A helper object for writing FileFormat data out to a location. */ object LakeSoulFileWriter extends Logging { /** - * Basic work flow of this command is: - * 1. Driver side setup, including output committer initialization and data source specific - * preparation work for the write job to be issued. - * 2. Issues a write job consists of one or more executor side tasks, each of which writes all - * rows within an RDD partition. - * 3. If no exception is thrown in a task, commits that task, otherwise aborts that task; If any - * exception is thrown during task commitment, also aborts that task. - * 4. If all tasks are committed, commit the job, otherwise aborts the job; If any exception is - * thrown during job commitment, also aborts the job. - * 5. If the job is successfully committed, perform post-commit operations such as - * processing statistics. - * - * @return The set of all partition paths that were updated during this write job. - */ + * Basic work flow of this command is: + * 1. Driver side setup, including output committer initialization and data source specific + * preparation work for the write job to be issued. + * 2. Issues a write job consists of one or more executor side tasks, each of which writes all + * rows within an RDD partition. + * 3. If no exception is thrown in a task, commits that task, otherwise aborts that task; If any + * exception is thrown during task commitment, also aborts that task. + * 4. If all tasks are committed, commit the job, otherwise aborts the job; If any exception is + * thrown during job commitment, also aborts the job. + * 5. If the job is successfully committed, perform post-commit operations such as + * processing statistics. + * + * @return The set of all partition paths that were updated during this write job. + */ def write( sparkSession: SparkSession, plan: SparkPlan, @@ -143,6 +143,7 @@ object LakeSoulFileWriter extends Logging { val isCDC = caseInsensitiveOptions.getOrElse("isCDC", "false").toBoolean val isCompaction = caseInsensitiveOptions.getOrElse("isCompaction", "false").toBoolean + val staticBucketId = caseInsensitiveOptions.getOrElse("staticBucketId", "-1").toInt // We should first sort by partition columns, then bucket id, and finally sorting columns. val requiredOrdering = @@ -171,6 +172,7 @@ object LakeSoulFileWriter extends Logging { committer.setupJob(job) val nativeIOEnable = sparkSession.sessionState.conf.getConf(LakeSoulSQLConf.NATIVE_IO_ENABLE) + def nativeWrap(plan: SparkPlan): RDD[InternalRow] = { if (isCompaction && !isCDC && nativeIOEnable) { plan match { @@ -236,7 +238,7 @@ object LakeSoulFileWriter extends Logging { description = description, jobIdInstant = jobIdInstant, sparkStageId = taskContext.stageId(), - sparkPartitionId = taskContext.partitionId(), + sparkPartitionId = if (isCompaction && staticBucketId != -1) staticBucketId else taskContext.partitionId(), sparkAttemptNumber = taskContext.taskAttemptId().toInt & Integer.MAX_VALUE, committer, iterator = iter, @@ -349,9 +351,9 @@ object LakeSoulFileWriter extends Logging { } /** - * For every registered [[WriteJobStatsTracker]], call `processStats()` on it, passing it - * the corresponding [[WriteTaskStats]] from all executors. - */ + * For every registered [[WriteJobStatsTracker]], call `processStats()` on it, passing it + * the corresponding [[WriteTaskStats]] from all executors. + */ private[datasources] def processStats( statsTrackers: Seq[WriteJobStatsTracker], statsPerTask: Seq[Seq[WriteTaskStats]], @@ -383,13 +385,13 @@ object LakeSoulFileWriter extends Logging { outputColumns: Seq[Attribute]) private class StaticPartitionedDataWriter( - description: WriteJobDescription, - taskAttemptContext: TaskAttemptContext, - committer: FileCommitProtocol, - options: Map[String, String], - partitionId: Int, - bucketSpec: Option[BucketSpec], - customMetrics: Map[String, SQLMetric] = Map.empty) + description: WriteJobDescription, + taskAttemptContext: TaskAttemptContext, + committer: FileCommitProtocol, + options: Map[String, String], + partitionId: Int, + bucketSpec: Option[BucketSpec], + customMetrics: Map[String, SQLMetric] = Map.empty) extends FileFormatDataWriter(description, taskAttemptContext, committer, customMetrics) { private var fileCounter: Int = _ private var recordsInFile: Long = _ diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala index 541007468..aee3c0056 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala @@ -128,7 +128,13 @@ trait TransactionalWrite { val hashPartitionSchema = tableInfo.hash_partition_schema var outputPath = SparkUtil.makeQualifiedTablePath(tableInfo.table_path) if (isCompaction) { - outputPath = SparkUtil.makeQualifiedTablePath(new Path(tableInfo.table_path.toString + "/compact_" + System.currentTimeMillis())) + + val compactPath = if (writeOptions.isDefined) { + writeOptions.get.options.getOrElse("compactPath", tableInfo.table_path.toString + "/compact_" + System.currentTimeMillis()) + } else { + tableInfo.table_path.toString + "/compact_" + System.currentTimeMillis() + } + outputPath = SparkUtil.makeQualifiedTablePath(new Path(compactPath)) } val dc = if (isCompaction) { val cdcCol = snapshot.getTableInfo.configuration.get(LakeSoulTableProperties.lakeSoulCDCChangePropKey) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala index 01dff4541..3a3a7408f 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala @@ -4,7 +4,8 @@ package org.apache.spark.sql.lakesoul.commands -import com.dmetasoul.lakesoul.meta.{DataFileInfo, PartitionInfoScala, SparkMetaVersion} +import com.dmetasoul.lakesoul.meta.entity.DataCommitInfo +import com.dmetasoul.lakesoul.meta.{DBUtil, DataFileInfo, PartitionInfoScala, SparkMetaVersion} import com.dmetasoul.lakesoul.spark.clean.CleanOldCompaction.cleanOldCommitOpDiskData import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging @@ -23,8 +24,9 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.util.Utils -import scala.collection.JavaConversions._ import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConverters._ case class CompactionCommand(snapshotManagement: SnapshotManagement, conditionString: String, @@ -32,10 +34,12 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, mergeOperatorInfo: Map[String, String], hiveTableName: String = "", hivePartitionName: String = "", - cleanOldCompaction: Boolean + cleanOldCompaction: Boolean, + fileNumLimit: Option[Int] = None ) extends LeafRunnableCommand with PredicateHelper with Logging { + val compactPath = snapshotManagement.getTableInfoOnly.table_path.toString + "/compact_" + System.currentTimeMillis() def filterPartitionNeedCompact(spark: SparkSession, force: Boolean, @@ -43,10 +47,10 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, partitionInfo.read_files.length >= 1 } - def executeCompaction(spark: SparkSession, tc: TransactionCommit, files: Seq[DataFileInfo], readPartitionInfo: Array[PartitionInfoScala]): Unit = { + def executeCompaction(spark: SparkSession, tc: TransactionCommit, files: Seq[DataFileInfo], readPartitionInfo: Array[PartitionInfoScala]): List[DataCommitInfo] = { if (readPartitionInfo.forall(p => p.commit_op.equals("CompactionCommit") && p.read_files.length == 1)) { logInfo("=========== All Partitions Have Compacted, This Operation Will Cancel!!! ===========") - return + return List.empty } val fileIndex = BatchDataSoulFileIndexV2(spark, snapshotManagement, files) val table = LakeSoulTableV2( @@ -58,7 +62,7 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, Option(mergeOperatorInfo) ) val option = new CaseInsensitiveStringMap( - Map("basePath" -> tc.tableInfo.table_path_s.get, "isCompaction" -> "true")) + Map("basePath" -> tc.tableInfo.table_path_s.get, "isCompaction" -> "true").asJava) val partitionNames = readPartitionInfo.head.range_value.split(',').map(p => { p.split('=').head @@ -93,14 +97,79 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, ) tc.setReadFiles(newReadFiles) - tc.setCommitType("compaction") val map = mutable.HashMap[String, String]() map.put("isCompaction", "true") + map.put("compactPath", compactPath) if (readPartitionInfo.nonEmpty) { map.put("partValue", readPartitionInfo.head.range_value) } + if (tc.tableInfo.hash_partition_columns.nonEmpty) { + val headBucketId = files.head.file_bucket_id + if (files.forall(_.file_bucket_id == headBucketId)) { + map.put("staticBucketId", headBucketId.toString) + } + } val (newFiles, path) = tc.writeFiles(compactDF, Some(new LakeSoulOptions(map.toMap, spark.sessionState.conf)), isCompaction = true) - tc.commit(newFiles, Seq.empty, readPartitionInfo) + + tc.createDataCommitInfo(newFiles, Seq.empty, "", -1)._1 + } + + def escapeSingleBackQuotedString(str: String): String = { + val builder = mutable.StringBuilder.newBuilder + + str.foreach { + case '\'' => "" + case '`' => "" + case ch => builder += ch + } + + builder.toString() + } + + def compactSinglePartition(sparkSession: SparkSession, tc: TransactionCommit, files: Seq[DataFileInfo], sourcePartition: PartitionInfoScala) = { + val bucketedFiles = if (tc.tableInfo.hash_partition_columns.isEmpty) { + Seq(-1 -> files) + } else { + files.groupBy(_.file_bucket_id) + } + println(bucketedFiles) + val allDataCommitInfo = bucketedFiles.flatMap(groupByBucketId => { + val (bucketId, files) = groupByBucketId + val groupedFiles = if (fileNumLimit.isDefined) { + val groupSize = fileNumLimit.get + val groupedFiles = new ArrayBuffer[Seq[DataFileInfo]] + for (i <- files.indices by groupSize) { + groupedFiles += files.slice(i, i + groupSize) + } + groupedFiles + } else { + Seq(files) + } + + groupedFiles.flatMap(files => { + lazy val hasNoDeltaFile = if (force) { + false + } else { + files.forall(_.size == 1) + } + if (!hasNoDeltaFile) { + executeCompaction(sparkSession, tc, files, Array(sourcePartition)) + } else { + logInfo(s"== Partition ${sourcePartition.range_value} has no delta file.") + None + } + }) + }) + + val compactPartitionInfo = List.newBuilder[PartitionInfoScala] + compactPartitionInfo += PartitionInfoScala( + table_id = tc.tableInfo.table_id, + range_value = sourcePartition.range_value, + read_files = allDataCommitInfo.map(_.getCommitId).map(DBUtil.toJavaUUID).toArray + ) + + tc.commitDataCommitInfo(allDataCommitInfo.toList, compactPartitionInfo.result, "", -1, Array(sourcePartition)) + val partitionStr = escapeSingleBackQuotedString(conditionString) if (hiveTableName.nonEmpty) { val spark = SparkSession.active @@ -109,30 +178,18 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, spark.sessionState.catalogManager.setCurrentCatalog(SESSION_CATALOG_NAME) if (hivePartitionName.nonEmpty) { spark.sql(s"ALTER TABLE $hiveTableName DROP IF EXISTS partition($hivePartitionName)") - spark.sql(s"ALTER TABLE $hiveTableName ADD partition($hivePartitionName) location '${path.toString}/$partitionStr'") + spark.sql(s"ALTER TABLE $hiveTableName ADD partition($hivePartitionName) location '${compactPath}/$partitionStr'") } else { spark.sql(s"ALTER TABLE $hiveTableName DROP IF EXISTS partition($conditionString)") - spark.sql(s"ALTER TABLE $hiveTableName ADD partition($conditionString) location '${path.toString}/$partitionStr'") + spark.sql(s"ALTER TABLE $hiveTableName ADD partition($conditionString) location '${compactPath}/$partitionStr'") } }) { spark.sessionState.catalogManager.setCurrentCatalog(currentCatalog) } } - logInfo("=========== Compaction Success!!! ===========") - } - - def escapeSingleBackQuotedString(str: String): String = { - val builder = mutable.StringBuilder.newBuilder - - str.foreach { - case '\'' => "" - case '`' => "" - case ch => builder += ch - } - builder.toString() } override def run(sparkSession: SparkSession): Seq[Row] = { @@ -147,8 +204,8 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, splitConjunctivePredicates(condition.get) snapshotManagement.withNewTransaction(tc => { + tc.setCommitType("compaction") val files = tc.filterFiles(targetOnlyPredicates) - //ensure only one partition execute compaction command val partitionSet = files.map(_.range_partitions).toSet if (partitionSet.isEmpty) { @@ -157,25 +214,17 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, throw LakeSoulErrors.partitionColumnNotFoundException(condition.get, partitionSet.size) } - lazy val hasNoDeltaFile = if (force) { - false - } else { - files.groupBy(_.file_bucket_id).forall(_._2.size == 1) - } + val partitionInfo = SparkMetaVersion.getSinglePartitionInfo( + snapshotManagement.getTableInfoOnly.table_id, + partitionSet.head, + "" + ) - if (hasNoDeltaFile) { - logInfo("== Compaction: This partition has been compacted or has no delta file.") - } else { - val partitionInfo = SparkMetaVersion.getSinglePartitionInfo( - snapshotManagement.getTableInfoOnly.table_id, - partitionSet.head, - "" - ) - executeCompaction(sparkSession, tc, files, Array(partitionInfo)) - if (cleanOldCompaction) { - val tablePath = snapshotManagement.table_path - cleanOldCommitOpDiskData(tablePath, partitionSet.head, sparkSession) - } + compactSinglePartition(sparkSession, tc, files, partitionInfo) + + if (cleanOldCompaction) { + val tablePath = snapshotManagement.table_path + cleanOldCommitOpDiskData(tablePath, partitionSet.head, sparkSession) } }) @@ -187,21 +236,14 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, partitionsNeedCompact.foreach(part => { snapshotManagement.withNewTransaction(tc => { + tc.setCommitType("compaction") val files = tc.getCompactionPartitionFiles(part) - val hasNoDeltaFile = if (force) { - false - } else { - files.groupBy(_.file_bucket_id).forall(_._2.size == 1) - } - if (hasNoDeltaFile) { - logInfo(s"== Partition ${part.range_value} has no delta file.") - } else { - executeCompaction(sparkSession, tc, files, Array(part)) - if (cleanOldCompaction) { - val tablePath = snapshotManagement.table_path - cleanOldCommitOpDiskData(tablePath, null, sparkSession) - } + compactSinglePartition(sparkSession, tc, files, part) + + if (cleanOldCompaction) { + val tablePath = snapshotManagement.table_path + cleanOldCommitOpDiskData(tablePath, null, sparkSession) } }) }) @@ -210,270 +252,3 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, } } - -//// SPDX-FileCopyrightText: 2023 LakeSoul Contributors -//// -//// SPDX-License-Identifier: Apache-2.0 -// -//package org.apache.spark.sql.lakesoul.commands -// -//import com.dmetasoul.lakesoul.meta.entity.{DataCommitInfo, Uuid, CommitOp, DataFileOp, FileOp} -//import com.dmetasoul.lakesoul.meta.{DBUtil, DataFileInfo, PartitionInfoScala, SparkMetaVersion} -//import com.dmetasoul.lakesoul.spark.clean.CleanOldCompaction.cleanOldCommitOpDiskData -//import org.apache.hadoop.fs.Path -//import org.apache.spark.internal.Logging -//import org.apache.spark.sql.catalyst.expressions.PredicateHelper -//import org.apache.spark.sql.execution.command.LeafRunnableCommand -//import org.apache.spark.sql.execution.datasources.v2.merge.MergeDeltaParquetScan -//import org.apache.spark.sql.execution.datasources.v2.parquet.{NativeParquetScan, ParquetScan} -//import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} -//import org.apache.spark.sql.functions.expr -//import org.apache.spark.sql.lakesoul.catalog.LakeSoulTableV2 -//import org.apache.spark.sql.lakesoul.exception.LakeSoulErrors -//import org.apache.spark.sql.lakesoul.{BatchDataSoulFileIndexV2, LakeSoulOptions, SnapshotManagement, TransactionCommit} -//import org.apache.spark.sql.types.StructType -//import org.apache.spark.sql.util.CaseInsensitiveStringMap -//import org.apache.spark.sql.{Dataset, Row, SparkSession} -// -//import scala.collection.mutable -//import scala.collection.mutable.ArrayBuffer -//import scala.collection.JavaConverters._ -//import java.util.{UUID, ArrayList} -// -//case class CompactionCommand(snapshotManagement: SnapshotManagement, -// conditionString: String, -// force: Boolean, -// mergeOperatorInfo: Map[String, String], -// hiveTableName: String = "", -// hivePartitionName: String = "", -// cleanOldCompaction: Boolean, -// maxSnapshotsPerGroup: Option[Int] -// ) -// extends LeafRunnableCommand with PredicateHelper with Logging { -// -// def filterPartitionNeedCompact(spark: SparkSession, -// force: Boolean, -// partitionInfo: PartitionInfoScala): Boolean = { -// partitionInfo.read_files.length >= 1 -// } -// -// def executeCompaction(spark: SparkSession, tc: TransactionCommit, files: Seq[DataFileInfo], readPartitionInfo: Array[PartitionInfoScala]): DataCommitInfo = { -// -// val fileIndex = BatchDataSoulFileIndexV2(spark, snapshotManagement, files) -// val table = LakeSoulTableV2( -// spark, -// new Path(snapshotManagement.table_path), -// None, -// None, -// Option(fileIndex), -// Option(mergeOperatorInfo) -// ) -// val option = new CaseInsensitiveStringMap( -// Map("basePath" -> tc.tableInfo.table_path_s.get, "isCompaction" -> "true").asJava) -// -// val partitionNames = readPartitionInfo.head.range_value.split(',').map(p => { -// p.split('=').head -// }) -// -// val scan = table.newScanBuilder(option).build() -// val newReadFiles = if (scan.isInstanceOf[ParquetScan] || scan.isInstanceOf[NativeParquetScan]) { -// fileIndex.getFileInfo(Nil) -// } else { -// scan.asInstanceOf[MergeDeltaParquetScan].newFileIndex.getFileInfo(Nil) -// } -// -// val tableSchemaWithoutPartitions = StructType(table.schema().filter(f => { -// !partitionNames.contains(f.name) -// })) -// -// val v2Relation = DataSourceV2Relation( -// table, -// tableSchemaWithoutPartitions.toAttributes, -// None, -// None, -// option -// ) -// -// val compactDF = Dataset.ofRows( -// spark, -// DataSourceV2ScanRelation( -// v2Relation, -// scan, -// tableSchemaWithoutPartitions.toAttributes -// ) -// ) -// -// tc.setReadFiles(newReadFiles) -// val map = mutable.HashMap[String, String]() -// map.put("isCompaction", "true") -// if (readPartitionInfo.nonEmpty) { -// map.put("partValue", readPartitionInfo.head.range_value) -// } -// val (newFiles, path) = tc.writeFiles(compactDF, Some(new LakeSoulOptions(map.toMap, spark.sessionState.conf)), isCompaction = true) -// tc.createDataCommitInfo(newFiles, Seq.empty, readPartitionInfo) -// -// val fileOps = new ArrayList[DataFileOp]() -// newFiles.foreach { file => -// fileOps.add(DataFileOp.newBuilder() -// .setPath(file.path) -// .setFileOp(FileOp.add) -// .setSize(file.size) -// .setFileExistCols(file.file_exist_cols) -// .build()) -// } -// -// val commitId = DBUtil.toProtoUuid(UUID.randomUUID()) -// -// DataCommitInfo.newBuilder() -// .setTableId(tc.tableInfo.table_id) -// .setPartitionDesc(readPartitionInfo.head.range_value) -// .setCommitId(commitId) -// .addAllFileOps(fileOps) -// .setCommitOp(CommitOp.CompactionCommit) -// .setTimestamp(System.currentTimeMillis()) -// .setCommitted(false) -// .setDomain("") // 如果需要设置 domain,请替换为适当的值 -// .build() -// } -// -// def escapeSingleBackQuotedString(str: String): String = { -// val builder = mutable.StringBuilder.newBuilder -// -// str.foreach { -// case '\'' => "" -// case '`' => "" -// case ch => builder += ch -// } -// -// builder.toString() -// } -// -// override def run(sparkSession: SparkSession): Seq[Row] = { -// val condition = conditionString match { -// case "" => None -// case _: String => Option(expr(conditionString).expr) -// } -// //when condition is defined, only one partition need compaction, -// //else we will check whole table -// if (condition.isDefined) { -// val targetOnlyPredicates = -// splitConjunctivePredicates(condition.get) -// -// snapshotManagement.withNewTransaction(tc => { -// tc.setCommitType("compaction") -// val files = tc.filterFiles(targetOnlyPredicates) -// //ensure only one partition execute compaction command -// val partitionSet = files.map(_.range_partitions).toSet -// if (partitionSet.isEmpty) { -// throw LakeSoulErrors.partitionColumnNotFoundException(condition.get, 0) -// } else if (partitionSet.size > 1) { -// throw LakeSoulErrors.partitionColumnNotFoundException(condition.get, partitionSet.size) -// } -// -// lazy val hasNoDeltaFile = if (force) { -// false -// } else { -// files.groupBy(_.file_bucket_id).forall(_._2.size == 1) -// } -// -// val groupedFiles = if (maxSnapshotsPerGroup.isDefined) { -// val groupSize = maxSnapshotsPerGroup.get -// var groupedFiles = new ArrayBuffer[Seq[DataFileInfo]] -// for (i <- files.indices by groupSize) { -// groupedFiles += files.slice(i, i + groupSize) -// } -// groupedFiles.toSeq -// } else { -// Seq(files) -// } -// -// -// val allDataCommitInfo = groupedFiles.flatMap(files => { -// if (!hasNoDeltaFile) { -// val partitionInfo = SparkMetaVersion.getSinglePartitionInfo( -// snapshotManagement.getTableInfoOnly.table_id, -// partitionSet.head, -// "" -// ) -// if (partitionInfo.commit_op.equals("CompactionCommit") && partitionInfo.read_files.length == 1) { -// logInfo("=========== All Partitions Have Compacted, This Operation Will Cancel!!! ===========") -// None -// } else { -// Some(executeCompaction(sparkSession, tc, files, Array(partitionInfo))) -// } -// } else { -// logInfo("== Compaction: This partition has been compacted or has no delta file.") -// None -// } -// }) -// -// // 提交所有的 DataCommitInfo -// tc.commitDataCommitInfo(allDataCommitInfo.toList, Array.empty[PartitionInfoScala].toList, "", 0, Array.empty[PartitionInfoScala]) -// -// if (cleanOldCompaction) { -// val tablePath = snapshotManagement.table_path -// cleanOldCommitOpDiskData(tablePath, partitionSet.head, sparkSession) -// } -// -// }) -// } else { -// -// val allInfo = SparkMetaVersion.getAllPartitionInfo(snapshotManagement.getTableInfoOnly.table_id) -// val partitionsNeedCompact = allInfo -// .filter(filterPartitionNeedCompact(sparkSession, force, _)) -// -// partitionsNeedCompact.foreach(part => { -// snapshotManagement.withNewTransaction(tc => { -// tc.setCommitType("compaction") -// val files = tc.getCompactionPartitionFiles(part) -// lazy val hasNoDeltaFile = if (force) { -// false -// } else { -// files.groupBy(_.file_bucket_id).forall(_._2.size == 1) -// } -// -// val groupedFiles = if (maxSnapshotsPerGroup.isDefined) { -// val groupSize = maxSnapshotsPerGroup.get -// var groupedFiles = new ArrayBuffer[Seq[DataFileInfo]] -// for (i <- files.indices by groupSize) { -// groupedFiles += files.slice(i, i + groupSize) -// } -// groupedFiles.toSeq -// } else { -// Seq(files) -// } -// -// -// val allDataCommitInfo = groupedFiles.flatMap(files => { -// if (!hasNoDeltaFile) { -// val partitionInfo = SparkMetaVersion.getSinglePartitionInfo( -// snapshotManagement.getTableInfoOnly.table_id, -// part.range_value, -// "" -// ) -// if (partitionInfo.commit_op.equals("CompactionCommit") && partitionInfo.read_files.length == 1) { -// logInfo("=========== All Partitions Have Compacted, This Operation Will Cancel!!! ===========") -// None -// } else { -// Some(executeCompaction(sparkSession, tc, files, Array(partitionInfo))) -// } -// } else { -// logInfo(s"== Partition ${part.range_value} has no delta file.") -// None -// } -// }) -// -// // 提交所有的 DataCommitInfo -// tc.commitDataCommitInfo(allDataCommitInfo.toList, Array.empty[PartitionInfoScala].toList, "", 0, Array(part)) -// -// if (cleanOldCompaction) { -// val tablePath = snapshotManagement.table_path -// cleanOldCommitOpDiskData(tablePath, part.range_value, sparkSession) -// } -// }) -// }) -// } -// Seq.empty -// } -// -//} diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala index 6efe295cd..af2f3cdc4 100644 --- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala +++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala @@ -4,7 +4,8 @@ package org.apache.spark.sql.lakesoul.commands -import com.dmetasoul.lakesoul.meta.SparkMetaVersion +import com.dmetasoul.lakesoul.meta.LakeSoulOptions.SHORT_TABLE_NAME +import com.dmetasoul.lakesoul.meta.{DataOperation, SparkMetaVersion} import com.dmetasoul.lakesoul.tables.LakeSoulTable import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf @@ -437,6 +438,7 @@ class CompactionSuite extends QueryTest "v1" -> new MergeOpInt(), "v2" -> "org.apache.spark.sql.lakesoul.test.MergeOpString") table.compaction(true, mergeOperatorInfo, true) + LakeSoulTable.uncached(tableName) checkAnswer(table.toDF.select("range", "hash", "v1", "v2"), result) }) @@ -503,12 +505,18 @@ class CompactionSuite extends QueryTest } } - test("compaction with limited file group") { + test("compaction with limited file") { withTempDir { tempDir => + // val tempDir = Utils.createDirectory(System.getProperty("java.io.tmpdir")) val tablePath = tempDir.getCanonicalPath val spark = SparkSession.active - // 创建测试数据 + val hashBucketNum = 4 + val compactRounds = 5 + val dataPerRounds = 10 + val compactGroupSize = 3 + + // Create test data val df = Seq( (1, "2023-01-01", 10, 1), (2, "2023-01-02", 20, 1), @@ -517,54 +525,68 @@ class CompactionSuite extends QueryTest (5, "2023-01-05", 50, 1) ).toDF("id", "date", "value", "range") - // 写入初始数据 + // Write initial data df.write .format("lakesoul") .option("rangePartitions", "range") .option("hashPartitions", "id") - .option("hashBucketNum", "4") + .option(SHORT_TABLE_NAME, "compaction_limit_table") + .option("hashBucketNum", hashBucketNum.toString) .save(tablePath) val lakeSoulTable = LakeSoulTable.forPath(tablePath) - // 模拟多次追加操作 - for (i <- 1 to 10) { - val appendDf = Seq( - (i * 10, s"2023-02-0$i", i * 100, 1) - ).toDF("id", "date", "value", "range") - lakeSoulTable.upsert(appendDf) - } + for (c <- 0 until compactRounds) { + // Simulate multiple append operations + for (i <- c * dataPerRounds + 1 to (c + 1) * dataPerRounds) { + val appendDf = Seq( + (i * 10, s"2023-02-0$i", i * 100, 1) + ).toDF("id", "date", "value", "range") + lakeSoulTable.upsert(appendDf) + } + + // Get initial PartitionInfo count + val initialFileCount = getFileCount(tablePath) + println(s"before compact initialPartitionInfoCount=$initialFileCount") + lakeSoulTable.toDF.show - // 获取初始的PartitionInfo数量 - val initialPartitionInfoCount = getSnapshotCount(tablePath) + // Perform limited compaction (group every 3 PartitionInfo) + lakeSoulTable.compaction(condition = "", force = true, mergeOperatorInfo = Map.empty, + hiveTableName = "", hivePartitionName = "", cleanOldCompaction = false, + fileNumLimit = Some(compactGroupSize)) - // 执行受限的Compaction操作 (每3个PartitionInfo一组) - lakeSoulTable.compaction(condition = "", force = true, mergeOperatorInfo = Map.empty, - hiveTableName = "", hivePartitionName = "", cleanOldCompaction = false, - maxSnapshotsPerGroup = None) + // Get PartitionInfo count after compaction + val compactedFileCount = getFileCount(tablePath) - // 获取Compaction后的PartitionInfo数量 - val compactedPartitionInfoCount = getSnapshotCount(tablePath) + println(s"after compact compactedPartitionInfoCount=$compactedFileCount") - // 验证结果 - assert(compactedPartitionInfoCount < initialPartitionInfoCount, - s"Compaction应该减少PartitionInfo的数量,但是从${initialPartitionInfoCount}变为$compactedPartitionInfoCount") + lakeSoulTable.toDF.show - assert(compactedPartitionInfoCount > 1, - s"Compaction应该产生多个PartitionInfo组,但只有${compactedPartitionInfoCount}个") + // Verify results + assert(compactedFileCount < initialFileCount, + s"Compaction should reduce the number of files, but it changed from ${initialFileCount} to $compactedFileCount") + + + assert(compactedFileCount >= (initialFileCount - 1) / compactGroupSize + 1, + s"Compaction should produce files above a lower bound, but there are ${compactedFileCount} files") + + assert(compactedFileCount <= (initialFileCount - 1) / compactGroupSize + 1 + hashBucketNum, + s"Compaction should produce files below a upper bound, but there are ${compactedFileCount} files") + } - // 验证数据完整性 - val compactedData = lakeSoulTable.toDF.orderBy("date", "id").collect() - assert(compactedData.length == 15, s"压缩后的数据应该有15行,但实际有${compactedData.length}行") + // Verify data integrity + val compactedData = lakeSoulTable.toDF.orderBy("id", "date").collect() + println(compactedData.mkString("Array(", ", ", ")")) + assert(compactedData.length == 5 + dataPerRounds * compactRounds, s"The compressed data should have ${5 + dataPerRounds * compactRounds} rows, but it actually has ${compactedData.length} rows") } } - // 辅助方法:获取Snapshot的数量 - def getSnapshotCount(tablePath: String): Int = { + // Auxiliary method: Get the number of files + def getFileCount(tablePath: String): Int = { val sm = SnapshotManagement(SparkUtil.makeQualifiedTablePath(new Path(tablePath)).toString) - var count = 0 - SparkMetaVersion.getAllPartitionInfo(sm.getTableInfoOnly.table_id).foreach(p => count += p.read_files.length) - count + val partitionList = SparkMetaVersion.getAllPartitionInfo(sm.getTableInfoOnly.table_id) + val files = DataOperation.getTableDataInfo(partitionList) + files.length } } From b9cbcb087e64cdf35e0a04ddbdf47163d74a49de Mon Sep 17 00:00:00 2001 From: maosen Date: Thu, 17 Oct 2024 14:39:42 +0800 Subject: [PATCH 14/33] [WIP]support compaction with changing bucket num Signed-off-by: zenghua --- .../lakesoul/tables/LakeSoulTable.scala | 32 ++++----- .../execution/LakeSoulTableOperations.scala | 6 +- .../datasources/LakeSoulFileWriter.scala | 8 ++- .../v2/parquet/NativeParquetFileFormat.scala | 3 +- .../sql/lakesoul/TransactionalWrite.scala | 22 ++++++- .../lakesoul/commands/CompactionCommand.scala | 35 ++++++++-- .../lakesoul/commands/CompactionSuite.scala | 66 ++++++++++++++++++- 7 files changed, 142 insertions(+), 30 deletions(-) diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala index ee9bc9c4c..db28a60c1 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala @@ -307,63 +307,63 @@ class LakeSoulTable(df: => Dataset[Row], snapshotManagement: SnapshotManagement) //by default, force perform compaction on whole table def compaction(): Unit = { - compaction("", true, Map.empty[String, Any], "", "", false, None) + compaction("", true, Map.empty[String, Any], "", "", false, None, None) } def compaction(fileNumLimit: Option[Int]): Unit = { - compaction("", true, Map.empty[String, Any], "", "", false, fileNumLimit) + compaction("", true, Map.empty[String, Any], "", "", false, fileNumLimit, None) } def compaction(cleanOldCompaction: Boolean): Unit = { - compaction("", true, Map.empty[String, Any], "", "", cleanOldCompaction, None) + compaction("", true, Map.empty[String, Any], "", "", cleanOldCompaction, None, None) } def compaction(condition: String): Unit = { - compaction(condition, true, Map.empty[String, Any], "", "", false, None) + compaction(condition, true, Map.empty[String, Any], "", "", false, None, None) } def compaction(condition: String, cleanOldCompaction: Boolean): Unit = { - compaction(condition, true, Map.empty[String, Any], "", "", cleanOldCompaction, None) + compaction(condition, true, Map.empty[String, Any], "", "", cleanOldCompaction, None, None) } def compaction(condition: String, cleanOldCompaction: Boolean, fileNumLimit: Option[Int]): Unit = { - compaction(condition, true, Map.empty[String, Any], "", "", cleanOldCompaction, fileNumLimit) + compaction(condition, true, Map.empty[String, Any], "", "", cleanOldCompaction, fileNumLimit, None) } def compaction(mergeOperatorInfo: Map[String, Any]): Unit = { - compaction("", true, mergeOperatorInfo, "", "", false, None) + compaction("", true, mergeOperatorInfo, "", "", false, None, None) } def compaction(condition: String, mergeOperatorInfo: Map[String, Any]): Unit = { - compaction(condition, true, mergeOperatorInfo, "", "", false, None) + compaction(condition, true, mergeOperatorInfo, "", "", false, None, None) } def compaction(condition: String, hiveTableName: String): Unit = { - compaction(condition, true, Map.empty[String, Any], hiveTableName, "", false, None) + compaction(condition, true, Map.empty[String, Any], hiveTableName, "", false, None, None) } def compaction(condition: String, hiveTableName: String, hivePartitionName: String): Unit = { - compaction(condition, true, Map.empty[String, Any], hiveTableName, hivePartitionName, false, None) + compaction(condition, true, Map.empty[String, Any], hiveTableName, hivePartitionName, false, None, None) } def compaction(force: Boolean, mergeOperatorInfo: Map[String, Any] = Map.empty[String, Any], cleanOldCompaction: Boolean): Unit = { - compaction("", force, mergeOperatorInfo, "", "", cleanOldCompaction, None) + compaction("", force, mergeOperatorInfo, "", "", cleanOldCompaction, None, None) } def compaction(condition: String, force: Boolean, cleanOldCompaction: Boolean): Unit = { - compaction(condition, true, Map.empty[String, Any], "", "", cleanOldCompaction, None) + compaction(condition, true, Map.empty[String, Any], "", "", cleanOldCompaction, None, None) } def compaction(condition: String, force: Boolean, mergeOperatorInfo: java.util.Map[String, Any], cleanOldCompaction: Boolean): Unit = { - compaction(condition, force, mergeOperatorInfo.asScala.toMap, "", "", cleanOldCompaction, None) + compaction(condition, force, mergeOperatorInfo.asScala.toMap, "", "", cleanOldCompaction, None, None) } def compaction(condition: String, @@ -372,7 +372,9 @@ class LakeSoulTable(df: => Dataset[Row], snapshotManagement: SnapshotManagement) hiveTableName: String, hivePartitionName: String, cleanOldCompaction: Boolean, - fileNumLimit: Option[Int]): Unit = { + fileNumLimit: Option[Int], + newBucketNum: Option[Int] + ): Unit = { val newMergeOpInfo = mergeOperatorInfo.map(m => { val key = if (!m._1.startsWith(LakeSoulUtils.MERGE_OP_COL)) { @@ -388,7 +390,7 @@ class LakeSoulTable(df: => Dataset[Row], snapshotManagement: SnapshotManagement) (key, value) }) - executeCompaction(df, snapshotManagement, condition, force, newMergeOpInfo, hiveTableName, hivePartitionName, cleanOldCompaction, fileNumLimit) + executeCompaction(df, snapshotManagement, condition, force, newMergeOpInfo, hiveTableName, hivePartitionName, cleanOldCompaction, fileNumLimit, newBucketNum) } def setCompactionTtl(days: Int): LakeSoulTable = { diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala index 155f5b212..28f0d7e5b 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala @@ -174,7 +174,8 @@ trait LakeSoulTableOperations extends AnalysisHelper { hiveTableName: String = "", hivePartitionName: String = "", cleanOldCompaction: Boolean, - fileNumLimit: Option[Int]): Unit = { + fileNumLimit: Option[Int], + newBucketNum: Option[Int]): Unit = { toDataset(sparkSession, CompactionCommand( snapshotManagement, condition, @@ -183,7 +184,8 @@ trait LakeSoulTableOperations extends AnalysisHelper { hiveTableName, hivePartitionName, cleanOldCompaction, - fileNumLimit + fileNumLimit, + newBucketNum )) } diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala index 63cd2f203..4c905476a 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala @@ -145,6 +145,7 @@ object LakeSoulFileWriter extends Logging { val isCompaction = caseInsensitiveOptions.getOrElse("isCompaction", "false").toBoolean val staticBucketId = caseInsensitiveOptions.getOrElse("staticBucketId", "-1").toInt + val isBucketNumChanged = caseInsensitiveOptions.getOrElse("isBucketNumChanged", "false").toBoolean // We should first sort by partition columns, then bucket id, and finally sorting columns. val requiredOrdering = partitionColumns ++ writerBucketSpec.map(_.bucketIdExpression) ++ sortColumns @@ -174,7 +175,7 @@ object LakeSoulFileWriter extends Logging { val nativeIOEnable = sparkSession.sessionState.conf.getConf(LakeSoulSQLConf.NATIVE_IO_ENABLE) def nativeWrap(plan: SparkPlan): RDD[InternalRow] = { - if (isCompaction && !isCDC && nativeIOEnable) { + if (isCompaction && !isCDC && !isBucketNumChanged && nativeIOEnable) { plan match { case withPartitionAndOrdering(_, _, child) => return nativeWrap(child) @@ -198,7 +199,7 @@ object LakeSoulFileWriter extends Logging { try { // for compaction, we won't break ordering from batch scan - val (rdd, concurrentOutputWriterSpec) = if (orderingMatched || isCompaction) { + val (rdd, concurrentOutputWriterSpec) = if (!isBucketNumChanged && (orderingMatched || isCompaction)) { (nativeWrap(empty2NullPlan), None) } else { // SPARK-21165: the `requiredOrdering` is based on the attributes from analyzed plan, and @@ -308,13 +309,14 @@ object LakeSoulFileWriter extends Logging { committer.setupTask(taskAttemptContext) val isCompaction = options.getOrElse("isCompaction", "false").toBoolean + val isBucketNumChanged = options.getOrElse("isBucketNumChanged", "false").toBoolean val dataWriter = if (!iterator.hasNext) { new EmptyDirectoryDataWriter(description, taskAttemptContext, committer) } else if (description.partitionColumns.isEmpty && description.bucketSpec.isEmpty && !isCompaction) { new SingleDirectoryDataWriter(description, taskAttemptContext, committer) - } else if (isCompaction) { + } else if (isCompaction && !isBucketNumChanged) { new StaticPartitionedDataWriter(description, taskAttemptContext, committer, options, sparkPartitionId, bucketSpec) } else { concurrentOutputWriterSpec match { diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetFileFormat.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetFileFormat.scala index ee173f004..e47044cb5 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetFileFormat.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetFileFormat.scala @@ -30,7 +30,8 @@ class NativeParquetFileFormat extends FileFormat .getOrElse(DateTimeUtils.TIMEZONE_OPTION, sparkSession.sessionState.conf.sessionLocalTimeZone) if (options.getOrElse("isCompaction", "false").toBoolean && - !options.getOrElse("isCDC", "false").toBoolean + !options.getOrElse("isCDC", "false").toBoolean && + !options.getOrElse("isBucketNumChanged", "false").toBoolean ) { new OutputWriterFactory { override def newInstance( diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala index aee3c0056..dbf43187e 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala @@ -103,10 +103,22 @@ trait TransactionalWrite { // LakeSoul always writes timestamp data with timezone=UTC spark.conf.set("spark.sql.session.timeZone", "UTC") spark.sharedState.cacheManager.uncacheQuery(oriData, true) + + val newBucketNum = if (writeOptions.isDefined) { + writeOptions.get.options.get("newBucketNum").map(_.toInt) + } else { + None + } + val bucketNumChanged = newBucketNum.exists(tableInfo.bucket_num != _) + val data = Dataset.ofRows(spark, (if (!isCompaction && tableInfo.hash_partition_columns.nonEmpty) { oriData.repartition(tableInfo.bucket_num, tableInfo.hash_partition_columns.map(col): _*) } else { - oriData + if (isCompaction && newBucketNum.exists(tableInfo.bucket_num.equals) && tableInfo.hash_partition_columns.nonEmpty) { + oriData.repartition(newBucketNum.get, tableInfo.hash_partition_columns.map(col): _*) + } else { + oriData + } }).logicalPlan) hasWritten = true @@ -137,6 +149,11 @@ trait TransactionalWrite { outputPath = SparkUtil.makeQualifiedTablePath(new Path(compactPath)) } val dc = if (isCompaction) { + if (bucketNumChanged) { + options.put("isBucketNumChanged", "true") + } else { + options.put("isBucketNumChanged", "false") + } val cdcCol = snapshot.getTableInfo.configuration.get(LakeSoulTableProperties.lakeSoulCDCChangePropKey) if (cdcCol.nonEmpty) { options.put("isCDC", "true") @@ -203,6 +220,9 @@ trait TransactionalWrite { val hashBucketSpec = tableInfo.hash_column match { case "" => None + case _ if bucketNumChanged => Option(BucketSpec(newBucketNum.get, + tableInfo.hash_partition_columns, + tableInfo.hash_partition_columns)) case _ => Option(BucketSpec(tableInfo.bucket_num, tableInfo.hash_partition_columns, tableInfo.hash_partition_columns)) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala index 3a3a7408f..7986099c9 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala @@ -4,6 +4,8 @@ package org.apache.spark.sql.lakesoul.commands +import com.alibaba.fastjson.JSON +import com.dmetasoul.lakesoul.meta.DBConfig.TableInfoProperty import com.dmetasoul.lakesoul.meta.entity.DataCommitInfo import com.dmetasoul.lakesoul.meta.{DBUtil, DataFileInfo, PartitionInfoScala, SparkMetaVersion} import com.dmetasoul.lakesoul.spark.clean.CleanOldCompaction.cleanOldCommitOpDiskData @@ -18,6 +20,7 @@ import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, Data import org.apache.spark.sql.functions.expr import org.apache.spark.sql.lakesoul.catalog.LakeSoulTableV2 import org.apache.spark.sql.lakesoul.exception.LakeSoulErrors +import org.apache.spark.sql.lakesoul.utils.TableInfo import org.apache.spark.sql.lakesoul.{BatchDataSoulFileIndexV2, LakeSoulOptions, SnapshotManagement, TransactionCommit} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -27,6 +30,7 @@ import org.apache.spark.util.Utils import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConverters._ +import scala.util.parsing.json.JSON.headOptionTailToFunList case class CompactionCommand(snapshotManagement: SnapshotManagement, conditionString: String, @@ -35,11 +39,16 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, hiveTableName: String = "", hivePartitionName: String = "", cleanOldCompaction: Boolean, - fileNumLimit: Option[Int] = None + fileNumLimit: Option[Int] = None, + newBucketNum: Option[Int] = None ) extends LeafRunnableCommand with PredicateHelper with Logging { - val compactPath = snapshotManagement.getTableInfoOnly.table_path.toString + "/compact_" + System.currentTimeMillis() + lazy val compactPath: String = tableInfo.table_path.toString + "/compact_" + System.currentTimeMillis() + + lazy val bucketNumChanged: Boolean = newBucketNum.exists(tableInfo.bucket_num != _) + + lazy val tableInfo: TableInfo = snapshotManagement.getTableInfoOnly def filterPartitionNeedCompact(spark: SparkSession, force: Boolean, @@ -95,6 +104,7 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, tableSchemaWithoutPartitions.toAttributes ) ) + compactDF.show tc.setReadFiles(newReadFiles) val map = mutable.HashMap[String, String]() @@ -109,6 +119,11 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, map.put("staticBucketId", headBucketId.toString) } } + if (bucketNumChanged) { + map.put("newBucketNum", newBucketNum.get.toString) + } + println(map) + val (newFiles, path) = tc.writeFiles(compactDF, Some(new LakeSoulOptions(map.toMap, spark.sessionState.conf)), isCompaction = true) tc.createDataCommitInfo(newFiles, Seq.empty, "", -1)._1 @@ -132,7 +147,6 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, } else { files.groupBy(_.file_bucket_id) } - println(bucketedFiles) val allDataCommitInfo = bucketedFiles.flatMap(groupByBucketId => { val (bucketId, files) = groupByBucketId val groupedFiles = if (fileNumLimit.isDefined) { @@ -215,7 +229,7 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, } val partitionInfo = SparkMetaVersion.getSinglePartitionInfo( - snapshotManagement.getTableInfoOnly.table_id, + tableInfo.table_id, partitionSet.head, "" ) @@ -230,7 +244,7 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, }) } else { - val allInfo = SparkMetaVersion.getAllPartitionInfo(snapshotManagement.getTableInfoOnly.table_id) + val allInfo = SparkMetaVersion.getAllPartitionInfo(tableInfo.table_id) val partitionsNeedCompact = allInfo .filter(filterPartitionNeedCompact(sparkSession, force, _)) @@ -248,7 +262,16 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, }) }) } + if (bucketNumChanged) { + snapshotManagement.withNewTransaction(tc => { + val newConfiguration = new mutable.HashMap[String, String] + newConfiguration ++= tableInfo.configuration + newConfiguration.put(TableInfoProperty.HASH_BUCKET_NUM, newBucketNum.get.toString) + val newTableInfo = tableInfo.copy(configuration = newConfiguration.toMap) + tc.updateTableInfo(newTableInfo) + }) + } Seq.empty } -} +} \ No newline at end of file diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala index af2f3cdc4..9ef9d24da 100644 --- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala +++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala @@ -18,6 +18,7 @@ import org.apache.spark.sql.lakesoul.test.{LakeSoulSQLCommandTest, LakeSoulTestS import org.apache.spark.sql.lakesoul.utils.SparkUtil import org.apache.spark.sql.test.{SharedSparkSession, TestSparkSession} import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SparkSession} +import org.apache.spark.util.Utils import org.junit.runner.RunWith import org.scalatest.BeforeAndAfterEach import org.scalatestplus.junit.JUnitRunner @@ -550,10 +551,10 @@ class CompactionSuite extends QueryTest println(s"before compact initialPartitionInfoCount=$initialFileCount") lakeSoulTable.toDF.show - // Perform limited compaction (group every 3 PartitionInfo) + // Perform limited compaction (group every compactGroupSize PartitionInfo) lakeSoulTable.compaction(condition = "", force = true, mergeOperatorInfo = Map.empty, hiveTableName = "", hivePartitionName = "", cleanOldCompaction = false, - fileNumLimit = Some(compactGroupSize)) + fileNumLimit = Some(compactGroupSize), newBucketNum = None) // Get PartitionInfo count after compaction val compactedFileCount = getFileCount(tablePath) @@ -589,4 +590,65 @@ class CompactionSuite extends QueryTest files.length } + test("compaction with newBucketNum") { + // withTempDir { tempDir => + val tempDir = Utils.createDirectory(System.getProperty("java.io.tmpdir")) + val tablePath = tempDir.getCanonicalPath + val spark = SparkSession.active + + val hashBucketNum = 4 + val newHashBucketNum = 7 + val compactRounds = 5 + val dataPerRounds = 10 + val compactGroupSize = 3 + + // Create test data + val df = Seq( + (1, "2023-01-01", 10, 1), + (2, "2023-01-02", 20, 1), + (3, "2023-01-03", 30, 1), + (4, "2023-01-04", 40, 1), + (5, "2023-01-05", 50, 1) + ).toDF("id", "date", "value", "range") + + // Write initial data + df.write + .format("lakesoul") + .option("rangePartitions", "range") + .option("hashPartitions", "id") + .option(SHORT_TABLE_NAME, "rebucket_table") + .option("hashBucketNum", hashBucketNum.toString) + .save(tablePath) + + val lakeSoulTable = LakeSoulTable.forPath(tablePath) + + for (i <- 1 to 100) { + val appendDf = Seq( + (i * 10, s"2023-02-0$i", i * 100, 1) + ).toDF("id", "date", "value", "range") + lakeSoulTable.upsert(appendDf) + } + assert(getFileBucketSet(tablePath).size == hashBucketNum) + + lakeSoulTable.compaction(condition = "", force = true, mergeOperatorInfo = Map.empty, + hiveTableName = "", hivePartitionName = "", cleanOldCompaction = false, + fileNumLimit = None, newBucketNum = Some(newHashBucketNum)) + + assert(getFileBucketSet(tablePath).size == newHashBucketNum) + val compactedData = lakeSoulTable.toDF.orderBy("id", "date").collect() + println(compactedData.mkString("Array(", ", ", ")")) + assert(compactedData.length == 105, s"The compressed data should have ${105} rows, but it actually has ${compactedData.length} rows") + + // } + } + + // Auxiliary method: Get the bucket number of table + def getFileBucketSet(tablePath: String): Set[Int] = { + val sm = SnapshotManagement(SparkUtil.makeQualifiedTablePath(new Path(tablePath)).toString) + val partitionList = SparkMetaVersion.getAllPartitionInfo(sm.getTableInfoOnly.table_id) + val files = DataOperation.getTableDataInfo(partitionList) + println(files.mkString("Array(", ", ", ")")) + files.groupBy(_.file_bucket_id).keys.toSet + } + } From da187f62a4f09324e13227b23d62f476441b726b Mon Sep 17 00:00:00 2001 From: zenghua Date: Thu, 17 Oct 2024 20:22:22 +0800 Subject: [PATCH 15/33] fix changing bucket num Signed-off-by: zenghua --- .../lakesoul/tables/LakeSoulTable.scala | 70 ++-------- .../datasources/LakeSoulFileWriter.scala | 2 +- .../sql/lakesoul/TransactionalWrite.scala | 2 +- .../lakesoul/commands/CallExecCommand.scala | 6 +- .../lakesoul/commands/CompactionCommand.scala | 16 +-- .../commands/CleanOldCompactionSuite.scala | 12 +- .../lakesoul/commands/CompactionSuite.scala | 122 ++++++++++-------- .../CompactionDoNotChangeResult.scala | 2 +- 8 files changed, 93 insertions(+), 139 deletions(-) diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala index db28a60c1..51fbc81cf 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala @@ -310,70 +310,18 @@ class LakeSoulTable(df: => Dataset[Row], snapshotManagement: SnapshotManagement) compaction("", true, Map.empty[String, Any], "", "", false, None, None) } - def compaction(fileNumLimit: Option[Int]): Unit = { - compaction("", true, Map.empty[String, Any], "", "", false, fileNumLimit, None) - } - - def compaction(cleanOldCompaction: Boolean): Unit = { - compaction("", true, Map.empty[String, Any], "", "", cleanOldCompaction, None, None) - } - def compaction(condition: String): Unit = { - compaction(condition, true, Map.empty[String, Any], "", "", false, None, None) - } - - def compaction(condition: String, cleanOldCompaction: Boolean): Unit = { - compaction(condition, true, Map.empty[String, Any], "", "", cleanOldCompaction, None, None) - } - - def compaction(condition: String, cleanOldCompaction: Boolean, fileNumLimit: Option[Int]): Unit = { - compaction(condition, true, Map.empty[String, Any], "", "", cleanOldCompaction, fileNumLimit, None) - } - - def compaction(mergeOperatorInfo: Map[String, Any]): Unit = { - compaction("", true, mergeOperatorInfo, "", "", false, None, None) - } - - def compaction(condition: String, - mergeOperatorInfo: Map[String, Any]): Unit = { - compaction(condition, true, mergeOperatorInfo, "", "", false, None, None) - } - - def compaction(condition: String, hiveTableName: String): Unit = { - compaction(condition, true, Map.empty[String, Any], hiveTableName, "", false, None, None) - } - - def compaction(condition: String, hiveTableName: String, hivePartitionName: String): Unit = { - compaction(condition, true, Map.empty[String, Any], hiveTableName, hivePartitionName, false, None, None) - } - - def compaction(force: Boolean, - mergeOperatorInfo: Map[String, Any] = Map.empty[String, Any], - cleanOldCompaction: Boolean): Unit = { - compaction("", force, mergeOperatorInfo, "", "", cleanOldCompaction, None, None) - } - - def compaction(condition: String, - force: Boolean, - cleanOldCompaction: Boolean): Unit = { - compaction(condition, true, Map.empty[String, Any], "", "", cleanOldCompaction, None, None) - } - - def compaction(condition: String, - force: Boolean, - mergeOperatorInfo: java.util.Map[String, Any], - cleanOldCompaction: Boolean): Unit = { - compaction(condition, force, mergeOperatorInfo.asScala.toMap, "", "", cleanOldCompaction, None, None) + compaction(condition = condition, true) } - def compaction(condition: String, - force: Boolean, - mergeOperatorInfo: Map[String, Any], - hiveTableName: String, - hivePartitionName: String, - cleanOldCompaction: Boolean, - fileNumLimit: Option[Int], - newBucketNum: Option[Int] + def compaction(condition: String = "", + force: Boolean = true, + mergeOperatorInfo: Map[String, Any] = Map.empty, + hiveTableName: String = "", + hivePartitionName: String = "", + cleanOldCompaction: Boolean = false, + fileNumLimit: Option[Int] = None, + newBucketNum: Option[Int] = None, ): Unit = { val newMergeOpInfo = mergeOperatorInfo.map(m => { val key = diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala index 4c905476a..25d140fc8 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala @@ -316,7 +316,7 @@ object LakeSoulFileWriter extends Logging { new EmptyDirectoryDataWriter(description, taskAttemptContext, committer) } else if (description.partitionColumns.isEmpty && description.bucketSpec.isEmpty && !isCompaction) { new SingleDirectoryDataWriter(description, taskAttemptContext, committer) - } else if (isCompaction && !isBucketNumChanged) { + } else if (isCompaction) { new StaticPartitionedDataWriter(description, taskAttemptContext, committer, options, sparkPartitionId, bucketSpec) } else { concurrentOutputWriterSpec match { diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala index dbf43187e..72c098ca9 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala @@ -114,7 +114,7 @@ trait TransactionalWrite { val data = Dataset.ofRows(spark, (if (!isCompaction && tableInfo.hash_partition_columns.nonEmpty) { oriData.repartition(tableInfo.bucket_num, tableInfo.hash_partition_columns.map(col): _*) } else { - if (isCompaction && newBucketNum.exists(tableInfo.bucket_num.equals) && tableInfo.hash_partition_columns.nonEmpty) { + if (isCompaction && bucketNumChanged && tableInfo.hash_partition_columns.nonEmpty) { oriData.repartition(newBucketNum.get, tableInfo.hash_partition_columns.map(col): _*) } else { oriData diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CallExecCommand.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CallExecCommand.scala index b0001dc66..f13c133ca 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CallExecCommand.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CallExecCommand.scala @@ -42,7 +42,7 @@ case class CallExecCommand(action: String, args: Seq[CallArgument]) extends Leaf val table = getLakeSoulTable(argsMap) if (argsMap.contains(toVersion)) { table.rollbackPartition(getPartitionVal(argsMap(parValue)), argsMap(toVersion).toString().toInt) - }else if (argsMap.contains(zoneId)) { + } else if (argsMap.contains(zoneId)) { table.rollbackPartition(getPartitionVal(argsMap(parValue)), argsMap(toTime).toString(), argsMap(zoneId).toString()) } else { table.rollbackPartition(getPartitionVal(argsMap(parValue)), argsMap(toTime).toString()) @@ -57,10 +57,10 @@ case class CallExecCommand(action: String, args: Seq[CallArgument]) extends Leaf "" } if (argsMap.contains(hiveTableName)) { - table.compaction(conditons, argsMap(hiveTableName).toString()) + table.compaction(conditons, hiveTableName = argsMap(hiveTableName).toString()) } if (argsMap.contains(cleanOld)) { - table.compaction(argsMap(cleanOld).toString().toBoolean) + table.compaction(cleanOldCompaction = argsMap(cleanOld).toString().toBoolean) } table.compaction(conditons) } diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala index 7986099c9..d15d8a7c7 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala @@ -17,7 +17,7 @@ import org.apache.spark.sql.execution.command.LeafRunnableCommand import org.apache.spark.sql.execution.datasources.v2.merge.MergeDeltaParquetScan import org.apache.spark.sql.execution.datasources.v2.parquet.{NativeParquetScan, ParquetScan} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} -import org.apache.spark.sql.functions.expr +import org.apache.spark.sql.functions.{expr, forall} import org.apache.spark.sql.lakesoul.catalog.LakeSoulTableV2 import org.apache.spark.sql.lakesoul.exception.LakeSoulErrors import org.apache.spark.sql.lakesoul.utils.TableInfo @@ -30,7 +30,6 @@ import org.apache.spark.util.Utils import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConverters._ -import scala.util.parsing.json.JSON.headOptionTailToFunList case class CompactionCommand(snapshotManagement: SnapshotManagement, conditionString: String, @@ -104,7 +103,6 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, tableSchemaWithoutPartitions.toAttributes ) ) - compactDF.show tc.setReadFiles(newReadFiles) val map = mutable.HashMap[String, String]() @@ -113,16 +111,15 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, if (readPartitionInfo.nonEmpty) { map.put("partValue", readPartitionInfo.head.range_value) } - if (tc.tableInfo.hash_partition_columns.nonEmpty) { + if (bucketNumChanged) { + map.put("newBucketNum", newBucketNum.get.toString) + } else if (tableInfo.hash_partition_columns.nonEmpty) { val headBucketId = files.head.file_bucket_id if (files.forall(_.file_bucket_id == headBucketId)) { map.put("staticBucketId", headBucketId.toString) } } - if (bucketNumChanged) { - map.put("newBucketNum", newBucketNum.get.toString) - } - println(map) + logInfo(s"write CompactData with Option=$map") val (newFiles, path) = tc.writeFiles(compactDF, Some(new LakeSoulOptions(map.toMap, spark.sessionState.conf)), isCompaction = true) @@ -142,7 +139,8 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, } def compactSinglePartition(sparkSession: SparkSession, tc: TransactionCommit, files: Seq[DataFileInfo], sourcePartition: PartitionInfoScala) = { - val bucketedFiles = if (tc.tableInfo.hash_partition_columns.isEmpty) { + logInfo(s"Compacting Single Partition=${sourcePartition} with ${files.length} files") + val bucketedFiles = if (tableInfo.hash_partition_columns.isEmpty || bucketNumChanged) { Seq(-1 -> files) } else { files.groupBy(_.file_bucket_id) diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CleanOldCompactionSuite.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CleanOldCompactionSuite.scala index ba2c0c033..474b5d1d4 100644 --- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CleanOldCompactionSuite.scala +++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CleanOldCompactionSuite.scala @@ -104,7 +104,7 @@ class CleanOldCompactionSuite extends QueryTest .toDF("date", "id", "value") LakeSoulTable.forPath(tableName).upsert(df1) - LakeSoulTable.forPath(tableName).compaction(true) + LakeSoulTable.forPath(tableName).compaction(cleanOldCompaction = true) val sm = SnapshotManagement(SparkUtil.makeQualifiedTablePath(new Path(tableName)).toString) val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) @@ -135,7 +135,7 @@ class CleanOldCompactionSuite extends QueryTest .toDF("date", "id", "value") LakeSoulTable.forPath(tableName).upsert(df1) - LakeSoulTable.forPath(tableName).compaction(true) + LakeSoulTable.forPath(tableName).compaction(cleanOldCompaction = true) val sm = SnapshotManagement(SparkUtil.makeQualifiedTablePath(new Path(tableName)).toString) assert(LakeSoulTable.forPath(tableName).toDF.count() == 4) val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) @@ -171,7 +171,7 @@ class CleanOldCompactionSuite extends QueryTest .toDF("date", "id", "value") LakeSoulTable.forPath(tableName).upsert(df1) setPartitionInfoTimestamp(tableId, getExpiredDateZeroTimeStamp(3), 2) - LakeSoulTable.forPath(tableName).compaction(true) + LakeSoulTable.forPath(tableName).compaction(cleanOldCompaction = true) cleanAllPartitionExpiredData(spark) if (onlySaveOnceCompaction) { assert(readPartitionInfo(tableId, spark).count() == 2) @@ -326,7 +326,7 @@ class CleanOldCompactionSuite extends QueryTest .toDF("date", "id", "value") LakeSoulTable.forPath(tableName).upsert(df1) - LakeSoulTable.forPath(tableName).compaction(true) + LakeSoulTable.forPath(tableName).compaction(cleanOldCompaction = true) setPartitionInfoTimestamp(tableId, getExpiredDateZeroTimeStamp(6), 0) setPartitionInfoTimestamp(tableId, getExpiredDateZeroTimeStamp(5), 1) setPartitionInfoTimestamp(tableId, getExpiredDateZeroTimeStamp(4), 2) @@ -363,13 +363,13 @@ class CleanOldCompactionSuite extends QueryTest val sm = SnapshotManagement(SparkUtil.makeQualifiedTablePath(new Path(tableName)).toString) val tableId = sm.updateSnapshot().getTableInfo.table_id - LakeSoulTable.forPath(tableName).compaction(onlySaveOnceCompaction) + LakeSoulTable.forPath(tableName).compaction(cleanOldCompaction = onlySaveOnceCompaction) val df1 = Seq(("2020-01-02", 3, "a"), ("2020-01-01", 4, "b")) .toDF("date", "id", "value") LakeSoulTable.forPath(tableName).upsert(df1) - LakeSoulTable.forPath(tableName).compaction(onlySaveOnceCompaction) + LakeSoulTable.forPath(tableName).compaction(cleanOldCompaction = onlySaveOnceCompaction) cleanAllPartitionExpiredData(spark) assert(readPartitionInfo(tableId, spark).count() == 8) assert(readDataCommitInfo(tableId, spark).count() == 8) diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala index 9ef9d24da..c01237672 100644 --- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala +++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala @@ -94,7 +94,7 @@ class CompactionSuite extends QueryTest assert(!rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1))) - LakeSoulTable.forPath(tableName).compaction(true) + LakeSoulTable.forPath(tableName).compaction() rangeGroup = SparkUtil.allDataInfo(sm.updateSnapshot()).groupBy(_.range_partitions) rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1)) assert(rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1))) @@ -131,7 +131,7 @@ class CompactionSuite extends QueryTest assert(!rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1))) - LakeSoulTable.forPath(tableName).compaction(true) + LakeSoulTable.forPath(tableName).compaction() rangeGroup = SparkUtil.allDataInfo(sm.updateSnapshot()).groupBy(_.range_partitions) rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1)) assert(rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1))) @@ -165,7 +165,7 @@ class CompactionSuite extends QueryTest .mode("append") .save(tableName) - LakeSoulTable.forPath(tableName).compaction(true) + LakeSoulTable.forPath(tableName).compaction() }) } @@ -198,7 +198,7 @@ class CompactionSuite extends QueryTest assert(!rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1))) - LakeSoulTable.forPath(tableName).compaction(true) + LakeSoulTable.forPath(tableName).compaction() rangeGroup = SparkUtil.allDataInfo(sm.updateSnapshot()).groupBy(_.range_partitions) rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1)) assert(rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1))) @@ -235,7 +235,7 @@ class CompactionSuite extends QueryTest assert(!rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1))) - LakeSoulTable.forPath(tableName).compaction(true) + LakeSoulTable.forPath(tableName).compaction() rangeGroup = SparkUtil.allDataInfo(sm.updateSnapshot()).groupBy(_.range_partitions) rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1)) assert(rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1))) @@ -438,7 +438,7 @@ class CompactionSuite extends QueryTest val mergeOperatorInfo = Map( "v1" -> new MergeOpInt(), "v2" -> "org.apache.spark.sql.lakesoul.test.MergeOpString") - table.compaction(true, mergeOperatorInfo, true) + table.compaction(mergeOperatorInfo = mergeOperatorInfo, cleanOldCompaction = true) LakeSoulTable.uncached(tableName) checkAnswer(table.toDF.select("range", "hash", "v1", "v2"), result) @@ -464,12 +464,12 @@ class CompactionSuite extends QueryTest val e1 = intercept[AnalysisException] { class tmp {} val mergeOperatorInfo = Map("value" -> new tmp()) - table.compaction(true, mergeOperatorInfo, true) + table.compaction(mergeOperatorInfo = mergeOperatorInfo, cleanOldCompaction = true) } assert(e1.getMessage().contains("is not a legal merge operator class")) val e2 = intercept[ClassNotFoundException] { val mergeOperatorInfo = Map("value" -> "ClassWillNeverExsit") - table.compaction(true, mergeOperatorInfo, true) + table.compaction(mergeOperatorInfo = mergeOperatorInfo, cleanOldCompaction = true) } assert(e2.getMessage.contains("ClassWillNeverExsit")) @@ -498,7 +498,7 @@ class CompactionSuite extends QueryTest checkAnswer(spark.sql("show tables in default"), Seq(Row("default", "lakesoul_test_table", false))) val lakeSoulTable = LakeSoulTable.forName("lakesoul_test_table") - lakeSoulTable.compaction("date='2021-01-01'", "spark_catalog.default.external_table") + lakeSoulTable.compaction("date='2021-01-01'", hiveTableName = "spark_catalog.default.external_table") checkAnswer(spark.sql("show partitions spark_catalog.default.external_table"), Seq(Row("date=2021-01-01"))) checkAnswer(spark.sql("select * from spark_catalog.default.external_table order by id"), @@ -552,9 +552,7 @@ class CompactionSuite extends QueryTest lakeSoulTable.toDF.show // Perform limited compaction (group every compactGroupSize PartitionInfo) - lakeSoulTable.compaction(condition = "", force = true, mergeOperatorInfo = Map.empty, - hiveTableName = "", hivePartitionName = "", cleanOldCompaction = false, - fileNumLimit = Some(compactGroupSize), newBucketNum = None) + lakeSoulTable.compaction(fileNumLimit = Some(compactGroupSize)) // Get PartitionInfo count after compaction val compactedFileCount = getFileCount(tablePath) @@ -591,55 +589,65 @@ class CompactionSuite extends QueryTest } test("compaction with newBucketNum") { - // withTempDir { tempDir => - val tempDir = Utils.createDirectory(System.getProperty("java.io.tmpdir")) - val tablePath = tempDir.getCanonicalPath - val spark = SparkSession.active - - val hashBucketNum = 4 - val newHashBucketNum = 7 - val compactRounds = 5 - val dataPerRounds = 10 - val compactGroupSize = 3 - - // Create test data - val df = Seq( - (1, "2023-01-01", 10, 1), - (2, "2023-01-02", 20, 1), - (3, "2023-01-03", 30, 1), - (4, "2023-01-04", 40, 1), - (5, "2023-01-05", 50, 1) - ).toDF("id", "date", "value", "range") - - // Write initial data - df.write - .format("lakesoul") - .option("rangePartitions", "range") - .option("hashPartitions", "id") - .option(SHORT_TABLE_NAME, "rebucket_table") - .option("hashBucketNum", hashBucketNum.toString) - .save(tablePath) - - val lakeSoulTable = LakeSoulTable.forPath(tablePath) - - for (i <- 1 to 100) { - val appendDf = Seq( - (i * 10, s"2023-02-0$i", i * 100, 1) + withTempDir { tempDir => + // val tempDir = Utils.createDirectory(System.getProperty("java.io.tmpdir")) + val tablePath = tempDir.getCanonicalPath + val spark = SparkSession.active + + val hashBucketNum = 4 + val newHashBucketNum = 7 + val compactRounds = 5 + val dataPerRounds = 10 + val compactGroupSize = 3 + + // Create test data + val df = Seq( + (1, "2023-01-01", 10, 1), + (2, "2023-01-02", 20, 1), + (3, "2023-01-03", 30, 1), + (4, "2023-01-04", 40, 1), + (5, "2023-01-05", 50, 1) ).toDF("id", "date", "value", "range") - lakeSoulTable.upsert(appendDf) - } - assert(getFileBucketSet(tablePath).size == hashBucketNum) - lakeSoulTable.compaction(condition = "", force = true, mergeOperatorInfo = Map.empty, - hiveTableName = "", hivePartitionName = "", cleanOldCompaction = false, - fileNumLimit = None, newBucketNum = Some(newHashBucketNum)) + // val df = Seq( + // (1, "2023-01-01", 10, 1, "insert"), + // (2, "2023-01-02", 20, 1, "insert"), + // (3, "2023-01-03", 30, 1, "insert"), + // (4, "2023-01-04", 40, 1, "insert"), + // (5, "2023-01-05", 50, 1, "insert") + // ).toDF("id", "date", "value", "range", "op") - assert(getFileBucketSet(tablePath).size == newHashBucketNum) - val compactedData = lakeSoulTable.toDF.orderBy("id", "date").collect() - println(compactedData.mkString("Array(", ", ", ")")) - assert(compactedData.length == 105, s"The compressed data should have ${105} rows, but it actually has ${compactedData.length} rows") + // Write initial data + df.write + .format("lakesoul") + .option("rangePartitions", "range") + .option("hashPartitions", "id") + .option(SHORT_TABLE_NAME, "rebucket_table") + .option("hashBucketNum", hashBucketNum.toString) + // .option("lakesoul_cdc_change_column", "op") + .save(tablePath) + + val lakeSoulTable = LakeSoulTable.forPath(tablePath) + + for (i <- 1 to 100) { + val appendDf = Seq( + (i * 10, s"2023-02-0$i", i * 100, 1) + ).toDF("id", "date", "value", "range") + // val appendDf = Seq( + // (i * 10, s"2023-02-0$i", i * 100, 1, "insert") + // ).toDF("id", "date", "value", "range", "op") + lakeSoulTable.upsert(appendDf) + } + assert(getFileBucketSet(tablePath).size == hashBucketNum) + + lakeSoulTable.compaction(newBucketNum = Some(newHashBucketNum)) - // } + assert(getFileBucketSet(tablePath).size == newHashBucketNum) + val compactedData = lakeSoulTable.toDF.orderBy("id", "date").collect() + println(compactedData.mkString("Array(", ", ", ")")) + assert(compactedData.length == 105, s"The compressed data should have ${105} rows, but it actually has ${compactedData.length} rows") + + } } // Auxiliary method: Get the bucket number of table diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/manual_execute_suites/CompactionDoNotChangeResult.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/manual_execute_suites/CompactionDoNotChangeResult.scala index 98f927b9c..5c4af1fe8 100644 --- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/manual_execute_suites/CompactionDoNotChangeResult.scala +++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/manual_execute_suites/CompactionDoNotChangeResult.scala @@ -49,7 +49,7 @@ class CompactionDoNotChangeResult { assert(!rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1))) - LakeSoulTable.forPath(tableName).compaction(true) + LakeSoulTable.forPath(tableName).compaction(cleanOldCompaction = true) rangeGroup = SparkUtil.allDataInfo(sm.snapshot).groupBy(_.range_partitions) assert(rangeGroup.forall(_._2.groupBy(_.file_bucket_id).forall(_._2.length == 1))) From ca27495eeaa4e82c12160f2010f089f3c15607d0 Mon Sep 17 00:00:00 2001 From: maosen <738159199@qq.com> Date: Thu, 17 Oct 2024 21:36:41 +0800 Subject: [PATCH 16/33] update pg tableinfo with bucketnum --- .../sql/lakesoul/commands/CompactionCommand.scala | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala index d15d8a7c7..1fabe923a 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala @@ -261,13 +261,11 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, }) } if (bucketNumChanged) { - snapshotManagement.withNewTransaction(tc => { - val newConfiguration = new mutable.HashMap[String, String] - newConfiguration ++= tableInfo.configuration - newConfiguration.put(TableInfoProperty.HASH_BUCKET_NUM, newBucketNum.get.toString) - val newTableInfo = tableInfo.copy(configuration = newConfiguration.toMap) - tc.updateTableInfo(newTableInfo) - }) + val properties = SparkMetaVersion.dbManager.getTableInfoByTableId(tableInfo.table_id).getProperties + val newProperties = JSON.parseObject(properties); + newProperties.put(TableInfoProperty.HASH_BUCKET_NUM, newBucketNum.get.toString) + SparkMetaVersion.dbManager.updateTableProperties(tableInfo.table_id, newProperties.toJSONString) + snapshotManagement.updateSnapshot() } Seq.empty } From 291f979346cadc247d3e9877447cb25be05bbd91 Mon Sep 17 00:00:00 2001 From: zenghua Date: Fri, 18 Oct 2024 10:11:57 +0800 Subject: [PATCH 17/33] update assertion of ut Signed-off-by: zenghua --- .../spark/sql/lakesoul/commands/CompactionSuite.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala index c01237672..1837a0c04 100644 --- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala +++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala @@ -15,7 +15,7 @@ import org.apache.spark.sql.lakesoul.SnapshotManagement import org.apache.spark.sql.lakesoul.catalog.LakeSoulCatalog import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf import org.apache.spark.sql.lakesoul.test.{LakeSoulSQLCommandTest, LakeSoulTestSparkSession, MergeOpInt} -import org.apache.spark.sql.lakesoul.utils.SparkUtil +import org.apache.spark.sql.lakesoul.utils.{SparkUtil, TableInfo} import org.apache.spark.sql.test.{SharedSparkSession, TestSparkSession} import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SparkSession} import org.apache.spark.util.Utils @@ -639,10 +639,13 @@ class CompactionSuite extends QueryTest lakeSoulTable.upsert(appendDf) } assert(getFileBucketSet(tablePath).size == hashBucketNum) + assert(getTableInfo(tablePath).bucket_num == hashBucketNum) lakeSoulTable.compaction(newBucketNum = Some(newHashBucketNum)) assert(getFileBucketSet(tablePath).size == newHashBucketNum) + assert(getTableInfo(tablePath).bucket_num == newHashBucketNum) + val compactedData = lakeSoulTable.toDF.orderBy("id", "date").collect() println(compactedData.mkString("Array(", ", ", ")")) assert(compactedData.length == 105, s"The compressed data should have ${105} rows, but it actually has ${compactedData.length} rows") @@ -659,4 +662,10 @@ class CompactionSuite extends QueryTest files.groupBy(_.file_bucket_id).keys.toSet } + // Auxiliary method: Get the bucket number of table + def getTableInfo(tablePath: String): TableInfo = { + val sm = SnapshotManagement(SparkUtil.makeQualifiedTablePath(new Path(tablePath)).toString) + sm.getTableInfoOnly + } + } From 864a507e18de56ac0d3fb134281fbe10a4e94406 Mon Sep 17 00:00:00 2001 From: zenghua Date: Fri, 18 Oct 2024 10:27:25 +0800 Subject: [PATCH 18/33] add CompactionTask parameter file_num_limit Signed-off-by: zenghua --- .../lakesoul/spark/compaction/CompactionTask.scala | 9 +++++++-- .../com/dmetasoul/lakesoul/tables/LakeSoulTable.scala | 8 -------- 2 files changed, 7 insertions(+), 10 deletions(-) diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala index 475f8aa62..a4c310243 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala @@ -22,18 +22,23 @@ object CompactionTask { val dateFormat: SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") val THREADPOOL_SIZE_PARAMETER = "threadpool.size" val DATABASE_PARAMETER = "database" + val FILE_NUM_LIMIT_PARAMETER = "file_num_limit" val NOTIFY_CHANNEL_NAME = "lakesoul_compaction_notify" val threadMap: java.util.Map[String, Integer] = new ConcurrentHashMap var threadPoolSize = 8 var database = "" + var fileNumLimit: Option[Int] = None def main(args: Array[String]): Unit = { val parameter = ParametersTool.fromArgs(args) threadPoolSize = parameter.getInt(THREADPOOL_SIZE_PARAMETER, 8) database = parameter.get(DATABASE_PARAMETER, "") + if (parameter.has(FILE_NUM_LIMIT_PARAMETER)) { + fileNumLimit = Some(parameter.getInt(FILE_NUM_LIMIT_PARAMETER)) + } val builder = SparkSession.builder() .config("spark.sql.parquet.mergeSchema", value = true) @@ -96,14 +101,14 @@ object CompactionTask { println("------ " + threadName + " is compressing table path is: " + path + " ------") val table = LakeSoulTable.forPath(path) if (partitionDesc == "") { - table.compaction(fileNumLimit = Some(100)) + table.compaction(fileNumLimit = fileNumLimit) } else { val partitions = partitionDesc.split(",").map( partition => { partition.replace("=", "='") + "'" } ).mkString(" and ") - table.compaction(partitions, true, fileNumLimit = Some(100)) + table.compaction(partitions, cleanOldCompaction = true, fileNumLimit = fileNumLimit) } } catch { case e: Exception => { diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala index 51fbc81cf..1681de98e 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala @@ -306,14 +306,6 @@ class LakeSoulTable(df: => Dataset[Row], snapshotManagement: SnapshotManagement) } //by default, force perform compaction on whole table - def compaction(): Unit = { - compaction("", true, Map.empty[String, Any], "", "", false, None, None) - } - - def compaction(condition: String): Unit = { - compaction(condition = condition, true) - } - def compaction(condition: String = "", force: Boolean = true, mergeOperatorInfo: Map[String, Any] = Map.empty, From eda74b24781c1ade66d3ab922bd32fec5b09876f Mon Sep 17 00:00:00 2001 From: zenghua Date: Fri, 18 Oct 2024 16:20:57 +0800 Subject: [PATCH 19/33] fix case of cleanOldCompaction Signed-off-by: zenghua --- .../lakesoul/commands/CompactionCommand.scala | 92 +++++++++++-------- .../datasource/ParquetScanSuite.scala | 1 + 2 files changed, 54 insertions(+), 39 deletions(-) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala index 1fabe923a..ee005a5f5 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.util.Utils +import java.util.UUID import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConverters._ @@ -43,7 +44,7 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, ) extends LeafRunnableCommand with PredicateHelper with Logging { - lazy val compactPath: String = tableInfo.table_path.toString + "/compact_" + System.currentTimeMillis() + def newCompactPath: String = tableInfo.table_path.toString + "/compact_" + System.currentTimeMillis() lazy val bucketNumChanged: Boolean = newBucketNum.exists(tableInfo.bucket_num != _) @@ -55,7 +56,7 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, partitionInfo.read_files.length >= 1 } - def executeCompaction(spark: SparkSession, tc: TransactionCommit, files: Seq[DataFileInfo], readPartitionInfo: Array[PartitionInfoScala]): List[DataCommitInfo] = { + def executeCompaction(spark: SparkSession, tc: TransactionCommit, files: Seq[DataFileInfo], readPartitionInfo: Array[PartitionInfoScala], compactPath: String): List[DataCommitInfo] = { if (readPartitionInfo.forall(p => p.commit_op.equals("CompactionCommit") && p.read_files.length == 1)) { logInfo("=========== All Partitions Have Compacted, This Operation Will Cancel!!! ===========") return List.empty @@ -138,13 +139,14 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, builder.toString() } - def compactSinglePartition(sparkSession: SparkSession, tc: TransactionCommit, files: Seq[DataFileInfo], sourcePartition: PartitionInfoScala) = { + def compactSinglePartition(sparkSession: SparkSession, tc: TransactionCommit, files: Seq[DataFileInfo], sourcePartition: PartitionInfoScala): String = { logInfo(s"Compacting Single Partition=${sourcePartition} with ${files.length} files") val bucketedFiles = if (tableInfo.hash_partition_columns.isEmpty || bucketNumChanged) { Seq(-1 -> files) } else { files.groupBy(_.file_bucket_id) } + val compactPath = newCompactPath val allDataCommitInfo = bucketedFiles.flatMap(groupByBucketId => { val (bucketId, files) = groupByBucketId val groupedFiles = if (fileNumLimit.isDefined) { @@ -165,7 +167,7 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, files.forall(_.size == 1) } if (!hasNoDeltaFile) { - executeCompaction(sparkSession, tc, files, Array(sourcePartition)) + executeCompaction(sparkSession, tc, files, Array(sourcePartition), compactPath) } else { logInfo(s"== Partition ${sourcePartition.range_value} has no delta file.") None @@ -173,34 +175,47 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, }) }) - val compactPartitionInfo = List.newBuilder[PartitionInfoScala] - compactPartitionInfo += PartitionInfoScala( - table_id = tc.tableInfo.table_id, - range_value = sourcePartition.range_value, - read_files = allDataCommitInfo.map(_.getCommitId).map(DBUtil.toJavaUUID).toArray - ) + if (allDataCommitInfo.nonEmpty) { + val compactDataCommitInfoId = UUID.randomUUID + val compactDataCommitInfo = + DataCommitInfo.newBuilder(allDataCommitInfo.head) + .setCommitId(DBUtil.toProtoUuid(compactDataCommitInfoId)) + .clearFileOps + .addAllFileOps(allDataCommitInfo.flatMap(_.getFileOpsList.asScala).asJava) + .build + + val compactPartitionInfo = List.newBuilder[PartitionInfoScala] + compactPartitionInfo += PartitionInfoScala( + table_id = tc.tableInfo.table_id, + range_value = sourcePartition.range_value, + read_files = Array(compactDataCommitInfoId) + ) - tc.commitDataCommitInfo(allDataCommitInfo.toList, compactPartitionInfo.result, "", -1, Array(sourcePartition)) - - val partitionStr = escapeSingleBackQuotedString(conditionString) - if (hiveTableName.nonEmpty) { - val spark = SparkSession.active - val currentCatalog = spark.sessionState.catalogManager.currentCatalog.name() - Utils.tryWithSafeFinally({ - spark.sessionState.catalogManager.setCurrentCatalog(SESSION_CATALOG_NAME) - if (hivePartitionName.nonEmpty) { - spark.sql(s"ALTER TABLE $hiveTableName DROP IF EXISTS partition($hivePartitionName)") - spark.sql(s"ALTER TABLE $hiveTableName ADD partition($hivePartitionName) location '${compactPath}/$partitionStr'") - } else { - spark.sql(s"ALTER TABLE $hiveTableName DROP IF EXISTS partition($conditionString)") - spark.sql(s"ALTER TABLE $hiveTableName ADD partition($conditionString) location '${compactPath}/$partitionStr'") - } + tc.commitDataCommitInfo(List(compactDataCommitInfo), compactPartitionInfo.result, "", -1, Array(sourcePartition)) + + val partitionStr = escapeSingleBackQuotedString(conditionString) + if (hiveTableName.nonEmpty) { + val spark = SparkSession.active + val currentCatalog = spark.sessionState.catalogManager.currentCatalog.name() + Utils.tryWithSafeFinally({ + spark.sessionState.catalogManager.setCurrentCatalog(SESSION_CATALOG_NAME) + if (hivePartitionName.nonEmpty) { + spark.sql(s"ALTER TABLE $hiveTableName DROP IF EXISTS partition($hivePartitionName)") + spark.sql(s"ALTER TABLE $hiveTableName ADD partition($hivePartitionName) location '${compactPath}/$partitionStr'") + } else { + spark.sql(s"ALTER TABLE $hiveTableName DROP IF EXISTS partition($conditionString)") + spark.sql(s"ALTER TABLE $hiveTableName ADD partition($conditionString) location '${compactPath}/$partitionStr'") + } - }) { - spark.sessionState.catalogManager.setCurrentCatalog(currentCatalog) + }) { + spark.sessionState.catalogManager.setCurrentCatalog(currentCatalog) + } } + logInfo("=========== Compaction Success!!! ===========") + compactPath + } else { + "" } - logInfo("=========== Compaction Success!!! ===========") } @@ -232,9 +247,9 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, "" ) - compactSinglePartition(sparkSession, tc, files, partitionInfo) + val newCompact = compactSinglePartition(sparkSession, tc, files, partitionInfo) - if (cleanOldCompaction) { + if (newCompact.nonEmpty && cleanOldCompaction) { val tablePath = snapshotManagement.table_path cleanOldCommitOpDiskData(tablePath, partitionSet.head, sparkSession) } @@ -251,21 +266,20 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, tc.setCommitType("compaction") val files = tc.getCompactionPartitionFiles(part) - compactSinglePartition(sparkSession, tc, files, part) - - if (cleanOldCompaction) { + val newCompact = compactSinglePartition(sparkSession, tc, files, part) + if (newCompact.nonEmpty && cleanOldCompaction) { val tablePath = snapshotManagement.table_path - cleanOldCommitOpDiskData(tablePath, null, sparkSession) + cleanOldCommitOpDiskData(tablePath, part.range_value, sparkSession) } }) }) } if (bucketNumChanged) { - val properties = SparkMetaVersion.dbManager.getTableInfoByTableId(tableInfo.table_id).getProperties - val newProperties = JSON.parseObject(properties); - newProperties.put(TableInfoProperty.HASH_BUCKET_NUM, newBucketNum.get.toString) - SparkMetaVersion.dbManager.updateTableProperties(tableInfo.table_id, newProperties.toJSONString) - snapshotManagement.updateSnapshot() + val properties = SparkMetaVersion.dbManager.getTableInfoByTableId(tableInfo.table_id).getProperties + val newProperties = JSON.parseObject(properties); + newProperties.put(TableInfoProperty.HASH_BUCKET_NUM, newBucketNum.get.toString) + SparkMetaVersion.dbManager.updateTableProperties(tableInfo.table_id, newProperties.toJSONString) + snapshotManagement.updateSnapshot() } Seq.empty } diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/execution/datasource/ParquetScanSuite.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/execution/datasource/ParquetScanSuite.scala index c36296760..cb2d8a4c8 100644 --- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/execution/datasource/ParquetScanSuite.scala +++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/execution/datasource/ParquetScanSuite.scala @@ -106,6 +106,7 @@ class ParquetScanSuite extends QueryTest .save(tablePath) val table = LakeSoulTable.forPath(tablePath) table.compaction() + LakeSoulTable.uncached(tablePath) val plan = table.toDF.queryExecution.toString() From 6f8e8fcdf2daf3bd7e35ba3d159186a71d4fc8c1 Mon Sep 17 00:00:00 2001 From: fphantam Date: Fri, 18 Oct 2024 23:26:28 +0800 Subject: [PATCH 20/33] continue compaction operation when newBucketNum exists Signed-off-by: fphantam --- .../apache/spark/sql/lakesoul/commands/CompactionCommand.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala index ee005a5f5..88e5b88c9 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala @@ -57,7 +57,7 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, } def executeCompaction(spark: SparkSession, tc: TransactionCommit, files: Seq[DataFileInfo], readPartitionInfo: Array[PartitionInfoScala], compactPath: String): List[DataCommitInfo] = { - if (readPartitionInfo.forall(p => p.commit_op.equals("CompactionCommit") && p.read_files.length == 1)) { + if (!newBucketNum.isDefined && readPartitionInfo.forall(p => p.commit_op.equals("CompactionCommit") && p.read_files.length == 1)) { logInfo("=========== All Partitions Have Compacted, This Operation Will Cancel!!! ===========") return List.empty } From 339005c369a5e9271a5c3be19eca67aa2694da6f Mon Sep 17 00:00:00 2001 From: fphantam Date: Sat, 19 Oct 2024 10:16:20 +0800 Subject: [PATCH 21/33] fileNumLimit compaction do not filter 'delete' Signed-off-by: fphantam --- .../org/apache/spark/sql/lakesoul/TransactionalWrite.scala | 6 +++++- .../spark/sql/lakesoul/commands/CompactionCommand.scala | 3 +++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala index 72c098ca9..2e27f3a1d 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala @@ -161,7 +161,11 @@ trait TransactionalWrite { data.withColumn(cdcColName, when(col(cdcColName) === "update", "insert") .otherwise(col(cdcColName)) - ).where(s"$cdcColName != 'delete'") + ) + if (!writeOptions.isDefined || writeOptions.get.options.getOrElse("fileNumLimit", "false").equals("false")) { + data.where(s"$cdcColName != 'delete'") + } + data } else { data } diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala index 88e5b88c9..bdf82ce3e 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala @@ -109,6 +109,9 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, val map = mutable.HashMap[String, String]() map.put("isCompaction", "true") map.put("compactPath", compactPath) + if (fileNumLimit.isDefined) { + map.put("fileNumLimit", "true") + } if (readPartitionInfo.nonEmpty) { map.put("partValue", readPartitionInfo.head.range_value) } From dd733f25ff0077bc7ae72e13a021fa9f30b01202 Mon Sep 17 00:00:00 2001 From: fphantam Date: Sat, 19 Oct 2024 10:41:14 +0800 Subject: [PATCH 22/33] fix compaction filter bug Signed-off-by: fphantam --- .../spark/sql/lakesoul/TransactionalWrite.scala | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala index 2e27f3a1d..978ea21ac 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala @@ -158,14 +158,17 @@ trait TransactionalWrite { if (cdcCol.nonEmpty) { options.put("isCDC", "true") val cdcColName = cdcCol.get - data.withColumn(cdcColName, - when(col(cdcColName) === "update", "insert") - .otherwise(col(cdcColName)) - ) if (!writeOptions.isDefined || writeOptions.get.options.getOrElse("fileNumLimit", "false").equals("false")) { - data.where(s"$cdcColName != 'delete'") + data.withColumn(cdcColName, + when(col(cdcColName) === "update", "insert") + .otherwise(col(cdcColName)) + ).where(s"$cdcColName != 'delete'") + } else { + data.withColumn(cdcColName, + when(col(cdcColName) === "update", "insert") + .otherwise(col(cdcColName)) + ) } - data } else { data } From 9909ece0ede194d8babcf311f367a6d59f52870f Mon Sep 17 00:00:00 2001 From: fphantam Date: Sat, 19 Oct 2024 11:32:11 +0800 Subject: [PATCH 23/33] optimize clean sql and compaction task add reconnect function Signed-off-by: fphantam --- .../spark/clean/CleanExpiredData.scala | 29 +++------ .../spark/clean/CleanOldCompaction.scala | 22 ++++--- .../spark/compaction/CompactionTask.scala | 62 ++++++++++++------- 3 files changed, 63 insertions(+), 50 deletions(-) diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanExpiredData.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanExpiredData.scala index 6c646a8b4..a6293c2f9 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanExpiredData.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanExpiredData.scala @@ -68,7 +68,6 @@ object CleanExpiredData { tableOnlySaveOnceCompaction = onlySaveOnceCompaction } - val latestCompactionTimestampBeforeRedundantTTL = getLatestCompactionTimestamp(tableId, partitionDesc, tableRedundantTTL, spark) println("========== deal with new partition ========== ") println("tableId:" + tableId) println("partitionDesc:" + partitionDesc) @@ -100,6 +99,7 @@ object CleanExpiredData { } } } else { + val latestCompactionTimestampBeforeRedundantTTL = getLatestCompactionTimestamp(tableId, partitionDesc, tableRedundantTTL, spark) println("last compactionTimestamp before expiration:" + latestCompactionTimestampBeforeRedundantTTL) //no compaction action if (latestCompactionTimestampBeforeRedundantTTL == 0L) { @@ -214,19 +214,12 @@ object CleanExpiredData { val sql = s""" |DELETE FROM partition_info - |WHERE (table_id,partition_desc,version) - |IN - |(SELECT - | table_id, - | partition_desc, - | version - |FROM partition_info |WHERE - | timestamp < $deadTimestamp - |AND | table_id = '$tableId' |AND - | partition_desc='$partitionDesc') + | partition_desc='$partitionDesc' + |AND + | timestamp < $deadTimestamp |""".stripMargin val stmt = conn.prepareStatement(sql) stmt.execute() @@ -236,13 +229,14 @@ object CleanExpiredData { val sql = s""" |SELECT - | max(timestamp) max_time + | timestamp max_time |from | partition_info |where | table_id='$table_id' |AND partition_desc = '$partitionDesc' - | + |ORDER BY + | version DESC |""".stripMargin sqlToDataframe(sql, spark).select("max_time").first().getLong(0) } @@ -253,7 +247,7 @@ object CleanExpiredData { val expiredDateZeroTimeMils = getExpiredDateZeroTimeStamp(expiredDaysField.toString.toInt) val sql = s""" - |SELECT DISTINCT ON (table_id) + |SELECT | table_id, | commit_op, | timestamp @@ -266,7 +260,7 @@ object CleanExpiredData { | AND timestamp < '$expiredDateZeroTimeMils' | ORDER BY | table_id, - | timestamp DESC + | version DESC |""".stripMargin if (sqlToDataframe(sql, spark).count() > 0) latestTimestampMils = sqlToDataframe(sql, spark).select("timestamp").first().getLong(0) @@ -278,7 +272,7 @@ object CleanExpiredData { val redundantDataTTlZeroTimeMils = getExpiredDateZeroTimeStamp(redundantDataTTL.toString.toInt) val sql = s""" - |SELECT DISTINCT ON (table_id) + |SELECT | table_id, | commit_op, | timestamp @@ -289,9 +283,6 @@ object CleanExpiredData { | AND partition_desc = '$partitionDesc' | AND table_id = '$table_id' | AND timestamp > '$redundantDataTTlZeroTimeMils' - | ORDER BY - | table_id, - | timestamp DESC |""".stripMargin if (sqlToDataframe(sql, spark).count() > 0) { true diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanOldCompaction.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanOldCompaction.scala index 6a28eee93..3d3d3b6f5 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanOldCompaction.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanOldCompaction.scala @@ -41,10 +41,11 @@ object CleanOldCompaction { val sql = s""" |WITH expiredCommit AS ( - | SELECT DISTINCT ON (table_id) + | SELECT | table_id, | commit_op, | partition_desc, + | snapshot, | timestamp | FROM partition_info | WHERE commit_op = 'CompactionCommit' @@ -52,20 +53,21 @@ object CleanOldCompaction { | AND table_id= '$tableId' | ORDER BY | table_id, - | timestamp DESC + | version DESC |) |SELECT file_op.path AS path |FROM ( | SELECT file_ops | FROM data_commit_info - | WHERE commit_id IN ( - | SELECT DISTINCT unnest(pi.snapshot) AS commit_id - | FROM partition_info pi - | LEFT JOIN expiredCommit ec ON pi.table_id = ec.table_id - | WHERE pi.timestamp < ec.timestamp - | AND pi.commit_op = 'CompactionCommit' - | AND pi.partition_desc = ec.partition_desc - | ) + | WHERE table_id= '$tableId' + | AND partition_desc = '$partitionDesc' + | AND commit_id IN ( + | SELECT DISTINCT unnest(snapshot) + | FROM ( + | SELECT snapshot FROM expiredCommit + | LIMIT (SELECT CASE WHEN COUNT(1) = 0 THEN 1 ELSE COUNT(1) END - 1 FROM expiredCommit) OFFSET 1 + | ) t_limit + | ) |) t |CROSS JOIN LATERAL ( | SELECT diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala index a4c310243..12cb3218a 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala @@ -22,6 +22,7 @@ object CompactionTask { val dateFormat: SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") val THREADPOOL_SIZE_PARAMETER = "threadpool.size" val DATABASE_PARAMETER = "database" + val CLEAN_OLD_COMPACTION = "clean_old_compaction" val FILE_NUM_LIMIT_PARAMETER = "file_num_limit" val NOTIFY_CHANNEL_NAME = "lakesoul_compaction_notify" @@ -29,6 +30,7 @@ object CompactionTask { var threadPoolSize = 8 var database = "" + var cleanOldCompaction: Option[Boolean] = Some(false) var fileNumLimit: Option[Int] = None def main(args: Array[String]): Unit = { @@ -36,6 +38,9 @@ object CompactionTask { val parameter = ParametersTool.fromArgs(args) threadPoolSize = parameter.getInt(THREADPOOL_SIZE_PARAMETER, 8) database = parameter.get(DATABASE_PARAMETER, "") + if (parameter.has(CLEAN_OLD_COMPACTION)) { + cleanOldCompaction = Some(parameter.getBoolean(CLEAN_OLD_COMPACTION)) + } if (parameter.has(FILE_NUM_LIMIT_PARAMETER)) { fileNumLimit = Some(parameter.getInt(FILE_NUM_LIMIT_PARAMETER)) } @@ -59,8 +64,8 @@ object CompactionTask { } class Listener extends Thread { - private val conn = DBConnector.getConn - private val pgconn = conn.unwrap(classOf[PGConnection]) + private var conn = DBConnector.getConn + private var pgconn = conn.unwrap(classOf[PGConnection]) val threadPool: ExecutorService = Executors.newFixedThreadPool(threadPoolSize) @@ -71,25 +76,40 @@ object CompactionTask { val jsonParser = new JsonParser() while (true) { - val notifications = pgconn.getNotifications - if (notifications.nonEmpty) { - notifications.foreach(notification => { - val notificationParameter = notification.getParameter - if (threadMap.get(notificationParameter) != 1) { - threadMap.put(notificationParameter, 1) - val jsonObj = jsonParser.parse(notificationParameter).asInstanceOf[JsonObject] - println("========== " + dateFormat.format(new Date()) + " start processing notification: " + jsonObj + " ==========") - val tablePath = jsonObj.get("table_path").getAsString - val partitionDesc = jsonObj.get("table_partition_desc").getAsString - val tableNamespace = jsonObj.get("table_namespace").getAsString - if (tableNamespace.equals(database) || database.equals("")) { - val rsPartitionDesc = if (partitionDesc.equals(MetaUtils.DEFAULT_RANGE_PARTITION_VALUE)) "" else partitionDesc - threadPool.execute(new CompactionTableInfo(tablePath, rsPartitionDesc, notificationParameter)) + try { + if (!conn.isValid(5000)) { + conn.close() + conn = DBConnector.getConn + pgconn = conn.unwrap(classOf[PGConnection]) + val stmt = conn.createStatement + stmt.execute("LISTEN " + NOTIFY_CHANNEL_NAME) + stmt.close() + } + val notifications = pgconn.getNotifications + if (notifications.nonEmpty) { + notifications.foreach(notification => { + val notificationParameter = notification.getParameter + if (threadMap.get(notificationParameter) != 1) { + threadMap.put(notificationParameter, 1) + val jsonObj = jsonParser.parse(notificationParameter).asInstanceOf[JsonObject] + println("========== " + dateFormat.format(new Date()) + " start processing notification: " + jsonObj + " ==========") + val tablePath = jsonObj.get("table_path").getAsString + val partitionDesc = jsonObj.get("table_partition_desc").getAsString + val tableNamespace = jsonObj.get("table_namespace").getAsString + if (tableNamespace.equals(database) || database.equals("")) { + val rsPartitionDesc = if (partitionDesc.equals(MetaUtils.DEFAULT_RANGE_PARTITION_VALUE)) "" else partitionDesc + threadPool.execute(new CompactionTableInfo(tablePath, rsPartitionDesc, notificationParameter)) + } } - } - }) + }) + } + Thread.sleep(10000) + } catch { + case e: Exception => { + println("** " + dateFormat.format(new Date()) + " find exception in while codes **") + throw e + } } - Thread.sleep(10000) } } } @@ -101,14 +121,14 @@ object CompactionTask { println("------ " + threadName + " is compressing table path is: " + path + " ------") val table = LakeSoulTable.forPath(path) if (partitionDesc == "") { - table.compaction(fileNumLimit = fileNumLimit) + table.compaction(cleanOldCompaction = cleanOldCompaction.get, fileNumLimit = fileNumLimit) } else { val partitions = partitionDesc.split(",").map( partition => { partition.replace("=", "='") + "'" } ).mkString(" and ") - table.compaction(partitions, cleanOldCompaction = true, fileNumLimit = fileNumLimit) + table.compaction(partitions, cleanOldCompaction = cleanOldCompaction.get, fileNumLimit = fileNumLimit) } } catch { case e: Exception => { From f4508199e6db6e46df185e708c1997f4cf0fb41a Mon Sep 17 00:00:00 2001 From: maosen <738159199@qq.com> Date: Wed, 23 Oct 2024 18:02:10 +0800 Subject: [PATCH 24/33] add limit push down for table source for batch and stream,but not work with order by clause because of flink --- .../source/LakeSoulOneSplitRecordsReader.java | 14 ++++++++++++ .../lakesoul/table/LakeSoulTableSource.java | 8 ++++++- .../lakesoul/tool/LakeSoulSinkOptions.java | 7 +++++- .../lakesoul/test/flinkSource/DMLSuite.java | 22 +++++++++++++++++++ 4 files changed, 49 insertions(+), 2 deletions(-) diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulOneSplitRecordsReader.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulOneSplitRecordsReader.java index 9dfba2f71..e982b40f9 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulOneSplitRecordsReader.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulOneSplitRecordsReader.java @@ -32,6 +32,7 @@ import java.util.stream.Collectors; import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.BATCH_SIZE; +import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.LIMIT; public class LakeSoulOneSplitRecordsReader implements RecordsWithSplitIds, AutoCloseable { @@ -79,6 +80,9 @@ public class LakeSoulOneSplitRecordsReader implements RecordsWithSplitIds= this.limit) { + this.reader.close(); + LOG.info("Reach limit condition {}", split); + return null; + } if (curRecordIdx >= currentVCR.getRowCount()) { if (this.reader.hasNext()) { this.currentVCR = this.reader.nextResultVectorSchemaRoot(); @@ -262,6 +274,7 @@ public RowData nextRecordFromSplit() { rd = this.curArrowReaderRequestedSchema.read(rowId); // change rowkind if needed rd.setRowKind(rk); + totalRead++; return rd; } } @@ -274,6 +287,7 @@ public Set finishedSplits() { @Override public void close() throws Exception { + System.out.println("finish close:" + totalRead); if (this.currentVCR != null) { this.currentVCR.close(); this.currentVCR = null; diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/table/LakeSoulTableSource.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/table/LakeSoulTableSource.java index 2e3cd4cd4..033c0f52b 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/table/LakeSoulTableSource.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/table/LakeSoulTableSource.java @@ -25,6 +25,7 @@ import org.apache.flink.table.connector.source.ScanTableSource; import org.apache.flink.table.connector.source.SourceProvider; import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; import org.apache.flink.table.connector.source.abilities.SupportsRowLevelModificationScan; import org.apache.flink.table.expressions.ResolvedExpression; @@ -47,7 +48,7 @@ public class LakeSoulTableSource implements SupportsFilterPushDown, SupportsProjectionPushDown, ScanTableSource, - SupportsRowLevelModificationScan { + SupportsRowLevelModificationScan, SupportsLimitPushDown { private static final Logger LOG = LoggerFactory.getLogger(LakeSoulTableSource.class); @@ -351,4 +352,9 @@ public RowLevelModificationScanContext applyRowLevelModificationScan( public LakeSoulRowLevelModificationScanContext getModificationContext() { return modificationContext; } + + @Override + public void applyLimit(long limit) { + this.optionParams.put(LakeSoulSinkOptions.LIMIT.key(),String.valueOf(limit)); + } } diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/LakeSoulSinkOptions.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/LakeSoulSinkOptions.java index 7cd9474e4..4c20dd41b 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/LakeSoulSinkOptions.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/LakeSoulSinkOptions.java @@ -11,7 +11,7 @@ import java.time.Duration; -public class LakeSoulSinkOptions { +public class LakeSoulSinkOptions { public static final String FACTORY_IDENTIFIER = "lakesoul"; @@ -231,6 +231,11 @@ public class LakeSoulSinkOptions { .booleanType() .defaultValue(false) .withDescription("If true, lakesoul sink will auto change sink table's schema"); + public static final ConfigOption LIMIT = ConfigOptions + .key("lakesoul.limit") + .longType() + .defaultValue(Long.MAX_VALUE) + .withDescription("limit io read num"); } diff --git a/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/flinkSource/DMLSuite.java b/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/flinkSource/DMLSuite.java index edf855fa7..9e2e55fbb 100644 --- a/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/flinkSource/DMLSuite.java +++ b/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/flinkSource/DMLSuite.java @@ -284,6 +284,28 @@ public void testDeletePartitionOnlySQL() throws ExecutionException, InterruptedE TestUtils.checkEqualInAnyOrder(results, new String[]{"+I[2, Alice, 80]", "+I[4, Bob, 110]"}); } + @Test + public void testSelectWithLimit() throws ExecutionException, InterruptedException { + TableEnvironment tEnv = TestUtils.createTableEnv(BATCH_TYPE); + createLakeSoulSourceTableUserWithRange(tEnv); + tEnv.executeSql( + "INSERT INTO user_info_1 VALUES (2, 'Alice', 80),(3, 'Jack', 75),(3, 'Amy', 95),(4, 'Bob', 110)") + .await(); + String testSelect = "select * from user_info_1 limit 2"; + List result = CollectionUtil.iteratorToList(tEnv.executeSql(testSelect).collect()); + assert result.size() == 2; + String testSelect1 = "select * from user_info_1 limit 9"; + List result1 = CollectionUtil.iteratorToList(tEnv.executeSql(testSelect1).collect()); + assert result1.size() == 4; + StreamTableEnvironment streamEnv = TestUtils.createStreamTableEnv(BATCH_TYPE); + TableImpl flinkTable = (TableImpl) streamEnv.sqlQuery(testSelect); + List results = CollectionUtil.iteratorToList(flinkTable.execute().collect()); + assert result.size() == 2; + TableImpl flinkTable1 = (TableImpl) streamEnv.sqlQuery(testSelect1); + List result2 = CollectionUtil.iteratorToList(flinkTable1.execute().collect()); + assert result2.size() == 4; + } + @Test public void testDeleteAllPartitionedDataExactlySQL() throws ExecutionException, InterruptedException { TableEnvironment tEnv = TestUtils.createTableEnv(BATCH_TYPE); From 8d73e4c3b62e01f3012d9a00b378f690e233bbe5 Mon Sep 17 00:00:00 2001 From: maosen <738159199@qq.com> Date: Wed, 23 Oct 2024 19:07:05 +0800 Subject: [PATCH 25/33] fix out put --- .../flink/lakesoul/source/LakeSoulOneSplitRecordsReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulOneSplitRecordsReader.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulOneSplitRecordsReader.java index e982b40f9..bbe53981d 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulOneSplitRecordsReader.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulOneSplitRecordsReader.java @@ -287,7 +287,7 @@ public Set finishedSplits() { @Override public void close() throws Exception { - System.out.println("finish close:" + totalRead); + LOG.info("finish close read num:" + totalRead); if (this.currentVCR != null) { this.currentVCR.close(); this.currentVCR = null; From cd8a435a156885db38ecf65db86b048216b43126 Mon Sep 17 00:00:00 2001 From: hw_syl_zenghua Date: Thu, 24 Oct 2024 16:11:01 +0800 Subject: [PATCH 26/33] update the build.yml --- .cloudbuild/build.yml | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) create mode 100644 .cloudbuild/build.yml diff --git a/.cloudbuild/build.yml b/.cloudbuild/build.yml new file mode 100644 index 000000000..5d1c93b07 --- /dev/null +++ b/.cloudbuild/build.yml @@ -0,0 +1,18 @@ +# This YAML is the default template and can be modified based on this +--- +version: 2.0 +steps: + PRE_BUILD: + - checkout: + name: "checkout" + inputs: + scm: "codehub" + url: "git@codehub.devcloud.cn-east-3.huaweicloud.com:42b369588d84469d95d7b738fc58da8e/LakeSoul/for-nanhang.git" + # branch: "merge_main" + lfs: false + submodule: false + BUILD: + - sh: + name: "sh" + inputs: + command: "echo hello" From 04b120cabace184e6854f76863fcfe08698d4c99 Mon Sep 17 00:00:00 2001 From: hw_syl_zenghua Date: Thu, 24 Oct 2024 16:11:41 +0800 Subject: [PATCH 27/33] update the build.yml --- .cloudbuild/build.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.cloudbuild/build.yml b/.cloudbuild/build.yml index 5d1c93b07..7aa8e019c 100644 --- a/.cloudbuild/build.yml +++ b/.cloudbuild/build.yml @@ -8,7 +8,7 @@ steps: inputs: scm: "codehub" url: "git@codehub.devcloud.cn-east-3.huaweicloud.com:42b369588d84469d95d7b738fc58da8e/LakeSoul/for-nanhang.git" - # branch: "merge_main" + branch: "merge_main" lfs: false submodule: false BUILD: From 61d89e427bd2f4df8ab8267de4d7d2c059a2e602 Mon Sep 17 00:00:00 2001 From: zenghua Date: Wed, 23 Oct 2024 18:59:48 +0800 Subject: [PATCH 28/33] add compaction paras of size limit Signed-off-by: zenghua --- .../com/dmetasoul/lakesoul/meta/DBUtil.java | 25 +- .../parquet/NativeVectorizedReader.java | 17 +- .../spark/clean/CleanOldCompaction.scala | 8 +- .../spark/compaction/CompactionTask.scala | 9 +- .../lakesoul/tables/LakeSoulTable.scala | 6 +- .../execution/LakeSoulTableOperations.scala | 6 +- ...veMergeParquetPartitionReaderFactory.scala | 15 +- .../NativeParquetPartitionReaderFactory.scala | 64 ++-- .../sql/lakesoul/TransactionCommit.scala | 31 +- .../sql/lakesoul/TransactionalWrite.scala | 6 +- .../catalog/LakeSoulScanBuilder.scala | 15 +- .../lakesoul/catalog/LakeSoulTableV2.scala | 10 +- .../lakesoul/commands/CompactionCommand.scala | 97 +++++- .../lakesoul/sources/LakeSoulSQLConf.scala | 22 +- .../lakesoul/commands/CompactionSuite.scala | 282 ++++++++++++++++-- .../src/datasource/physical_plan/merge.rs | 19 +- rust/lakesoul-io/src/lakesoul_io_config.rs | 10 + 17 files changed, 553 insertions(+), 89 deletions(-) diff --git a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBUtil.java b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBUtil.java index be3774904..b42589f64 100644 --- a/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBUtil.java +++ b/lakesoul-common/src/main/java/com/dmetasoul/lakesoul/meta/DBUtil.java @@ -22,6 +22,8 @@ import java.sql.SQLException; import java.sql.Statement; import java.util.*; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import java.util.stream.Collectors; import static com.dmetasoul.lakesoul.meta.DBConfig.*; @@ -373,4 +375,25 @@ public void report() { } } -} + + public static long parseMemoryExpression(String memoryExpression) { + Pattern pattern = Pattern.compile("(\\d+)(\\w+)"); + Matcher matcher = pattern.matcher(memoryExpression); + if (matcher.find()) { + long value = Long.parseLong(matcher.group(1)); + String unit = matcher.group(2); + switch (unit) { + case "KB": + return value * 1024; + case "MB": + return value * 1024 * 1024; + case "GB": + return value * 1024 * 1024 * 1024; + default: + return value; + } + } + return 0; + } + +} \ No newline at end of file diff --git a/lakesoul-spark/src/main/java/org/apache/spark/sql/execution/datasources/parquet/NativeVectorizedReader.java b/lakesoul-spark/src/main/java/org/apache/spark/sql/execution/datasources/parquet/NativeVectorizedReader.java index 8ba22d55b..9fa695480 100644 --- a/lakesoul-spark/src/main/java/org/apache/spark/sql/execution/datasources/parquet/NativeVectorizedReader.java +++ b/lakesoul-spark/src/main/java/org/apache/spark/sql/execution/datasources/parquet/NativeVectorizedReader.java @@ -142,7 +142,11 @@ public void initialize(InputSplit[] inputSplits, TaskAttemptContext taskAttemptC initialize(inputSplits, taskAttemptContext, null, requestSchema, null); } - public void initialize(InputSplit[] inputSplits, TaskAttemptContext taskAttemptContext, String[] primaryKeys, StructType requestSchema, Map mergeOperatorInfo) + public void initialize(InputSplit[] inputSplits, + TaskAttemptContext taskAttemptContext, + String[] primaryKeys, + StructType requestSchema, + Map mergeOperatorInfo) throws IOException, InterruptedException, UnsupportedOperationException { super.initialize(inputSplits[0], taskAttemptContext); FileSplit split = (FileSplit) inputSplits[0]; @@ -220,6 +224,10 @@ public void setPrefetchBufferSize(int prefetchBufferSize) { this.prefetchBufferSize = prefetchBufferSize; } + public void setOptions(Map options) { + this.options = options; + } + public void setThreadNum(int threadNum) { this.threadNum = threadNum; } @@ -249,6 +257,12 @@ private void recreateNativeReader() throws IOException { reader.setBufferSize(prefetchBufferSize); reader.setThreadNum(threadNum); + if (options != null) { + for (Map.Entry kv : options.entrySet()) { + reader.setOption(kv.getKey(), kv.getValue()); + } + } + NativeIOUtils.setNativeIOOptions(reader, this.nativeIOOptions); if (filter != null) { @@ -371,6 +385,7 @@ private void initializeInternal() throws IOException, UnsupportedOperationExcept private NativeIOOptions nativeIOOptions; private Map mergeOps = null; + private Map options = null; private final FilterPredicate filter; } diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanOldCompaction.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanOldCompaction.scala index 3d3d3b6f5..88cef540b 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanOldCompaction.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/clean/CleanOldCompaction.scala @@ -79,7 +79,7 @@ object CleanOldCompaction { |""".stripMargin sqlToDataframe(sql, spark).rdd.collect().foreach(p => { - pathSet.add(getPath(p.get(0).toString)) + pathSet.add(splitCompactFilePath(p.get(0).toString)._1) }) pathSet.foreach(p => { @@ -92,16 +92,18 @@ object CleanOldCompaction { }) } - def getPath(filePath: String): String = { + def splitCompactFilePath(filePath: String): (String, String) = { val targetString = "compact_" var directoryPath = "" + var basePath = "" val lastCompactIndex = filePath.lastIndexOf(targetString) if (lastCompactIndex != -1) { val nextDirectoryIndex = filePath.indexOf("/", lastCompactIndex) if (nextDirectoryIndex != -1) { directoryPath = filePath.substring(0, nextDirectoryIndex) + basePath = filePath.substring(nextDirectoryIndex + 1) } } - directoryPath + directoryPath -> basePath } } \ No newline at end of file diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala index 12cb3218a..4585bc506 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/spark/compaction/CompactionTask.scala @@ -24,6 +24,7 @@ object CompactionTask { val DATABASE_PARAMETER = "database" val CLEAN_OLD_COMPACTION = "clean_old_compaction" val FILE_NUM_LIMIT_PARAMETER = "file_num_limit" + val FILE_SIZE_LIMIT_PARAMETER = "file_size_limit" val NOTIFY_CHANNEL_NAME = "lakesoul_compaction_notify" val threadMap: java.util.Map[String, Integer] = new ConcurrentHashMap @@ -32,6 +33,7 @@ object CompactionTask { var database = "" var cleanOldCompaction: Option[Boolean] = Some(false) var fileNumLimit: Option[Int] = None + var fileSizeLimit: Option[String] = None def main(args: Array[String]): Unit = { @@ -44,6 +46,9 @@ object CompactionTask { if (parameter.has(FILE_NUM_LIMIT_PARAMETER)) { fileNumLimit = Some(parameter.getInt(FILE_NUM_LIMIT_PARAMETER)) } + if (parameter.has(FILE_SIZE_LIMIT_PARAMETER)) { + fileSizeLimit = Some(parameter.get(FILE_SIZE_LIMIT_PARAMETER)) + } val builder = SparkSession.builder() .config("spark.sql.parquet.mergeSchema", value = true) @@ -121,14 +126,14 @@ object CompactionTask { println("------ " + threadName + " is compressing table path is: " + path + " ------") val table = LakeSoulTable.forPath(path) if (partitionDesc == "") { - table.compaction(cleanOldCompaction = cleanOldCompaction.get, fileNumLimit = fileNumLimit) + table.compaction(cleanOldCompaction = cleanOldCompaction.get, fileNumLimit = fileNumLimit, fileSizeLimit = fileSizeLimit, force = fileSizeLimit.isEmpty) } else { val partitions = partitionDesc.split(",").map( partition => { partition.replace("=", "='") + "'" } ).mkString(" and ") - table.compaction(partitions, cleanOldCompaction = cleanOldCompaction.get, fileNumLimit = fileNumLimit) + table.compaction(partitions, cleanOldCompaction = cleanOldCompaction.get, fileNumLimit = fileNumLimit, fileSizeLimit = fileSizeLimit, force = fileSizeLimit.isEmpty) } } catch { case e: Exception => { diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala index 1681de98e..9cb5202f2 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/LakeSoulTable.scala @@ -5,7 +5,7 @@ package com.dmetasoul.lakesoul.tables import com.dmetasoul.lakesoul.meta.DBConfig.{LAKESOUL_HASH_PARTITION_SPLITTER, LAKESOUL_RANGE_PARTITION_SPLITTER} -import com.dmetasoul.lakesoul.meta.SparkMetaVersion +import com.dmetasoul.lakesoul.meta.{DBUtil, SparkMetaVersion} import com.dmetasoul.lakesoul.tables.execution.LakeSoulTableOperations import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging @@ -314,6 +314,7 @@ class LakeSoulTable(df: => Dataset[Row], snapshotManagement: SnapshotManagement) cleanOldCompaction: Boolean = false, fileNumLimit: Option[Int] = None, newBucketNum: Option[Int] = None, + fileSizeLimit: Option[String] = None, ): Unit = { val newMergeOpInfo = mergeOperatorInfo.map(m => { val key = @@ -330,7 +331,8 @@ class LakeSoulTable(df: => Dataset[Row], snapshotManagement: SnapshotManagement) (key, value) }) - executeCompaction(df, snapshotManagement, condition, force, newMergeOpInfo, hiveTableName, hivePartitionName, cleanOldCompaction, fileNumLimit, newBucketNum) + val parsedFileSizeLimit = fileSizeLimit.map(DBUtil.parseMemoryExpression) + executeCompaction(df, snapshotManagement, condition, force, newMergeOpInfo, hiveTableName, hivePartitionName, cleanOldCompaction, fileNumLimit, newBucketNum, parsedFileSizeLimit) } def setCompactionTtl(days: Int): LakeSoulTable = { diff --git a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala index 28f0d7e5b..307366654 100644 --- a/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala +++ b/lakesoul-spark/src/main/scala/com/dmetasoul/lakesoul/tables/execution/LakeSoulTableOperations.scala @@ -175,7 +175,8 @@ trait LakeSoulTableOperations extends AnalysisHelper { hivePartitionName: String = "", cleanOldCompaction: Boolean, fileNumLimit: Option[Int], - newBucketNum: Option[Int]): Unit = { + newBucketNum: Option[Int], + fileSizeLimit: Option[Long]): Unit = { toDataset(sparkSession, CompactionCommand( snapshotManagement, condition, @@ -185,7 +186,8 @@ trait LakeSoulTableOperations extends AnalysisHelper { hivePartitionName, cleanOldCompaction, fileNumLimit, - newBucketNum + newBucketNum, + fileSizeLimit )) } diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/parquet/Native/NativeMergeParquetPartitionReaderFactory.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/parquet/Native/NativeMergeParquetPartitionReaderFactory.scala index be5a552e6..a01072054 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/parquet/Native/NativeMergeParquetPartitionReaderFactory.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/parquet/Native/NativeMergeParquetPartitionReaderFactory.scala @@ -17,15 +17,15 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec -import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader} +import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.execution.datasources.parquet._ import org.apache.spark.sql.execution.datasources.v2.merge.MergePartitionedFile import org.apache.spark.sql.execution.datasources.v2.merge.parquet.batch.merge_operator.MergeOperator import org.apache.spark.sql.execution.datasources.{DataSourceUtils, RecordReaderIterator} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf.{NATIVE_IO_ENABLE, NATIVE_IO_PREFETCHER_BUFFER_SIZE, NATIVE_IO_READER_AWAIT_TIMEOUT, NATIVE_IO_THREAD_NUM} +import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf._ import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.{AtomicType, StructField, StructType} +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration @@ -70,7 +70,8 @@ case class NativeMergeParquetPartitionReaderFactory(sqlConf: SQLConf, private val nativeIOPrefecherBufferSize = sqlConf.getConf(NATIVE_IO_PREFETCHER_BUFFER_SIZE) private val nativeIOThreadNum = sqlConf.getConf(NATIVE_IO_THREAD_NUM) private val nativeIOAwaitTimeout = sqlConf.getConf(NATIVE_IO_READER_AWAIT_TIMEOUT) - + private val nativeIOCdcColumn = sqlConf.getConf(NATIVE_IO_CDC_COLUMN) + private val nativeIOIsCompacted = sqlConf.getConf(NATIVE_IO_IS_COMPACTED) // schemea: path->schema source: path->file|path->file|path->file private val requestSchemaMap: mutable.Map[String, String] = broadcastedConf.value.value @@ -174,6 +175,12 @@ case class NativeMergeParquetPartitionReaderFactory(sqlConf: SQLConf, logDebug(s"Appending $partitionSchema $partitionValues") val mergeOp = mergeOperatorInfo.map(tp => (tp._1, tp._2.toNativeName)) + val options = mutable.Map[String, String]() + if (nativeIOCdcColumn.nonEmpty) { + options += ("cdc_column" -> nativeIOCdcColumn) + } + options += ("is_compacted" -> nativeIOIsCompacted) + vectorizedReader.setOptions(options.asJava) // multi files val file = files.head diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetPartitionReaderFactory.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetPartitionReaderFactory.scala index 9645ac921..40df767cb 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetPartitionReaderFactory.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetPartitionReaderFactory.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.execution.datasources.parquet.{NativeVectorizedReader, ParquetFilters} import org.apache.spark.sql.execution.datasources.{DataSourceUtils, PartitionedFile, RecordReaderIterator} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf.{NATIVE_IO_ENABLE, NATIVE_IO_PREFETCHER_BUFFER_SIZE, NATIVE_IO_READER_AWAIT_TIMEOUT, NATIVE_IO_THREAD_NUM} +import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf.{NATIVE_IO_CDC_COLUMN, NATIVE_IO_ENABLE, NATIVE_IO_IS_COMPACTED, NATIVE_IO_PREFETCHER_BUFFER_SIZE, NATIVE_IO_READER_AWAIT_TIMEOUT, NATIVE_IO_THREAD_NUM} import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -29,7 +29,8 @@ import org.apache.spark.util.SerializableConfiguration import java.net.URI import java.time.ZoneId - +import scala.collection.JavaConverters.mutableMapAsJavaMapConverter +import scala.collection.mutable /** @@ -48,7 +49,7 @@ case class NativeParquetPartitionReaderFactory(sqlConf: SQLConf, readDataSchema: StructType, partitionSchema: StructType, filters: Array[Filter]) - extends NativeFilePartitionReaderFactory with Logging{ + extends NativeFilePartitionReaderFactory with Logging { private val isCaseSensitive = sqlConf.caseSensitiveAnalysis private val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled private val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion @@ -63,18 +64,20 @@ case class NativeParquetPartitionReaderFactory(sqlConf: SQLConf, private val nativeIOPrefecherBufferSize = sqlConf.getConf(NATIVE_IO_PREFETCHER_BUFFER_SIZE) private val nativeIOThreadNum = sqlConf.getConf(NATIVE_IO_THREAD_NUM) private val nativeIOAwaitTimeout = sqlConf.getConf(NATIVE_IO_READER_AWAIT_TIMEOUT) + private val nativeIOCdcColumn = sqlConf.getConf(NATIVE_IO_CDC_COLUMN) + private val nativeIOIsCompacted = sqlConf.getConf(NATIVE_IO_IS_COMPACTED) override def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] = { throw new Exception("LakeSoul native scan shouldn't use this method, only buildColumnarReader will be used.") } - def createVectorizedReader(file: PartitionedFile): RecordReader[Void,ColumnarBatch] = { + def createVectorizedReader(file: PartitionedFile): RecordReader[Void, ColumnarBatch] = { val recordReader = buildReaderBase(file, createParquetVectorizedReader) assert(nativeIOEnable) - val vectorizedReader=recordReader.asInstanceOf[NativeVectorizedReader] + val vectorizedReader = recordReader.asInstanceOf[NativeVectorizedReader] vectorizedReader.initBatch(partitionSchema, file.partitionValues) vectorizedReader.enableReturningBatches() - vectorizedReader.asInstanceOf[RecordReader[Void,ColumnarBatch]] + vectorizedReader.asInstanceOf[RecordReader[Void, ColumnarBatch]] } @@ -142,6 +145,7 @@ case class NativeParquetPartitionReaderFactory(sqlConf: SQLConf, } else { None } + // PARQUET_INT96_TIMESTAMP_CONVERSION says to apply timezone conversions to int96 timestamps' // *only* if the file was created by something other than "parquet-mr", so check the actual // writer here for this file. We have to do this per-file, as each file in the table may @@ -185,36 +189,44 @@ case class NativeParquetPartitionReaderFactory(sqlConf: SQLConf, convertTz: Option[ZoneId], datetimeRebaseSpec: RebaseSpec, int96RebaseSpec: RebaseSpec): - RecordReader[Void,ColumnarBatch] = - { + RecordReader[Void, ColumnarBatch] = { val taskContext = Option(TaskContext.get()) assert(nativeIOEnable) - val vectorizedReader = if (pushed.isDefined) { - new NativeVectorizedReader( - convertTz.orNull, - datetimeRebaseSpec.mode.toString, - int96RebaseSpec.mode.toString, - enableOffHeapColumnVector && taskContext.isDefined, - capacity, - pushed.get - )} else { - new NativeVectorizedReader( - convertTz.orNull, - datetimeRebaseSpec.mode.toString, - int96RebaseSpec.mode.toString, - enableOffHeapColumnVector && taskContext.isDefined, - capacity - ) - } + val vectorizedReader = if (pushed.isDefined) { + new NativeVectorizedReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + int96RebaseSpec.mode.toString, + enableOffHeapColumnVector && taskContext.isDefined, + capacity, + pushed.get + ) + } else { + new NativeVectorizedReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + int96RebaseSpec.mode.toString, + enableOffHeapColumnVector && taskContext.isDefined, + capacity + ) + } vectorizedReader.setPrefetchBufferSize(nativeIOPrefecherBufferSize) vectorizedReader.setThreadNum(nativeIOThreadNum) vectorizedReader.setAwaitTimeout(nativeIOAwaitTimeout) + val options = mutable.Map[String, String]() + + if (nativeIOCdcColumn.nonEmpty) { + options += ("cdc_column" -> nativeIOCdcColumn) + } + options += ("is_compacted" -> nativeIOIsCompacted) + vectorizedReader.setOptions(options.asJava) + val iter = new RecordReaderIterator(vectorizedReader) // SPARK-23457 Register a task completion listener before `initialization`. taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) logDebug(s"Appending $partitionSchema ${file.partitionValues}") vectorizedReader.initialize(Array(split), hadoopAttemptContext, readDataSchema) - vectorizedReader.asInstanceOf[RecordReader[Void,ColumnarBatch]] + vectorizedReader.asInstanceOf[RecordReader[Void, ColumnarBatch]] } } diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionCommit.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionCommit.scala index 5f76ccb16..b31394692 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionCommit.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionCommit.scala @@ -121,6 +121,8 @@ trait Transaction extends TransactionalWrite with Logging { protected lazy val table_path: String = tableInfo.table_path_s.get + protected val renameFiles = new ArrayBuffer[(Path, Path)] + /** * Tracks the data that could have been seen by recording the partition * predicates by which files have been queried by by this transaction. @@ -370,6 +372,26 @@ trait Transaction extends TransactionalWrite with Logging { (add_file_arr_buf.result(), add_partition_info_arr_buf.result()) } + def addRenameFile(src: Path, dst: Path): Unit = { + renameFiles += src -> dst + } + + def clearRenameFile(): Unit = renameFiles.clear() + + private def doRenameFile(): Unit = { + if (renameFiles.nonEmpty) { + val fs = renameFiles.head._1.getFileSystem(spark.sessionState.newHadoopConf()) + renameFiles.foreach(srcAndDst => fs.rename(srcAndDst._1, srcAndDst._2)) + } + } + + private def rollbackRenameFile(): Unit = { + if (renameFiles.nonEmpty) { + val fs = renameFiles.head._1.getFileSystem(spark.sessionState.newHadoopConf()) + renameFiles.foreach(srcAndDst => fs.rename(srcAndDst._2, srcAndDst._1)) + } + } + def commitDataCommitInfo(add_file_arr_buf: List[DataCommitInfo], add_partition_info_arr_buf: List[PartitionInfoScala], query_id: String, @@ -386,11 +408,16 @@ trait Transaction extends TransactionalWrite with Logging { ) try { + doRenameFile() val changeSchema = !isFirstCommit && newTableInfo.nonEmpty MetaCommit.doMetaCommit(meta_info, changeSchema) } catch { - case e: MetaRerunException => throw e - case e: Throwable => throw e + case e: MetaRerunException => + rollbackRenameFile() + throw e + case e: Throwable => + rollbackRenameFile() + throw e } committed = true diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala index 978ea21ac..8864d8048 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala @@ -84,6 +84,10 @@ trait TransactionalWrite { rangePartitionColumns } + def addRenameFiles(files: Seq[(Path, Path)]): Unit = { + files.foreach(srcAndDst => addRenameFile(srcAndDst._1, srcAndDst._2)) + } + def writeFiles(data: Dataset[_]): Seq[DataFileInfo] = writeFiles(data, None, isCompaction = false)._1 def writeFiles(data: Dataset[_], writeOptions: Option[LakeSoulOptions]): Seq[DataFileInfo] = @@ -158,7 +162,7 @@ trait TransactionalWrite { if (cdcCol.nonEmpty) { options.put("isCDC", "true") val cdcColName = cdcCol.get - if (!writeOptions.isDefined || writeOptions.get.options.getOrElse("fileNumLimit", "false").equals("false")) { + if (writeOptions.forall(_.options.getOrElse("fullCompact", "true").equals("true"))) { data.withColumn(cdcColName, when(col(cdcColName) === "update", "insert") .otherwise(col(cdcColName)) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulScanBuilder.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulScanBuilder.scala index 1fb7aabd1..dade3ce9a 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulScanBuilder.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulScanBuilder.scala @@ -4,7 +4,7 @@ package org.apache.spark.sql.lakesoul.catalog -import com.dmetasoul.lakesoul.meta.DataFileInfo +import com.dmetasoul.lakesoul.meta.{DataFileInfo, DataOperation} import org.apache.hadoop.conf.Configuration import org.apache.spark.internal.Logging import org.apache.spark.sql.{SparkSession, sources} @@ -18,6 +18,7 @@ import org.apache.spark.sql.execution.datasources.v2.merge.{MultiPartitionMergeB import org.apache.spark.sql.execution.datasources.v2.parquet.{EmptyParquetScan, NativeParquetScan, ParquetScan, StreamParquetScan} import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf +import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf.NATIVE_IO_IS_COMPACTED import org.apache.spark.sql.lakesoul.utils.{SparkUtil, TableInfo} import org.apache.spark.sql.lakesoul.{LakeSoulFileIndexV2, LakeSoulUtils} import org.apache.spark.sql.sources.Filter @@ -113,19 +114,17 @@ case class LakeSoulScanBuilder(sparkSession: SparkSession, } else { hasNoDeltaFile = fileInfo.forall(f => f._2.size <= 1) } + if (fileIndex.snapshotManagement.snapshot.getPartitionInfoArray.forall(p => p.commit_op.equals("CompactionCommit"))) { + sparkSession.sessionState.conf.setConfString(NATIVE_IO_IS_COMPACTED.key, "true") + } if (fileInfo.isEmpty) { EmptyParquetScan(sparkSession, hadoopConf, fileIndex, dataSchema, readDataSchema(), readPartitionSchema(), pushedParquetFilters, options, partitionFilters, dataFilters) } else if (tableInfo.hash_partition_columns.isEmpty) { parquetScan() } else if (onlyOnePartition) { - if (fileIndex.snapshotManagement.snapshot.getPartitionInfoArray.forall(p => p.commit_op.equals("CompactionCommit") - && p.read_files.length == 1)) { - parquetScan() - } else { - OnePartitionMergeBucketScan(sparkSession, hadoopConf, fileIndex, dataSchema, mergeReadDataSchema(), - readPartitionSchema(), pushedParquetFilters, options, tableInfo, partitionFilters, dataFilters) - } + OnePartitionMergeBucketScan(sparkSession, hadoopConf, fileIndex, dataSchema, mergeReadDataSchema(), + readPartitionSchema(), pushedParquetFilters, options, tableInfo, partitionFilters, dataFilters) } else { if (sparkSession.sessionState.conf .getConf(LakeSoulSQLConf.BUCKET_SCAN_MULTI_PARTITION_ENABLE)) { diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulTableV2.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulTableV2.scala index d35dda5d7..cfeac35d7 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulTableV2.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulTableV2.scala @@ -18,6 +18,7 @@ import org.apache.spark.sql.connector.write._ import org.apache.spark.sql.lakesoul._ import org.apache.spark.sql.lakesoul.commands.WriteIntoTable import org.apache.spark.sql.lakesoul.exception.LakeSoulErrors +import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf.NATIVE_IO_CDC_COLUMN import org.apache.spark.sql.lakesoul.sources.{LakeSoulDataSource, LakeSoulSQLConf, LakeSoulSourceUtils} import org.apache.spark.sql.lakesoul.utils.SparkUtil import org.apache.spark.sql.sources.{BaseRelation, Filter, InsertableRelation} @@ -64,7 +65,14 @@ case class LakeSoulTableV2(spark: SparkSession, // The loading of the SnapshotManagement is lazy in order to reduce the amount of FileSystem calls, // in cases where we will fallback to the V1 behavior. - lazy val snapshotManagement: SnapshotManagement = SnapshotManagement(rootPath, namespace) + lazy val snapshotManagement: SnapshotManagement = { + val mgr = SnapshotManagement(rootPath, namespace) + val cdcColumn = mgr.snapshot.getTableInfo.configuration.get(LakeSoulTableProperties.lakeSoulCDCChangePropKey) + if (cdcColumn.isDefined) { + spark.sessionState.conf.setConfString(NATIVE_IO_CDC_COLUMN.key, cdcColumn.get) + } + mgr + } override def name(): String = catalogTable.map(_.identifier.unquotedString) .orElse(tableIdentifier) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala index bdf82ce3e..5f968c119 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala @@ -6,10 +6,10 @@ package org.apache.spark.sql.lakesoul.commands import com.alibaba.fastjson.JSON import com.dmetasoul.lakesoul.meta.DBConfig.TableInfoProperty -import com.dmetasoul.lakesoul.meta.entity.DataCommitInfo +import com.dmetasoul.lakesoul.meta.entity.{DataCommitInfo, DataFileOp, FileOp} import com.dmetasoul.lakesoul.meta.{DBUtil, DataFileInfo, PartitionInfoScala, SparkMetaVersion} -import com.dmetasoul.lakesoul.spark.clean.CleanOldCompaction.cleanOldCommitOpDiskData -import org.apache.hadoop.fs.Path +import com.dmetasoul.lakesoul.spark.clean.CleanOldCompaction.{cleanOldCommitOpDiskData, splitCompactFilePath} +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.PredicateHelper import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME @@ -40,12 +40,19 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, hivePartitionName: String = "", cleanOldCompaction: Boolean, fileNumLimit: Option[Int] = None, - newBucketNum: Option[Int] = None + newBucketNum: Option[Int] = None, + fileSizeLimit: Option[Long] = None, ) extends LeafRunnableCommand with PredicateHelper with Logging { def newCompactPath: String = tableInfo.table_path.toString + "/compact_" + System.currentTimeMillis() + + def getFs(sparkSession: SparkSession, path: Path): FileSystem = { + val sessionHadoopConf = sparkSession.sessionState.newHadoopConf() + path.getFileSystem(sessionHadoopConf) + } + lazy val bucketNumChanged: Boolean = newBucketNum.exists(tableInfo.bucket_num != _) lazy val tableInfo: TableInfo = snapshotManagement.getTableInfoOnly @@ -56,8 +63,13 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, partitionInfo.read_files.length >= 1 } - def executeCompaction(spark: SparkSession, tc: TransactionCommit, files: Seq[DataFileInfo], readPartitionInfo: Array[PartitionInfoScala], compactPath: String): List[DataCommitInfo] = { - if (!newBucketNum.isDefined && readPartitionInfo.forall(p => p.commit_op.equals("CompactionCommit") && p.read_files.length == 1)) { + def executeCompaction(spark: SparkSession, + tc: TransactionCommit, + files: Seq[DataFileInfo], + readPartitionInfo: Array[PartitionInfoScala], + compactPath: String, + fullCompact: Boolean): List[DataCommitInfo] = { + if (newBucketNum.isEmpty && readPartitionInfo.forall(p => p.commit_op.equals("CompactionCommit") && p.read_files.length == 1)) { logInfo("=========== All Partitions Have Compacted, This Operation Will Cancel!!! ===========") return List.empty } @@ -109,6 +121,7 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, val map = mutable.HashMap[String, String]() map.put("isCompaction", "true") map.put("compactPath", compactPath) + map.put("fullCompact", fullCompact.toString) if (fileNumLimit.isDefined) { map.put("fileNumLimit", "true") } @@ -142,6 +155,43 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, builder.toString() } + def renameOldCompactedFile(tc: TransactionCommit, + files: Seq[DataFileInfo], + partitionDesc: String, + dstCompactPath: String): List[DataCommitInfo] = { + val srcFile = files.head + + val srcPath = new Path(srcFile.path) + val (srcCompactDir, srcBasePath) = splitCompactFilePath(srcFile.path) + val dstPath = new Path(dstCompactPath, srcBasePath) + val dstFile = srcFile.copy(path = dstPath.toString) + tc.addRenameFiles(Seq(srcPath -> dstPath)) + + val current = System.currentTimeMillis() + + List(DataCommitInfo.newBuilder() + .setTableId(tableInfo.table_id) + .setPartitionDesc(partitionDesc) + .addAllFileOps( + List(DataFileOp.newBuilder() + .setPath(dstFile.path) + .setFileOp(FileOp.add) + .setSize(dstFile.size) + .setFileExistCols(dstFile.file_exist_cols) + .build(), + DataFileOp.newBuilder() + .setPath(srcFile.path) + .setFileOp(FileOp.del) + .setSize(srcFile.size) + .setFileExistCols(srcFile.file_exist_cols) + .build(), + ).asJava + ) + .setTimestamp(current) + .setCommitted(false) + .build()) + } + def compactSinglePartition(sparkSession: SparkSession, tc: TransactionCommit, files: Seq[DataFileInfo], sourcePartition: PartitionInfoScala): String = { logInfo(s"Compacting Single Partition=${sourcePartition} with ${files.length} files") val bucketedFiles = if (tableInfo.hash_partition_columns.isEmpty || bucketNumChanged) { @@ -152,28 +202,47 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, val compactPath = newCompactPath val allDataCommitInfo = bucketedFiles.flatMap(groupByBucketId => { val (bucketId, files) = groupByBucketId - val groupedFiles = if (fileNumLimit.isDefined) { - val groupSize = fileNumLimit.get + val groupedFiles = if (fileNumLimit.isDefined || fileSizeLimit.isDefined) { val groupedFiles = new ArrayBuffer[Seq[DataFileInfo]] - for (i <- files.indices by groupSize) { - groupedFiles += files.slice(i, i + groupSize) + var groupHead = 0 + var groupSize = 0L + var groupFileCount = 0 + for (i <- files.indices) { + // each group contains at least one file + if (i == groupHead) { + groupSize += files(i).size + groupFileCount += 1 + } else if (fileSizeLimit.exists(groupSize + files(i).size > _) || fileNumLimit.exists(groupFileCount + 1 > _)) { + // if the file size limit is reached, or the file count limit is reached, we need to start a new group + groupedFiles += files.slice(groupHead, i) + groupHead = i + groupSize = files(i).size + groupFileCount = 1 + } else { + // otherwise, we add the file to the current group + groupSize += files(i).size + groupFileCount += 1 + } } + // add the last group to the groupedFiles + groupedFiles += files.slice(groupHead, files.length) groupedFiles } else { Seq(files) } + val fullCompact = groupedFiles.size == 1 groupedFiles.flatMap(files => { - lazy val hasNoDeltaFile = if (force) { + lazy val hasNoDeltaFile = if (force || newBucketNum.isDefined) { false } else { - files.forall(_.size == 1) + files.size == 1 && splitCompactFilePath(files.head.path)._1.nonEmpty } if (!hasNoDeltaFile) { - executeCompaction(sparkSession, tc, files, Array(sourcePartition), compactPath) + executeCompaction(sparkSession, tc, files, Array(sourcePartition), compactPath, fullCompact) } else { logInfo(s"== Partition ${sourcePartition.range_value} has no delta file.") - None + renameOldCompactedFile(tc, files, sourcePartition.range_value, compactPath) } }) }) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/sources/LakeSoulSQLConf.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/sources/LakeSoulSQLConf.scala index 493ce6884..caca2077f 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/sources/LakeSoulSQLConf.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/sources/LakeSoulSQLConf.scala @@ -99,17 +99,35 @@ object LakeSoulSQLConf { buildConf("native.io.enable") .doc( """ - |If ture, NativeIO would be enabled for both read and write + |If ture, NativeIO would be enabled for both read and write """.stripMargin) .booleanConf .createWithDefault(true) + val NATIVE_IO_CDC_COLUMN: ConfigEntry[String] = + buildConf("native.io.cdc_column") + .doc( + """ + |If empty, table have no cdc column + """.stripMargin) + .stringConf + .createWithDefault("") + + val NATIVE_IO_IS_COMPACTED: ConfigEntry[String] = + buildConf("native.io.is_compacted") + .doc( + """ + |If ture, Native Reader would read data as compacted data + """.stripMargin) + .stringConf + .createWithDefault("false") + val NATIVE_IO_PREFETCHER_BUFFER_SIZE: ConfigEntry[Int] = buildConf("native.io.prefetch.buffer.size") .doc( """ - |If NATIVE_IO_ENABLE=true, NATIVE_IO_PREFETCHER_BUFFER_SIZE of batches will be buffered while native-io prefetching + |If NATIVE_IO_ENABLE=true, NATIVE_IO_PREFETCHER_BUFFER_SIZE of batches will be buffered while native-io prefetching """.stripMargin) .intConf .createWithDefault(1) diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala index 1837a0c04..7d9639ae2 100644 --- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala +++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala @@ -5,7 +5,8 @@ package org.apache.spark.sql.lakesoul.commands import com.dmetasoul.lakesoul.meta.LakeSoulOptions.SHORT_TABLE_NAME -import com.dmetasoul.lakesoul.meta.{DataOperation, SparkMetaVersion} +import com.dmetasoul.lakesoul.meta.{DBUtil, DataFileInfo, DataOperation, SparkMetaVersion} +import com.dmetasoul.lakesoul.spark.clean.CleanOldCompaction.splitCompactFilePath import com.dmetasoul.lakesoul.tables.LakeSoulTable import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf @@ -18,7 +19,6 @@ import org.apache.spark.sql.lakesoul.test.{LakeSoulSQLCommandTest, LakeSoulTestS import org.apache.spark.sql.lakesoul.utils.{SparkUtil, TableInfo} import org.apache.spark.sql.test.{SharedSparkSession, TestSparkSession} import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SparkSession} -import org.apache.spark.util.Utils import org.junit.runner.RunWith import org.scalatest.BeforeAndAfterEach import org.scalatestplus.junit.JUnitRunner @@ -506,7 +506,7 @@ class CompactionSuite extends QueryTest } } - test("compaction with limited file") { + test("compaction with limited file number") { withTempDir { tempDir => // val tempDir = Utils.createDirectory(System.getProperty("java.io.tmpdir")) val tablePath = tempDir.getCanonicalPath @@ -547,7 +547,7 @@ class CompactionSuite extends QueryTest } // Get initial PartitionInfo count - val initialFileCount = getFileCount(tablePath) + val initialFileCount = getFileList(tablePath).length println(s"before compact initialPartitionInfoCount=$initialFileCount") lakeSoulTable.toDF.show @@ -555,7 +555,7 @@ class CompactionSuite extends QueryTest lakeSoulTable.compaction(fileNumLimit = Some(compactGroupSize)) // Get PartitionInfo count after compaction - val compactedFileCount = getFileCount(tablePath) + val compactedFileCount = getFileList(tablePath).length println(s"after compact compactedPartitionInfoCount=$compactedFileCount") @@ -580,17 +580,269 @@ class CompactionSuite extends QueryTest } } + test("compaction cdc table with limited file number") { + withTempDir { tempDir => + // val tempDir = org.apache.spark.util.Utils.createDirectory(System.getProperty("java.io.tmpdir")) + val tablePath = tempDir.getCanonicalPath + val spark = SparkSession.active + + val hashBucketNum = 4 + val compactRounds = 5 + val dataPerRounds = 10 + val compactGroupSize = 3 + + // Create test data + val df = Seq( + (1, "2023-01-01", 10, 1, "insert"), + (2, "2023-01-02", 20, 1, "insert"), + (3, "2023-01-03", 30, 1, "insert"), + (4, "2023-01-04", 40, 1, "insert"), + (5, "2023-01-05", 50, 1, "insert") + ).toDF("id", "date", "value", "range", "op") + + // Write initial data + df.write + .format("lakesoul") + .option("rangePartitions", "range") + .option("hashPartitions", "id") + .option(SHORT_TABLE_NAME, "compaction_limit_cdc_table") + .option("lakesoul_cdc_change_column", "op") + .option("hashBucketNum", hashBucketNum.toString) + .save(tablePath) + + val lakeSoulTable = LakeSoulTable.forPath(tablePath) + + for (c <- 0 until compactRounds) { + // Simulate multiple append operations + for (i <- c * dataPerRounds + 1 to (c + 1) * dataPerRounds) { + val appendDf = if (i % 2 == 0) { + Seq( + (i * 10, s"2023-02-0$i", i * 100, 1, "insert") + ).toDF("id", "date", "value", "range", "op") + } else { + Seq( + (i * 10, s"2023-02-0$i", i * 100, 1, "insert"), + (i * 10, s"2023-02-0$i", i * 100, 1, "delete") + ).toDF("id", "date", "value", "range", "op") + } + + lakeSoulTable.upsert(appendDf) + } + + // Get initial PartitionInfo count + val initialFiles = getFileList(tablePath) + println(initialFiles.mkString("Array(", ", ", ")")) + val initialFileCount = initialFiles.length + // println(s"before compact initialPartitionInfoCount=$initialFileCount") + lakeSoulTable.toDF.show + + // Perform limited compaction (group every compactGroupSize PartitionInfo) + lakeSoulTable.compaction(fileNumLimit = Some(compactGroupSize)) + + // Get PartitionInfo count after compaction + val compactedFiles = getFileList(tablePath) + // println(compactedFiles.mkString("Array(", ", ", ")")) + val compactedFileCount = compactedFiles.length + + println(s"after compact compactedPartitionInfoCount=$compactedFileCount") + + lakeSoulTable.toDF.show + + // Verify results + assert(compactedFileCount < initialFileCount, + s"Compaction should reduce the number of files, but it changed from ${initialFileCount} to $compactedFileCount") + + + assert(compactedFileCount >= (initialFileCount - 1) / compactGroupSize + 1, + s"Compaction should produce files above a lower bound, but there are ${compactedFileCount} files") + + assert(compactedFileCount <= (initialFileCount - 1) / compactGroupSize + 1 + hashBucketNum, + s"Compaction should produce files below a upper bound, but there are ${compactedFileCount} files") + } + + // Verify data integrity + val compactedData = lakeSoulTable.toDF.orderBy("id", "date").collect() + println(compactedData.mkString("Array(", ", ", ")")) + assert(compactedData.length == 5 + dataPerRounds * compactRounds / 2, s"The compressed data should have ${5 + dataPerRounds * compactRounds / 2} rows, but it actually has ${compactedData.length} rows") + } + } + + test("compaction with limited file size") { + withTempDir { tempDir => + // val tempDir = org.apache.spark.util.Utils.createDirectory(System.getProperty("java.io.tmpdir")) + val tablePath = tempDir.getCanonicalPath + val spark = SparkSession.active + + val hashBucketNum = 4 + val compactRounds = 5 + val upsertPerRounds = 10 + val rowsPerUpsert = 1000 + val compactFileSize = "10KB" + + // Create test data + val df = Seq( + (1, "2023-01-01", 10, 1), + (2, "2023-01-02", 20, 1), + (3, "2023-01-03", 30, 1), + (4, "2023-01-04", 40, 1), + (5, "2023-01-05", 50, 1) + ).toDF("id", "date", "value", "range") + + // Write initial data + df.write + .format("lakesoul") + .option("rangePartitions", "range") + .option("hashPartitions", "id") + .option(SHORT_TABLE_NAME, "compaction_size_limit_table") + .option("hashBucketNum", hashBucketNum.toString) + .save(tablePath) + + val lakeSoulTable = LakeSoulTable.forPath(tablePath) + + for (c <- 0 until compactRounds) { + // Simulate multiple append operations + for (i <- c * upsertPerRounds + 1 to (c + 1) * upsertPerRounds) { + val appendDf = (0 until rowsPerUpsert) + .map(j => (i * 1000 + j, s"2023-02-0$i", i * 100, 1)) + + .toDF("id", "date", "value", "range") + // val appendDf = Seq( + // (i * 10, s"2023-02-0$i", i * 100, 1) + // ).toDF("id", "date", "value", "range") + lakeSoulTable.upsert(appendDf) + } + + // Get initial PartitionInfo count + val initialMaxFileSize = getFileList(tablePath).map(_.size).max + println(s"before compact initialMaxFileSize=$initialMaxFileSize") + + // Perform limited compaction (group every compactGroupSize PartitionInfo) + LakeSoulTable.uncached(tablePath) + lakeSoulTable.compaction(fileSizeLimit = Some(compactFileSize), force = false) + + // Get PartitionInfo count after compaction + val compactedFiles = getFileList(tablePath) + val compactedFileMax = compactedFiles.map(_.size).max + + println(s"after compact compactedFileMax=$compactedFileMax") + + // Verify results + assert(compactedFileMax >= initialMaxFileSize, + s"Compaction should reduce the number of files, but it changed from ${initialMaxFileSize} to $compactedFileMax") + + assert(compactedFileMax <= DBUtil.parseMemoryExpression(compactFileSize) * 1.2, + s"Compaction should produce file with upper-bounded size, but there is a larger ${compactedFileMax} file size") + + val (compactDir, _) = splitCompactFilePath(compactedFiles.head.path) + assert(compactedFiles.forall(file => splitCompactFilePath(file.path)._1.equals(compactDir)), + s"Compaction should produce file with the same compaction dir, but the file list are ${compactedFiles.map(_.path).mkString("Array(", ", ", ")")}") + } + + // Verify data integrity + LakeSoulTable.uncached(tablePath) + val compactedData = lakeSoulTable.toDF.orderBy("id", "date").collect() + // println(compactedData.mkString("Array(", ", ", ")")) + + assert(compactedData.length == 5 + rowsPerUpsert * upsertPerRounds * compactRounds, s"The compressed data should have ${5 + rowsPerUpsert * upsertPerRounds * compactRounds} rows, but it actually has ${compactedData.length} rows") + } + } + + test("compaction cdc table with limited file size") { + withTempDir { tempDir => + // val tempDir = org.apache.spark.util.Utils.createDirectory(System.getProperty("java.io.tmpdir")) + val tablePath = tempDir.getCanonicalPath + val spark = SparkSession.active + + val hashBucketNum = 4 + val compactRounds = 5 + val upsertPerRounds = 10 + val rowsPerUpsert = 1002 + val compactFileSize = "10KB" + + // Create test data + val df = Seq( + (1, "2023-01-01", 10, 1, "insert"), + (2, "2023-01-02", 20, 1, "insert"), + (3, "2023-01-03", 30, 1, "insert"), + (4, "2023-01-04", 40, 1, "insert"), + (5, "2023-01-05", 50, 1, "insert"), + (rowsPerUpsert - 1, "2023-01-05", 50, 1, "insert") + ).toDF("id", "date", "value", "range", "op") + + // Write initial data + df.write + .format("lakesoul") + .option("rangePartitions", "range") + .option("hashPartitions", "id") + .option(SHORT_TABLE_NAME, "compaction_size_limit_cdc_table") + .option("lakesoul_cdc_change_column", "op") + .option("hashBucketNum", hashBucketNum.toString) + .save(tablePath) + + val lakeSoulTable = LakeSoulTable.forPath(tablePath) + + for (c <- 0 until compactRounds) { + // Simulate multiple append operations + for (i <- c * upsertPerRounds + 1 to (c + 1) * upsertPerRounds) { + val appendDf = (0 until rowsPerUpsert) + .flatMap(j => if (j % 2 == 0) { + Seq((i * rowsPerUpsert + j - 1, s"2023-02-0$i", i * 100, 1, "delete"), (i * rowsPerUpsert + j, s"2023-02-0$i", i * 100, 1, "insert")) + } else { + Seq((i * rowsPerUpsert + j, s"2023-02-0$i", i * 100, 1, "insert")) + }) + + .toDF("id", "date", "value", "range", "op") + // val appendDf = Seq( + // (i * 10, s"2023-02-0$i", i * 100, 1) + // ).toDF("id", "date", "value", "range") + lakeSoulTable.upsert(appendDf) + } + + // Get initial PartitionInfo count + val initialMaxFileSize = getFileList(tablePath).map(_.size).max + println(s"before compact initialMaxFileSize=$initialMaxFileSize") + + // Perform limited compaction (group every compactGroupSize PartitionInfo) + LakeSoulTable.uncached(tablePath) + lakeSoulTable.compaction(fileSizeLimit = Some(compactFileSize), force = false) + + // Get PartitionInfo count after compaction + val compactedFiles = getFileList(tablePath) + val compactedFileMax = compactedFiles.map(_.size).max + + println(s"after compact compactedFileMax=$compactedFileMax") + + // Verify results + // assert(compactedFileMax >= initialMaxFileSize, + // s"Compaction should reduce the number of files, but it changed from ${initialMaxFileSize} to $compactedFileMax") + + assert(compactedFileMax <= DBUtil.parseMemoryExpression(compactFileSize) * 1.2, + s"Compaction should produce file with upper-bounded size, but there is a larger ${compactedFileMax} file size") + + val (compactDir, _) = splitCompactFilePath(compactedFiles.head.path) + assert(compactedFiles.forall(file => splitCompactFilePath(file.path)._1.equals(compactDir)), + s"Compaction should produce file with the same compaction dir, but the file list are ${compactedFiles.map(_.path).mkString("Array(", ", ", ")")}") + } + + // Verify data integrity + LakeSoulTable.uncached(tablePath) + val compactedData = lakeSoulTable.toDF.orderBy("id", "date").collect() + // println(compactedData.mkString("Array(", ", ", ")")) + + assert(compactedData.length == 6 + rowsPerUpsert * upsertPerRounds * compactRounds / 2, s"The compressed data should have ${6 + rowsPerUpsert * upsertPerRounds * compactRounds / 2} rows, but it actually has ${compactedData.length} rows") + } + } + // Auxiliary method: Get the number of files - def getFileCount(tablePath: String): Int = { + def getFileList(tablePath: String): Array[DataFileInfo] = { val sm = SnapshotManagement(SparkUtil.makeQualifiedTablePath(new Path(tablePath)).toString) val partitionList = SparkMetaVersion.getAllPartitionInfo(sm.getTableInfoOnly.table_id) - val files = DataOperation.getTableDataInfo(partitionList) - files.length + DataOperation.getTableDataInfo(partitionList) } test("compaction with newBucketNum") { withTempDir { tempDir => - // val tempDir = Utils.createDirectory(System.getProperty("java.io.tmpdir")) + // val tempDir = org.apache.spark.util.Utils.createDirectory(System.getProperty("java.io.tmpdir")) val tablePath = tempDir.getCanonicalPath val spark = SparkSession.active @@ -638,12 +890,12 @@ class CompactionSuite extends QueryTest // ).toDF("id", "date", "value", "range", "op") lakeSoulTable.upsert(appendDf) } - assert(getFileBucketSet(tablePath).size == hashBucketNum) + assert(getFileList(tablePath).groupBy(_.file_bucket_id).keys.toSet.size == hashBucketNum) assert(getTableInfo(tablePath).bucket_num == hashBucketNum) lakeSoulTable.compaction(newBucketNum = Some(newHashBucketNum)) - assert(getFileBucketSet(tablePath).size == newHashBucketNum) + assert(getFileList(tablePath).groupBy(_.file_bucket_id).keys.toSet.size == newHashBucketNum) assert(getTableInfo(tablePath).bucket_num == newHashBucketNum) val compactedData = lakeSoulTable.toDF.orderBy("id", "date").collect() @@ -653,14 +905,6 @@ class CompactionSuite extends QueryTest } } - // Auxiliary method: Get the bucket number of table - def getFileBucketSet(tablePath: String): Set[Int] = { - val sm = SnapshotManagement(SparkUtil.makeQualifiedTablePath(new Path(tablePath)).toString) - val partitionList = SparkMetaVersion.getAllPartitionInfo(sm.getTableInfoOnly.table_id) - val files = DataOperation.getTableDataInfo(partitionList) - println(files.mkString("Array(", ", ", ")")) - files.groupBy(_.file_bucket_id).keys.toSet - } // Auxiliary method: Get the bucket number of table def getTableInfo(tablePath: String): TableInfo = { diff --git a/rust/lakesoul-io/src/datasource/physical_plan/merge.rs b/rust/lakesoul-io/src/datasource/physical_plan/merge.rs index 7e7f53aed..001b0c4be 100644 --- a/rust/lakesoul-io/src/datasource/physical_plan/merge.rs +++ b/rust/lakesoul-io/src/datasource/physical_plan/merge.rs @@ -32,6 +32,7 @@ pub struct MergeParquetExec { default_column_value: Arc>, merge_operators: Arc>, inputs: Vec>, + io_config: LakeSoulIOConfig, } impl MergeParquetExec { @@ -71,6 +72,7 @@ impl MergeParquetExec { .collect::>(), )); + let config = io_config.clone(); let primary_keys = Arc::new(io_config.primary_keys); let default_column_value = Arc::new(io_config.default_column_value); let merge_operators: Arc> = Arc::new(io_config.merge_operators); @@ -81,6 +83,7 @@ impl MergeParquetExec { primary_keys, default_column_value, merge_operators, + io_config: config }) } @@ -90,6 +93,7 @@ impl MergeParquetExec { io_config: LakeSoulIOConfig, default_column_value: Arc>, ) -> Result { + let config = io_config.clone(); let primary_keys = Arc::new(io_config.primary_keys); let merge_operators = Arc::new(io_config.merge_operators); @@ -99,6 +103,7 @@ impl MergeParquetExec { primary_keys, default_column_value, merge_operators, + io_config: config }) } @@ -149,6 +154,7 @@ impl ExecutionPlan for MergeParquetExec { primary_keys: self.primary_keys(), default_column_value: self.default_column_value(), merge_operators: self.merge_operators(), + io_config: self.io_config.clone() })) } @@ -180,6 +186,7 @@ impl ExecutionPlan for MergeParquetExec { self.default_column_value(), self.merge_operators(), context.session_config().batch_size(), + self.io_config.clone(), )?; Ok(merged_stream) @@ -193,8 +200,18 @@ pub fn merge_stream( default_column_value: Arc>, merge_operators: Arc>, batch_size: usize, + config: LakeSoulIOConfig, ) -> Result { - let merge_stream = if primary_keys.is_empty() { + let merge_on_read = if config.files.len() == 1 { + if config.primary_keys.is_empty() { + false + } else { + !config.merge_operators.is_empty() || !config.is_compacted() + } + } else { + true + }; + let merge_stream = if !merge_on_read { Box::pin(DefaultColumnStream::new_from_streams_with_default( streams, schema, diff --git a/rust/lakesoul-io/src/lakesoul_io_config.rs b/rust/lakesoul-io/src/lakesoul_io_config.rs index 0fe6fb07b..8a87d71ed 100644 --- a/rust/lakesoul-io/src/lakesoul_io_config.rs +++ b/rust/lakesoul-io/src/lakesoul_io_config.rs @@ -47,6 +47,8 @@ pub static OPTION_DEFAULT_VALUE_KEEP_ORDERS: &str = "false"; pub static OPTION_KEY_MEM_LIMIT: &str = "mem_limit"; pub static OPTION_KEY_POOL_SIZE: &str = "pool_size"; pub static OPTION_KEY_HASH_BUCKET_ID: &str = "hash_bucket_id"; +pub static OPTION_KEY_CDC_COLUMN: &str = "cdc_column"; +pub static OPTION_KEY_IS_COMPACTED: &str = "is_compacted"; pub static OPTION_KEY_MAX_FILE_SIZE: &str = "max_file_size"; #[derive(Debug, Derivative)] @@ -177,6 +179,14 @@ impl LakeSoulIOConfig { pub fn hash_bucket_id(&self) -> usize { self.option(OPTION_KEY_HASH_BUCKET_ID).map_or(0, |x| x.parse().unwrap()) } + + pub fn cdc_column(&self) -> Option { + self.option(OPTION_KEY_CDC_COLUMN).map(|x| x.to_string()) + } + + pub fn is_compacted(&self) -> bool { + self.option(OPTION_KEY_IS_COMPACTED).map_or(false, |x| x.eq("true")) + } } #[derive(Derivative, Debug)] From 2004a6f74c4a3968b4ba930236d3c89e93f587b4 Mon Sep 17 00:00:00 2001 From: zenghua Date: Fri, 25 Oct 2024 11:58:51 +0800 Subject: [PATCH 29/33] fix LakeSoulFileWriter bucketId Signed-off-by: zenghua --- .../datasources/LakeSoulFileWriter.scala | 22 ++- .../lakesoul/DelayedCopyCommitProtocol.scala | 140 ++++++++++++++++++ 2 files changed, 157 insertions(+), 5 deletions(-) create mode 100644 lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCopyCommitProtocol.scala diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala index 25d140fc8..a52a0006b 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala @@ -239,7 +239,7 @@ object LakeSoulFileWriter extends Logging { description = description, jobIdInstant = jobIdInstant, sparkStageId = taskContext.stageId(), - sparkPartitionId = if (isCompaction && staticBucketId != -1) staticBucketId else taskContext.partitionId(), + sparkPartitionId = if (isCompaction && staticBucketId != -1) staticBucketId else -1, sparkAttemptNumber = taskContext.taskAttemptId().toInt & Integer.MAX_VALUE, committer, iterator = iter, @@ -400,13 +400,25 @@ object LakeSoulFileWriter extends Logging { private val partValue: Option[String] = options.get("partValue").filter(_ != LAKESOUL_NON_PARTITION_TABLE_PART_DESC) .map(_.replace(LAKESOUL_RANGE_PARTITION_SPLITTER, "/")) - private def newOutputWriter(): Unit = { + /** Given an input row, returns the corresponding `bucketId` */ + protected lazy val getBucketId: InternalRow => Int = { + val proj = + UnsafeProjection.create(Seq(description.bucketSpec.get.bucketIdExpression), + description.allColumns) + row => proj(row).getInt(0) + } + + private def newOutputWriter(record: InternalRow): Unit = { recordsInFile = 0 releaseResources() val ext = description.outputWriterFactory.getFileExtension(taskAttemptContext) val suffix = if (bucketSpec.isDefined) { - val bucketIdStr = BucketingUtils.bucketIdToString(partitionId) + val bucketIdStr = if (partitionId == -1) { + BucketingUtils.bucketIdToString(getBucketId(record)) + } else { + BucketingUtils.bucketIdToString(partitionId) + } f"$bucketIdStr.c$fileCounter%03d" + ext } else { f"-c$fileCounter%03d" + ext @@ -427,13 +439,13 @@ object LakeSoulFileWriter extends Logging { override def write(record: InternalRow): Unit = { if (currentWriter == null) { - newOutputWriter() + newOutputWriter(record) } else if (description.maxRecordsPerFile > 0 && recordsInFile >= description.maxRecordsPerFile) { fileCounter += 1 assert(fileCounter < MAX_FILE_COUNTER, s"File counter $fileCounter is beyond max value $MAX_FILE_COUNTER") - newOutputWriter() + newOutputWriter(record) } currentWriter.write(record) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCopyCommitProtocol.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCopyCommitProtocol.scala new file mode 100644 index 000000000..086012cf0 --- /dev/null +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCopyCommitProtocol.scala @@ -0,0 +1,140 @@ +// SPDX-FileCopyrightText: 2023 LakeSoul Contributors +// +// SPDX-License-Identifier: Apache-2.0 + +package org.apache.spark.sql.lakesoul + +import com.dmetasoul.lakesoul.meta.{DataFileInfo, MetaUtils} + +import java.net.URI +import java.util.UUID +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.io.FileCommitProtocol +import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage +import org.apache.spark.sql.catalyst.expressions.Cast +import org.apache.spark.sql.lakesoul.utils.{DateFormatter, PartitionUtils, TimestampFormatter} +import org.apache.spark.sql.types.StringType + +import scala.collection.mutable.ArrayBuffer +import scala.util.Random + +/** + * Writes out the files to `path` and returns a list of them in `addedStatuses`. + */ +class DelayedCommitProtocol(jobId: String, + path: String, + randomPrefixLength: Option[Int]) + extends FileCommitProtocol + with Serializable with Logging { + + // Track the list of files added by a task, only used on the executors. + @transient private var addedFiles: ArrayBuffer[(List[(String, String)], String)] = _ + @transient val addedStatuses = new ArrayBuffer[DataFileInfo] + + val timestampPartitionPattern = "yyyy-MM-dd HH:mm:ss[.S]" + + + override def setupJob(jobContext: JobContext): Unit = { + + } + + override def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit = { + val fileStatuses = taskCommits.flatMap(_.obj.asInstanceOf[Seq[DataFileInfo]]).toArray + addedStatuses ++= fileStatuses + } + + override def abortJob(jobContext: JobContext): Unit = { + // TODO: Best effort cleanup + } + + override def setupTask(taskContext: TaskAttemptContext): Unit = { + addedFiles = new ArrayBuffer[(List[(String, String)], String)] + } + + protected def getFileName(taskContext: TaskAttemptContext, ext: String): String = { + // The file name looks like part-r-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003.gz.parquet + // Note that %05d does not truncate the split number, so if we have more than 100000 tasks, + // the file name is fine and won't overflow. + val split = taskContext.getTaskAttemptID.getTaskID.getId + val uuid = UUID.randomUUID.toString + f"part-$split%05d-$uuid$ext" + } + + protected def parsePartitions(dir: String): List[(String, String)] = { + // TODO: timezones? + // TODO: enable validatePartitionColumns? + val dateFormatter = DateFormatter() + val timestampFormatter = + TimestampFormatter(timestampPartitionPattern, java.util.TimeZone.getDefault) + val parsedPartition = + PartitionUtils + .parsePartition( + new Path(dir), + typeInference = false, + Set.empty, + Map.empty, + validatePartitionColumns = false, + java.util.TimeZone.getDefault, + dateFormatter, + timestampFormatter) + ._1 + .get + parsedPartition.columnNames.zip(parsedPartition.literals.map(l => Cast(l, StringType).eval()).map(Option(_).map(_.toString).orNull)).toList + } + + /** Generates a string created of `randomPrefixLength` alphanumeric characters. */ + private def getRandomPrefix(numChars: Int): String = { + Random.alphanumeric.take(numChars).mkString + } + + override def newTaskTempFile(taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = { + val filename = getFileName(taskContext, ext) + val partitionValues = dir.map(parsePartitions).getOrElse(List.empty[(String, String)]) + val unescapedDir = if (partitionValues.nonEmpty) { + Some(partitionValues.map(partitionValue => partitionValue._1 + "=" + partitionValue._2).mkString("/")) + } else { + dir + } + val relativePath = randomPrefixLength.map { prefixLength => + getRandomPrefix(prefixLength) // Generate a random prefix as a first choice + }.orElse { + // or else write into the partition unescaped directory if it is partitioned + unescapedDir + }.map { subDir => + new Path(subDir, filename) + }.getOrElse(new Path(filename)) // or directly write out to the output path + + val absolutePath = new Path(path, relativePath).toUri.toString + //returns the absolute path to the file + addedFiles.append((partitionValues, absolutePath)) + absolutePath + } + + override def newTaskTempFileAbsPath(taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { + throw new UnsupportedOperationException( + s"$this does not support adding files with an absolute path") + } + + override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = { + + if (addedFiles.nonEmpty) { + val fs = new Path(path, addedFiles.head._2).getFileSystem(taskContext.getConfiguration) + val statuses: Seq[DataFileInfo] = addedFiles.map { f => + + val filePath = new Path(new URI(f._2)) + val stat = fs.getFileStatus(filePath) + DataFileInfo(MetaUtils.getPartitionKeyFromList(f._1), fs.makeQualified(filePath).toString, "add", stat.getLen, stat.getModificationTime) + } + + new TaskCommitMessage(statuses) + } else { + new TaskCommitMessage(Nil) + } + } + + override def abortTask(taskContext: TaskAttemptContext): Unit = { + // TODO: we can also try delete the addedFiles as a best-effort cleanup. + } +} From 7c2b17442711cc64126c043d8e4182a4f48cd50a Mon Sep 17 00:00:00 2001 From: zenghua Date: Fri, 25 Oct 2024 17:03:19 +0800 Subject: [PATCH 30/33] copy compacted file by fs directly instead of spark read&write Signed-off-by: zenghua --- .../datasources/LakeSoulFileWriter.scala | 82 ++++++++++---- .../lakesoul/DelayedCopyCommitProtocol.scala | 104 ++++-------------- .../sql/lakesoul/TransactionalWrite.scala | 17 ++- .../lakesoul/commands/CompactionCommand.scala | 42 ++++--- .../lakesoul/sources/LakeSoulSQLConf.scala | 9 ++ 5 files changed, 127 insertions(+), 127 deletions(-) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala index a52a0006b..880a99649 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala @@ -16,7 +16,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils} import org.apache.spark.rdd.RDD import org.apache.spark.shuffle.FetchFailedException -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec @@ -34,6 +34,9 @@ import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf import org.apache.spark.sql.vectorized.ArrowFakeRowAdaptor import org.apache.spark.util.{SerializableConfiguration, Utils} import com.dmetasoul.lakesoul.meta.DBConfig.{LAKESOUL_NON_PARTITION_TABLE_PART_DESC, LAKESOUL_RANGE_PARTITION_SPLITTER} +import com.dmetasoul.lakesoul.spark.clean.CleanOldCompaction.splitCompactFilePath +import org.apache.spark.sql.lakesoul.DelayedCopyCommitProtocol +import org.apache.spark.sql.types.{DataTypes, StringType, StructField, StructType} import java.util.{Date, UUID} @@ -199,28 +202,32 @@ object LakeSoulFileWriter extends Logging { try { // for compaction, we won't break ordering from batch scan - val (rdd, concurrentOutputWriterSpec) = if (!isBucketNumChanged && (orderingMatched || isCompaction)) { - (nativeWrap(empty2NullPlan), None) - } else { - // SPARK-21165: the `requiredOrdering` is based on the attributes from analyzed plan, and - // the physical plan may have different attribute ids due to optimizer removing some - // aliases. Here we bind the expression ahead to avoid potential attribute ids mismatch. - val orderingExpr = bindReferences( - requiredOrdering.map(SortOrder(_, Ascending)), finalOutputSpec.outputColumns) - val sortPlan = SortExec( - orderingExpr, - global = false, - child = empty2NullPlan) - - val maxWriters = sparkSession.sessionState.conf.maxConcurrentOutputFileWriters - val concurrentWritersEnabled = maxWriters > 0 && sortColumns.isEmpty - if (concurrentWritersEnabled) { - (empty2NullPlan.execute(), - Some(ConcurrentOutputWriterSpec(maxWriters, () => sortPlan.createSorter()))) + val (rdd, concurrentOutputWriterSpec) = + if (isCompaction && options.getOrElse("copyCompactedFile", "").nonEmpty) { + val data = Seq(InternalRow(options("copyCompactedFile"))) + (sparkSession.sparkContext.parallelize(data), None) + } else if (!isBucketNumChanged && (orderingMatched || isCompaction)) { + (nativeWrap(empty2NullPlan), None) } else { - (nativeWrap(sortPlan), None) + // SPARK-21165: the `requiredOrdering` is based on the attributes from analyzed plan, and + // the physical plan may have different attribute ids due to optimizer removing some + // aliases. Here we bind the expression ahead to avoid potential attribute ids mismatch. + val orderingExpr = bindReferences( + requiredOrdering.map(SortOrder(_, Ascending)), finalOutputSpec.outputColumns) + val sortPlan = SortExec( + orderingExpr, + global = false, + child = empty2NullPlan) + + val maxWriters = sparkSession.sessionState.conf.maxConcurrentOutputFileWriters + val concurrentWritersEnabled = maxWriters > 0 && sortColumns.isEmpty + if (concurrentWritersEnabled) { + (empty2NullPlan.execute(), + Some(ConcurrentOutputWriterSpec(maxWriters, () => sortPlan.createSorter()))) + } else { + (nativeWrap(sortPlan), None) + } } - } // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single // partition rdd to make sure we at least set up one write task to write the metadata. @@ -317,7 +324,11 @@ object LakeSoulFileWriter extends Logging { } else if (description.partitionColumns.isEmpty && description.bucketSpec.isEmpty && !isCompaction) { new SingleDirectoryDataWriter(description, taskAttemptContext, committer) } else if (isCompaction) { - new StaticPartitionedDataWriter(description, taskAttemptContext, committer, options, sparkPartitionId, bucketSpec) + if (committer.isInstanceOf[DelayedCopyCommitProtocol]) { + new CopyFileWriter(description, taskAttemptContext, committer, options) + } else { + new StaticPartitionedDataWriter(description, taskAttemptContext, committer, options, sparkPartitionId, bucketSpec) + } } else { concurrentOutputWriterSpec match { case Some(spec) => @@ -453,4 +464,31 @@ object LakeSoulFileWriter extends Logging { recordsInFile += 1 } } + + private class CopyFileWriter( + description: WriteJobDescription, + taskAttemptContext: TaskAttemptContext, + committer: FileCommitProtocol, + options: Map[String, String], + customMetrics: Map[String, SQLMetric] = Map.empty) + extends FileFormatDataWriter(description, taskAttemptContext, committer, customMetrics) { + + private val partValue: Option[String] = options.get("partValue").filter(_ != LAKESOUL_NON_PARTITION_TABLE_PART_DESC) + .map(_.replace(LAKESOUL_RANGE_PARTITION_SPLITTER, "/")) + + /** Given an input row, returns the corresponding `bucketId` */ + protected lazy val getSrcPath: InternalRow => String = { + row => row.get(0, StringType).asInstanceOf[String] + } + + override def write(record: InternalRow): Unit = { + val dstPath = committer.newTaskTempFile( + taskAttemptContext, + partValue, + getSrcPath(record)) + + statsTrackers.foreach(_.newFile(dstPath)) + } + } + } diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCopyCommitProtocol.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCopyCommitProtocol.scala index 086012cf0..b46be810e 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCopyCommitProtocol.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCopyCommitProtocol.scala @@ -5,10 +5,8 @@ package org.apache.spark.sql.lakesoul import com.dmetasoul.lakesoul.meta.{DataFileInfo, MetaUtils} - -import java.net.URI -import java.util.UUID -import org.apache.hadoop.fs.Path +import com.dmetasoul.lakesoul.spark.clean.CleanOldCompaction.splitCompactFilePath +import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol @@ -17,99 +15,39 @@ import org.apache.spark.sql.catalyst.expressions.Cast import org.apache.spark.sql.lakesoul.utils.{DateFormatter, PartitionUtils, TimestampFormatter} import org.apache.spark.sql.types.StringType +import java.net.URI +import java.util.UUID import scala.collection.mutable.ArrayBuffer import scala.util.Random /** * Writes out the files to `path` and returns a list of them in `addedStatuses`. */ -class DelayedCommitProtocol(jobId: String, - path: String, - randomPrefixLength: Option[Int]) - extends FileCommitProtocol +class DelayedCopyCommitProtocol(jobId: String, + dstPath: String, + randomPrefixLength: Option[Int]) + extends DelayedCommitProtocol(jobId, dstPath, randomPrefixLength) with Serializable with Logging { - // Track the list of files added by a task, only used on the executors. - @transient private var addedFiles: ArrayBuffer[(List[(String, String)], String)] = _ - @transient val addedStatuses = new ArrayBuffer[DataFileInfo] - - val timestampPartitionPattern = "yyyy-MM-dd HH:mm:ss[.S]" - + @transient private var copyFiles: ArrayBuffer[(String, String)] = _ override def setupJob(jobContext: JobContext): Unit = { } - override def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit = { - val fileStatuses = taskCommits.flatMap(_.obj.asInstanceOf[Seq[DataFileInfo]]).toArray - addedStatuses ++= fileStatuses - } - override def abortJob(jobContext: JobContext): Unit = { // TODO: Best effort cleanup } override def setupTask(taskContext: TaskAttemptContext): Unit = { - addedFiles = new ArrayBuffer[(List[(String, String)], String)] + copyFiles = new ArrayBuffer[(String, String)] } - protected def getFileName(taskContext: TaskAttemptContext, ext: String): String = { - // The file name looks like part-r-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003.gz.parquet - // Note that %05d does not truncate the split number, so if we have more than 100000 tasks, - // the file name is fine and won't overflow. - val split = taskContext.getTaskAttemptID.getTaskID.getId - val uuid = UUID.randomUUID.toString - f"part-$split%05d-$uuid$ext" - } - - protected def parsePartitions(dir: String): List[(String, String)] = { - // TODO: timezones? - // TODO: enable validatePartitionColumns? - val dateFormatter = DateFormatter() - val timestampFormatter = - TimestampFormatter(timestampPartitionPattern, java.util.TimeZone.getDefault) - val parsedPartition = - PartitionUtils - .parsePartition( - new Path(dir), - typeInference = false, - Set.empty, - Map.empty, - validatePartitionColumns = false, - java.util.TimeZone.getDefault, - dateFormatter, - timestampFormatter) - ._1 - .get - parsedPartition.columnNames.zip(parsedPartition.literals.map(l => Cast(l, StringType).eval()).map(Option(_).map(_.toString).orNull)).toList - } - - /** Generates a string created of `randomPrefixLength` alphanumeric characters. */ - private def getRandomPrefix(numChars: Int): String = { - Random.alphanumeric.take(numChars).mkString - } override def newTaskTempFile(taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = { - val filename = getFileName(taskContext, ext) - val partitionValues = dir.map(parsePartitions).getOrElse(List.empty[(String, String)]) - val unescapedDir = if (partitionValues.nonEmpty) { - Some(partitionValues.map(partitionValue => partitionValue._1 + "=" + partitionValue._2).mkString("/")) - } else { - dir - } - val relativePath = randomPrefixLength.map { prefixLength => - getRandomPrefix(prefixLength) // Generate a random prefix as a first choice - }.orElse { - // or else write into the partition unescaped directory if it is partitioned - unescapedDir - }.map { subDir => - new Path(subDir, filename) - }.getOrElse(new Path(filename)) // or directly write out to the output path - - val absolutePath = new Path(path, relativePath).toUri.toString - //returns the absolute path to the file - addedFiles.append((partitionValues, absolutePath)) - absolutePath + val (srcCompactDir, srcBasePath) = splitCompactFilePath(ext) + copyFiles += dir.getOrElse("-5") -> ext + new Path(dstPath, srcBasePath).toString } override def newTaskTempFileAbsPath(taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { @@ -119,13 +57,15 @@ class DelayedCommitProtocol(jobId: String, override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = { - if (addedFiles.nonEmpty) { - val fs = new Path(path, addedFiles.head._2).getFileSystem(taskContext.getConfiguration) - val statuses: Seq[DataFileInfo] = addedFiles.map { f => - - val filePath = new Path(new URI(f._2)) - val stat = fs.getFileStatus(filePath) - DataFileInfo(MetaUtils.getPartitionKeyFromList(f._1), fs.makeQualified(filePath).toString, "add", stat.getLen, stat.getModificationTime) + if (copyFiles.nonEmpty) { + val fs = new Path(copyFiles.head._2).getFileSystem(taskContext.getConfiguration) + val statuses = copyFiles.map { f => + val (partitionDesc, srcPath) = f + val (srcCompactDir, srcBasePath) = splitCompactFilePath(srcPath) + val dstFile = new Path(dstPath, srcBasePath) + FileUtil.copy(fs, new Path(srcPath), fs, new Path(dstPath, srcBasePath), false, taskContext.getConfiguration) + val status = fs.getFileStatus(dstFile) + DataFileInfo(partitionDesc, fs.makeQualified(dstFile).toString, "add", status.getLen, status.getModificationTime) } new TaskCommitMessage(statuses) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala index 8864d8048..57248a58a 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala @@ -8,6 +8,7 @@ import com.dmetasoul.lakesoul.meta.DBConfig.{LAKESOUL_EMPTY_STRING, LAKESOUL_NUL import com.dmetasoul.lakesoul.meta.{CommitType, DataFileInfo} import com.dmetasoul.lakesoul.meta.entity.DataCommitInfo import org.apache.hadoop.fs.Path +import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.Dataset import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.Attribute @@ -145,12 +146,12 @@ trait TransactionalWrite { var outputPath = SparkUtil.makeQualifiedTablePath(tableInfo.table_path) if (isCompaction) { - val compactPath = if (writeOptions.isDefined) { - writeOptions.get.options.getOrElse("compactPath", tableInfo.table_path.toString + "/compact_" + System.currentTimeMillis()) + val compactionPath = if (writeOptions.isDefined) { + writeOptions.get.options.getOrElse("compactionPath", tableInfo.table_path.toString + "/compact_" + System.currentTimeMillis()) } else { tableInfo.table_path.toString + "/compact_" + System.currentTimeMillis() } - outputPath = SparkUtil.makeQualifiedTablePath(new Path(compactPath)) + outputPath = SparkUtil.makeQualifiedTablePath(new Path(compactionPath)) } val dc = if (isCompaction) { if (bucketNumChanged) { @@ -162,7 +163,7 @@ trait TransactionalWrite { if (cdcCol.nonEmpty) { options.put("isCDC", "true") val cdcColName = cdcCol.get - if (writeOptions.forall(_.options.getOrElse("fullCompact", "true").equals("true"))) { + if (writeOptions.forall(_.options.getOrElse("fullCompaction", "true").equals("true"))) { data.withColumn(cdcColName, when(col(cdcColName) === "update", "insert") .otherwise(col(cdcColName)) @@ -213,7 +214,13 @@ trait TransactionalWrite { output.length < data.schema.size) } - val committer = getCommitter(outputPath) + val committer = if (writeOptions.exists(_.options.getOrElse("copyCompactedFile", "").nonEmpty)) { + val srcPath = writeOptions.get.options.get("copyCompactedFile") + options.put("copyCompactedFile", srcPath.get) + new DelayedCopyCommitProtocol("lakesoul", outputPath.toString, None) + } else { + getCommitter(outputPath) + } SQLExecution.withNewExecutionId(queryExecution) { val outputSpec = LakeSoulFileWriter.OutputSpec( diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala index 5f968c119..f74a35bf5 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala @@ -20,6 +20,7 @@ import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, Data import org.apache.spark.sql.functions.{expr, forall} import org.apache.spark.sql.lakesoul.catalog.LakeSoulTableV2 import org.apache.spark.sql.lakesoul.exception.LakeSoulErrors +import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf.RENAME_COMPACTED_FILE import org.apache.spark.sql.lakesoul.utils.TableInfo import org.apache.spark.sql.lakesoul.{BatchDataSoulFileIndexV2, LakeSoulOptions, SnapshotManagement, TransactionCommit} import org.apache.spark.sql.types.StructType @@ -67,8 +68,9 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, tc: TransactionCommit, files: Seq[DataFileInfo], readPartitionInfo: Array[PartitionInfoScala], - compactPath: String, - fullCompact: Boolean): List[DataCommitInfo] = { + compactionPath: String, + fullCompaction: Boolean, + copyCompactedFile: String = ""): List[DataCommitInfo] = { if (newBucketNum.isEmpty && readPartitionInfo.forall(p => p.commit_op.equals("CompactionCommit") && p.read_files.length == 1)) { logInfo("=========== All Partitions Have Compacted, This Operation Will Cancel!!! ===========") return List.empty @@ -120,10 +122,10 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, tc.setReadFiles(newReadFiles) val map = mutable.HashMap[String, String]() map.put("isCompaction", "true") - map.put("compactPath", compactPath) - map.put("fullCompact", fullCompact.toString) - if (fileNumLimit.isDefined) { - map.put("fileNumLimit", "true") + map.put("compactionPath", compactionPath) + map.put("fullCompaction", fullCompaction.toString) + if (copyCompactedFile.nonEmpty) { + map.put("copyCompactedFile", copyCompactedFile) } if (readPartitionInfo.nonEmpty) { map.put("partValue", readPartitionInfo.head.range_value) @@ -156,10 +158,9 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, } def renameOldCompactedFile(tc: TransactionCommit, - files: Seq[DataFileInfo], + srcFile: DataFileInfo, partitionDesc: String, dstCompactPath: String): List[DataCommitInfo] = { - val srcFile = files.head val srcPath = new Path(srcFile.path) val (srcCompactDir, srcBasePath) = splitCompactFilePath(srcFile.path) @@ -199,7 +200,7 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, } else { files.groupBy(_.file_bucket_id) } - val compactPath = newCompactPath + val compactionPath = newCompactPath val allDataCommitInfo = bucketedFiles.flatMap(groupByBucketId => { val (bucketId, files) = groupByBucketId val groupedFiles = if (fileNumLimit.isDefined || fileSizeLimit.isDefined) { @@ -230,19 +231,24 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, } else { Seq(files) } - val fullCompact = groupedFiles.size == 1 + val fullCompaction = groupedFiles.size == 1 groupedFiles.flatMap(files => { - lazy val hasNoDeltaFile = if (force || newBucketNum.isDefined) { + lazy val incrementFiles = if (force || newBucketNum.isDefined) { false } else { files.size == 1 && splitCompactFilePath(files.head.path)._1.nonEmpty } - if (!hasNoDeltaFile) { - executeCompaction(sparkSession, tc, files, Array(sourcePartition), compactPath, fullCompact) + if (!incrementFiles) { + executeCompaction(sparkSession, tc, files, Array(sourcePartition), compactionPath, fullCompaction) } else { - logInfo(s"== Partition ${sourcePartition.range_value} has no delta file.") - renameOldCompactedFile(tc, files, sourcePartition.range_value, compactPath) + logInfo(s"== Partition ${sourcePartition.range_value} has no increment file.") + val origCompactedFile = files.head + if (sparkSession.sessionState.conf.getConf(RENAME_COMPACTED_FILE)) { + renameOldCompactedFile(tc, origCompactedFile, sourcePartition.range_value, compactionPath) + } else { + executeCompaction(sparkSession, tc, files, Array(sourcePartition), compactionPath, fullCompaction, origCompactedFile.path) + } } }) }) @@ -273,10 +279,10 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, spark.sessionState.catalogManager.setCurrentCatalog(SESSION_CATALOG_NAME) if (hivePartitionName.nonEmpty) { spark.sql(s"ALTER TABLE $hiveTableName DROP IF EXISTS partition($hivePartitionName)") - spark.sql(s"ALTER TABLE $hiveTableName ADD partition($hivePartitionName) location '${compactPath}/$partitionStr'") + spark.sql(s"ALTER TABLE $hiveTableName ADD partition($hivePartitionName) location '${compactionPath}/$partitionStr'") } else { spark.sql(s"ALTER TABLE $hiveTableName DROP IF EXISTS partition($conditionString)") - spark.sql(s"ALTER TABLE $hiveTableName ADD partition($conditionString) location '${compactPath}/$partitionStr'") + spark.sql(s"ALTER TABLE $hiveTableName ADD partition($conditionString) location '${compactionPath}/$partitionStr'") } }) { @@ -284,7 +290,7 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, } } logInfo("=========== Compaction Success!!! ===========") - compactPath + compactionPath } else { "" } diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/sources/LakeSoulSQLConf.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/sources/LakeSoulSQLConf.scala index caca2077f..9bdc4893e 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/sources/LakeSoulSQLConf.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/sources/LakeSoulSQLConf.scala @@ -158,4 +158,13 @@ object LakeSoulSQLConf { """.stripMargin) .intConf .createWithDefault(10000) + + val RENAME_COMPACTED_FILE: ConfigEntry[Boolean] = + buildConf("lakesoul.compact.rename") + .doc( + """ + |If NATIVE_IO_ENABLE=true, timeout for each iterate will be set to NATIVE_IO_READER_AWAIT_TIMEOUT mills + """.stripMargin) + .booleanConf + .createWithDefault(false) } From 1b0055837abdad95f8fe10e3268258f3640c0af3 Mon Sep 17 00:00:00 2001 From: zenghua Date: Sat, 26 Oct 2024 16:29:58 +0800 Subject: [PATCH 31/33] fix ci Signed-off-by: zenghua --- .../datasources/v2/merge/MergeParquetScan.scala | 2 +- .../NativeMergeParquetPartitionReaderFactory.scala | 7 ++++--- .../sql/lakesoul/catalog/LakeSoulScanBuilder.scala | 13 ++++++++----- .../sql/lakesoul/catalog/LakeSoulTableV2.scala | 10 +--------- 4 files changed, 14 insertions(+), 18 deletions(-) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/MergeParquetScan.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/MergeParquetScan.scala index a696c6644..534d08ad1 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/MergeParquetScan.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/MergeParquetScan.scala @@ -171,7 +171,7 @@ abstract class MergeDeltaParquetScan(sparkSession: SparkSession, val nativeIOEnable = sparkSession.sessionState.conf.getConf(LakeSoulSQLConf.NATIVE_IO_ENABLE) if (nativeIOEnable) { NativeMergeParquetPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, - dataSchema, readDataSchema, readPartitionSchema, pushedFilters, mergeOperatorInfo, defaultMergeOp) + dataSchema, readDataSchema, readPartitionSchema, pushedFilters, mergeOperatorInfo, defaultMergeOp, options.asScala.toMap) } else { MergeParquetPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, dataSchema, readDataSchema, readPartitionSchema, newFilters, mergeOperatorInfo, defaultMergeOp) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/parquet/Native/NativeMergeParquetPartitionReaderFactory.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/parquet/Native/NativeMergeParquetPartitionReaderFactory.scala index a01072054..7da15575e 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/parquet/Native/NativeMergeParquetPartitionReaderFactory.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/parquet/Native/NativeMergeParquetPartitionReaderFactory.scala @@ -52,7 +52,8 @@ case class NativeMergeParquetPartitionReaderFactory(sqlConf: SQLConf, partitionSchema: StructType, filters: Array[Filter], mergeOperatorInfo: Map[String, MergeOperator[Any]], - defaultMergeOp: MergeOperator[Any]) + defaultMergeOp: MergeOperator[Any], + options: Map[String, String] = Map.empty) extends NativeMergeFilePartitionReaderFactory(mergeOperatorInfo, defaultMergeOp) with Logging { private val isCaseSensitive = sqlConf.caseSensitiveAnalysis @@ -70,8 +71,8 @@ case class NativeMergeParquetPartitionReaderFactory(sqlConf: SQLConf, private val nativeIOPrefecherBufferSize = sqlConf.getConf(NATIVE_IO_PREFETCHER_BUFFER_SIZE) private val nativeIOThreadNum = sqlConf.getConf(NATIVE_IO_THREAD_NUM) private val nativeIOAwaitTimeout = sqlConf.getConf(NATIVE_IO_READER_AWAIT_TIMEOUT) - private val nativeIOCdcColumn = sqlConf.getConf(NATIVE_IO_CDC_COLUMN) - private val nativeIOIsCompacted = sqlConf.getConf(NATIVE_IO_IS_COMPACTED) + private val nativeIOCdcColumn = options.getOrElse(NATIVE_IO_CDC_COLUMN.key, "") + private val nativeIOIsCompacted = options.getOrElse(NATIVE_IO_IS_COMPACTED.key, "false") // schemea: path->schema source: path->file|path->file|path->file private val requestSchemaMap: mutable.Map[String, String] = broadcastedConf.value.value diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulScanBuilder.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulScanBuilder.scala index dade3ce9a..eb47fa81d 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulScanBuilder.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulScanBuilder.scala @@ -114,25 +114,28 @@ case class LakeSoulScanBuilder(sparkSession: SparkSession, } else { hasNoDeltaFile = fileInfo.forall(f => f._2.size <= 1) } + val writableOptions = mutable.Map.empty[String, String] ++ options.asScala if (fileIndex.snapshotManagement.snapshot.getPartitionInfoArray.forall(p => p.commit_op.equals("CompactionCommit"))) { - sparkSession.sessionState.conf.setConfString(NATIVE_IO_IS_COMPACTED.key, "true") + println(s"set NATIVE_IO_IS_COMPACTED with ${fileIndex.snapshotManagement.snapshot.getPartitionInfoArray.mkString("Array(", ", ", ")")}") + writableOptions.put(NATIVE_IO_IS_COMPACTED.key, "true") } + val updatedOptions = new CaseInsensitiveStringMap(writableOptions.asJava) if (fileInfo.isEmpty) { EmptyParquetScan(sparkSession, hadoopConf, fileIndex, dataSchema, readDataSchema(), - readPartitionSchema(), pushedParquetFilters, options, partitionFilters, dataFilters) + readPartitionSchema(), pushedParquetFilters, updatedOptions, partitionFilters, dataFilters) } else if (tableInfo.hash_partition_columns.isEmpty) { parquetScan() } else if (onlyOnePartition) { OnePartitionMergeBucketScan(sparkSession, hadoopConf, fileIndex, dataSchema, mergeReadDataSchema(), - readPartitionSchema(), pushedParquetFilters, options, tableInfo, partitionFilters, dataFilters) + readPartitionSchema(), pushedParquetFilters, updatedOptions, tableInfo, partitionFilters, dataFilters) } else { if (sparkSession.sessionState.conf .getConf(LakeSoulSQLConf.BUCKET_SCAN_MULTI_PARTITION_ENABLE)) { MultiPartitionMergeBucketScan(sparkSession, hadoopConf, fileIndex, dataSchema, mergeReadDataSchema(), - readPartitionSchema(), pushedParquetFilters, options, tableInfo, partitionFilters, dataFilters) + readPartitionSchema(), pushedParquetFilters, updatedOptions, tableInfo, partitionFilters, dataFilters) } else { MultiPartitionMergeScan(sparkSession, hadoopConf, fileIndex, dataSchema, mergeReadDataSchema(), - readPartitionSchema(), pushedParquetFilters, options, tableInfo, partitionFilters, dataFilters) + readPartitionSchema(), pushedParquetFilters, updatedOptions, tableInfo, partitionFilters, dataFilters) } } } diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulTableV2.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulTableV2.scala index cfeac35d7..d35dda5d7 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulTableV2.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulTableV2.scala @@ -18,7 +18,6 @@ import org.apache.spark.sql.connector.write._ import org.apache.spark.sql.lakesoul._ import org.apache.spark.sql.lakesoul.commands.WriteIntoTable import org.apache.spark.sql.lakesoul.exception.LakeSoulErrors -import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf.NATIVE_IO_CDC_COLUMN import org.apache.spark.sql.lakesoul.sources.{LakeSoulDataSource, LakeSoulSQLConf, LakeSoulSourceUtils} import org.apache.spark.sql.lakesoul.utils.SparkUtil import org.apache.spark.sql.sources.{BaseRelation, Filter, InsertableRelation} @@ -65,14 +64,7 @@ case class LakeSoulTableV2(spark: SparkSession, // The loading of the SnapshotManagement is lazy in order to reduce the amount of FileSystem calls, // in cases where we will fallback to the V1 behavior. - lazy val snapshotManagement: SnapshotManagement = { - val mgr = SnapshotManagement(rootPath, namespace) - val cdcColumn = mgr.snapshot.getTableInfo.configuration.get(LakeSoulTableProperties.lakeSoulCDCChangePropKey) - if (cdcColumn.isDefined) { - spark.sessionState.conf.setConfString(NATIVE_IO_CDC_COLUMN.key, cdcColumn.get) - } - mgr - } + lazy val snapshotManagement: SnapshotManagement = SnapshotManagement(rootPath, namespace) override def name(): String = catalogTable.map(_.identifier.unquotedString) .orElse(tableIdentifier) From 8556f1a6e9b23022f0b9a1d71db825b7f19850bd Mon Sep 17 00:00:00 2001 From: zenghua Date: Thu, 31 Oct 2024 11:39:19 +0800 Subject: [PATCH 32/33] compaction with file size condition in parallel Signed-off-by: zenghua --- .../datasources/LakeSoulFileWriter.scala | 75 ++++++++++----- .../v2/merge/MergeParquetScan.scala | 30 +++++- .../v2/parquet/NativeParquetFileFormat.scala | 3 +- .../parquet/NativeParquetOutputWriter.scala | 15 ++- .../sql/lakesoul/DelayedCommitProtocol.scala | 39 +++++--- .../lakesoul/DelayedCopyCommitProtocol.scala | 36 ++----- .../sql/lakesoul/TransactionalWrite.scala | 13 +-- .../catalog/LakeSoulScanBuilder.scala | 1 - .../lakesoul/commands/CompactionCommand.scala | 96 +++++++------------ .../lakesoul/sources/LakeSoulSQLConf.scala | 19 ++++ .../spark/sql/vectorized/NativeIOUtils.scala | 18 +++- .../lakesoul/commands/CompactionSuite.scala | 71 ++++++++------ .../lakesoul/lakesoul/io/NativeIOWriter.java | 9 ++ rust/lakesoul-io/src/lakesoul_io_config.rs | 4 + rust/lakesoul-io/src/lakesoul_writer.rs | 4 + 15 files changed, 258 insertions(+), 175 deletions(-) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala index 880a99649..da78e4b5b 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/LakeSoulFileWriter.scala @@ -34,14 +34,23 @@ import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf import org.apache.spark.sql.vectorized.ArrowFakeRowAdaptor import org.apache.spark.util.{SerializableConfiguration, Utils} import com.dmetasoul.lakesoul.meta.DBConfig.{LAKESOUL_NON_PARTITION_TABLE_PART_DESC, LAKESOUL_RANGE_PARTITION_SPLITTER} +import com.dmetasoul.lakesoul.meta.DBUtil import com.dmetasoul.lakesoul.spark.clean.CleanOldCompaction.splitCompactFilePath -import org.apache.spark.sql.lakesoul.DelayedCopyCommitProtocol +import org.apache.spark.sql.execution.datasources.v2.parquet.{NativeParquetCompactionColumnarOutputWriter, NativeParquetOutputWriter} +import org.apache.spark.sql.lakesoul.{DelayedCommitProtocol, DelayedCopyCommitProtocol} import org.apache.spark.sql.types.{DataTypes, StringType, StructField, StructType} import java.util.{Date, UUID} +import scala.collection.JavaConverters.mapAsScalaMapConverter +import scala.collection.convert.ImplicitConversions.`iterable AsScalaIterable` /** A helper object for writing FileFormat data out to a location. */ object LakeSoulFileWriter extends Logging { + val MAX_FILE_SIZE_KEY = "max_file_size" + val HASH_BUCKET_ID_KEY = "hash_bucket_id" + val SNAPPY_COMPRESS_RATIO = 3 + val COPY_FILE_WRITER_KEY = "copy_file_writer" + /** * Basic work flow of this command is: * 1. Driver side setup, including output committer initialization and data source specific @@ -178,7 +187,11 @@ object LakeSoulFileWriter extends Logging { val nativeIOEnable = sparkSession.sessionState.conf.getConf(LakeSoulSQLConf.NATIVE_IO_ENABLE) def nativeWrap(plan: SparkPlan): RDD[InternalRow] = { - if (isCompaction && !isCDC && !isBucketNumChanged && nativeIOEnable) { + if (isCompaction + && staticBucketId != -1 + && !isCDC + && !isBucketNumChanged + && nativeIOEnable) { plan match { case withPartitionAndOrdering(_, _, child) => return nativeWrap(child) @@ -203,8 +216,8 @@ object LakeSoulFileWriter extends Logging { try { // for compaction, we won't break ordering from batch scan val (rdd, concurrentOutputWriterSpec) = - if (isCompaction && options.getOrElse("copyCompactedFile", "").nonEmpty) { - val data = Seq(InternalRow(options("copyCompactedFile"))) + if (isCompaction && options.getOrElse(COPY_FILE_WRITER_KEY, "false").toBoolean) { + val data = Seq(InternalRow(COPY_FILE_WRITER_KEY)) (sparkSession.sparkContext.parallelize(data), None) } else if (!isBucketNumChanged && (orderingMatched || isCompaction)) { (nativeWrap(empty2NullPlan), None) @@ -410,6 +423,7 @@ object LakeSoulFileWriter extends Logging { private var recordsInFile: Long = _ private val partValue: Option[String] = options.get("partValue").filter(_ != LAKESOUL_NON_PARTITION_TABLE_PART_DESC) .map(_.replace(LAKESOUL_RANGE_PARTITION_SPLITTER, "/")) + private val maxFileSize = options.get(MAX_FILE_SIZE_KEY) /** Given an input row, returns the corresponding `bucketId` */ protected lazy val getBucketId: InternalRow => Int = { @@ -419,26 +433,56 @@ object LakeSoulFileWriter extends Logging { row => proj(row).getInt(0) } + override protected def releaseCurrentWriter(): Unit = { + if (currentWriter != null) { + try { + currentWriter.close() + if (maxFileSize.isDefined) { + currentWriter.asInstanceOf[NativeParquetOutputWriter].flushResult.foreach(result => { + val (partitionDesc, flushResult) = result + val partitionDescList = if (partitionDesc == "-4") { + DBUtil.parsePartitionDesc(options.getOrElse("partValue", LAKESOUL_NON_PARTITION_TABLE_PART_DESC)).asScala.toList + } else { + DBUtil.parsePartitionDesc(partitionDesc).asScala.toList + } + committer.asInstanceOf[DelayedCommitProtocol].addOutputFile(partitionDescList, flushResult.map(_.getFilePath).toList) + }) + } + statsTrackers.foreach(_.closeFile(currentWriter.path())) + } finally { + currentWriter = null + } + } + } + private def newOutputWriter(record: InternalRow): Unit = { recordsInFile = 0 releaseResources() val ext = description.outputWriterFactory.getFileExtension(taskAttemptContext) val suffix = if (bucketSpec.isDefined) { - val bucketIdStr = if (partitionId == -1) { - BucketingUtils.bucketIdToString(getBucketId(record)) + val bucketId = if (partitionId == -1) { + getBucketId(record) } else { - BucketingUtils.bucketIdToString(partitionId) + partitionId } + taskAttemptContext.getConfiguration.set(HASH_BUCKET_ID_KEY, bucketId.toString) + + val bucketIdStr = BucketingUtils.bucketIdToString(bucketId) f"$bucketIdStr.c$fileCounter%03d" + ext } else { f"-c$fileCounter%03d" + ext } + if (maxFileSize.isDefined) { + taskAttemptContext.getConfiguration.set(MAX_FILE_SIZE_KEY, maxFileSize.get) + } + val currentPath = committer.newTaskTempFile( taskAttemptContext, partValue, - suffix) + if (maxFileSize.isDefined) "" else suffix + ) currentWriter = description.outputWriterFactory.newInstance( path = currentPath, @@ -473,21 +517,8 @@ object LakeSoulFileWriter extends Logging { customMetrics: Map[String, SQLMetric] = Map.empty) extends FileFormatDataWriter(description, taskAttemptContext, committer, customMetrics) { - private val partValue: Option[String] = options.get("partValue").filter(_ != LAKESOUL_NON_PARTITION_TABLE_PART_DESC) - .map(_.replace(LAKESOUL_RANGE_PARTITION_SPLITTER, "/")) - - /** Given an input row, returns the corresponding `bucketId` */ - protected lazy val getSrcPath: InternalRow => String = { - row => row.get(0, StringType).asInstanceOf[String] - } - override def write(record: InternalRow): Unit = { - val dstPath = committer.newTaskTempFile( - taskAttemptContext, - partValue, - getSrcPath(record)) - - statsTrackers.foreach(_.newFile(dstPath)) + logInfo("copy file") } } diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/MergeParquetScan.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/MergeParquetScan.scala index 534d08ad1..3c3e0cc9e 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/MergeParquetScan.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/merge/MergeParquetScan.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.sources.{EqualTo, Filter, Not} import org.apache.spark.sql.lakesoul._ import org.apache.spark.sql.lakesoul.exception.LakeSoulErrors import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf +import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf.{COMPACTION_TASK, SCAN_FILE_NUMBER_LIMIT} import org.apache.spark.sql.lakesoul.utils.{SparkUtil, TableInfo, TimestampFormatter} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -403,15 +404,34 @@ case class OnePartitionMergeBucketScan(sparkSession: SparkSession, val fileWithBucketId = groupByPartition.head._2 .groupBy(_.fileBucketId).map(f => (f._1, f._2.toArray)) + val fileNumLimit = options.getOrDefault(SCAN_FILE_NUMBER_LIMIT.key, Int.MaxValue.toString).toInt + val isCompactionTask = options.getOrDefault(COMPACTION_TASK.key, COMPACTION_TASK.defaultValueString).toBoolean + Seq.tabulate(bucketNum) { bucketId => var files = fileWithBucketId.getOrElse(bucketId, Array.empty) - val isSingleFile = files.length == 1 + var groupedFiles = if (fileNumLimit < Int.MaxValue && isCompactionTask) { + val groupedFiles = new ArrayBuffer[Array[MergePartitionedFile]] + for (i <- files.indices by fileNumLimit) { + groupedFiles += files.slice(i, i + fileNumLimit) + } + groupedFiles.toArray + } else { + Array(files) + } - if (!isSingleFile) { - val versionFiles = for (version <- files.indices) yield files(version).copy(writeVersion = version + 1) - files = versionFiles.toArray + var allPartitionIsSingleFile = true + var isSingleFile = false + + for (index <- groupedFiles.indices) { + isSingleFile = groupedFiles(index).length == 1 + if (!isSingleFile) { + val versionFiles = for (elem <- groupedFiles(index).indices) yield groupedFiles(index)(elem).copy(writeVersion = elem) + groupedFiles(index) = versionFiles.toArray + allPartitionIsSingleFile = false + } } - MergeFilePartition(bucketId, Array(files), isSingleFile) + + MergeFilePartition(bucketId, groupedFiles, allPartitionIsSingleFile) } } diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetFileFormat.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetFileFormat.scala index e47044cb5..bc817a9a2 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetFileFormat.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetFileFormat.scala @@ -31,7 +31,8 @@ class NativeParquetFileFormat extends FileFormat if (options.getOrElse("isCompaction", "false").toBoolean && !options.getOrElse("isCDC", "false").toBoolean && - !options.getOrElse("isBucketNumChanged", "false").toBoolean + !options.getOrElse("isBucketNumChanged", "false").toBoolean && + options.contains("staticBucketId") ) { new OutputWriterFactory { override def newInstance( diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetOutputWriter.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetOutputWriter.scala index c4d599ef8..9cf9f9139 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetOutputWriter.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/NativeParquetOutputWriter.scala @@ -5,6 +5,7 @@ package org.apache.spark.sql.execution.datasources.v2.parquet import com.dmetasoul.lakesoul.lakesoul.io.NativeIOWriter +import com.dmetasoul.lakesoul.lakesoul.io.NativeIOWriter.FlushResult import org.apache.arrow.memory.BufferAllocator import org.apache.arrow.vector.VectorSchemaRoot import org.apache.arrow.vector.types.pojo.Schema @@ -18,19 +19,29 @@ import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.{GlutenUtils, NativeIOUtils} +import java.util +import scala.collection.JavaConverters.mapAsScalaMapConverter +import scala.collection.mutable + class NativeParquetOutputWriter(val path: String, dataSchema: StructType, timeZoneId: String, context: TaskAttemptContext) extends OutputWriter { val NATIVE_IO_WRITE_MAX_ROW_GROUP_SIZE: Int = SQLConf.get.getConf(LakeSoulSQLConf.NATIVE_IO_WRITE_MAX_ROW_GROUP_SIZE) private var recordCount = 0 + var flushResult: mutable.Map[String, util.List[FlushResult]] = mutable.Map.empty + val arrowSchema: Schema = ArrowUtils.toArrowSchema(dataSchema, timeZoneId) protected val nativeIOWriter: NativeIOWriter = new NativeIOWriter(arrowSchema) GlutenUtils.setArrowAllocator(nativeIOWriter) nativeIOWriter.setRowGroupRowNumber(NATIVE_IO_WRITE_MAX_ROW_GROUP_SIZE) - nativeIOWriter.addFile(path) + if (path.endsWith(".parquet")) { + nativeIOWriter.addFile(path) + } else { + nativeIOWriter.withPrefix(path) + } NativeIOUtils.setNativeIOOptions(nativeIOWriter, NativeIOUtils.getNativeIOOptions(context, new Path(path))) @@ -59,7 +70,7 @@ class NativeParquetOutputWriter(val path: String, dataSchema: StructType, timeZo recordWriter.finish() nativeIOWriter.write(root) - nativeIOWriter.flush() + flushResult = nativeIOWriter.flush().asScala recordWriter.reset() root.close() diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCommitProtocol.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCommitProtocol.scala index 086012cf0..0c82d0215 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCommitProtocol.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCommitProtocol.scala @@ -90,26 +90,39 @@ class DelayedCommitProtocol(jobId: String, } override def newTaskTempFile(taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = { - val filename = getFileName(taskContext, ext) val partitionValues = dir.map(parsePartitions).getOrElse(List.empty[(String, String)]) val unescapedDir = if (partitionValues.nonEmpty) { Some(partitionValues.map(partitionValue => partitionValue._1 + "=" + partitionValue._2).mkString("/")) } else { dir } - val relativePath = randomPrefixLength.map { prefixLength => - getRandomPrefix(prefixLength) // Generate a random prefix as a first choice - }.orElse { - // or else write into the partition unescaped directory if it is partitioned + if (ext.isEmpty) { unescapedDir - }.map { subDir => - new Path(subDir, filename) - }.getOrElse(new Path(filename)) // or directly write out to the output path - - val absolutePath = new Path(path, relativePath).toUri.toString - //returns the absolute path to the file - addedFiles.append((partitionValues, absolutePath)) - absolutePath + .map(new Path(path, _)) + .getOrElse(new Path(path)) + .toUri.toString + } else { + val filename = getFileName(taskContext, ext) + + val relativePath = randomPrefixLength.map { prefixLength => + getRandomPrefix(prefixLength) // Generate a random prefix as a first choice + }.orElse { + // or else write into the partition unescaped directory if it is partitioned + unescapedDir + }.map { subDir => + new Path(subDir, filename) + }.getOrElse(new Path(filename)) // or directly write out to the output path + + + val absolutePath = new Path(path, relativePath).toUri.toString + //returns the absolute path to the file + addedFiles.append((partitionValues, absolutePath)) + absolutePath + } + } + + def addOutputFile(partitionValues: List[(String, String)], files: List[String]): Unit = { + files.foreach(file => addedFiles.append((partitionValues, file))) } override def newTaskTempFileAbsPath(taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCopyCommitProtocol.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCopyCommitProtocol.scala index b46be810e..4284f3026 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCopyCommitProtocol.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/DelayedCopyCommitProtocol.scala @@ -23,31 +23,16 @@ import scala.util.Random /** * Writes out the files to `path` and returns a list of them in `addedStatuses`. */ -class DelayedCopyCommitProtocol(jobId: String, +class DelayedCopyCommitProtocol(srcFiles: Seq[DataFileInfo], + jobId: String, dstPath: String, randomPrefixLength: Option[Int]) extends DelayedCommitProtocol(jobId, dstPath, randomPrefixLength) with Serializable with Logging { - @transient private var copyFiles: ArrayBuffer[(String, String)] = _ - - override def setupJob(jobContext: JobContext): Unit = { - - } - - override def abortJob(jobContext: JobContext): Unit = { - // TODO: Best effort cleanup - } - - override def setupTask(taskContext: TaskAttemptContext): Unit = { - copyFiles = new ArrayBuffer[(String, String)] - } - - override def newTaskTempFile(taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = { - val (srcCompactDir, srcBasePath) = splitCompactFilePath(ext) - copyFiles += dir.getOrElse("-5") -> ext - new Path(dstPath, srcBasePath).toString + throw new UnsupportedOperationException( + s"$this does not support adding files with an absolute path") } override def newTaskTempFileAbsPath(taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { @@ -57,15 +42,14 @@ class DelayedCopyCommitProtocol(jobId: String, override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = { - if (copyFiles.nonEmpty) { - val fs = new Path(copyFiles.head._2).getFileSystem(taskContext.getConfiguration) - val statuses = copyFiles.map { f => - val (partitionDesc, srcPath) = f - val (srcCompactDir, srcBasePath) = splitCompactFilePath(srcPath) + if (srcFiles.nonEmpty) { + val fs = new Path(srcFiles.head.path).getFileSystem(taskContext.getConfiguration) + val statuses = srcFiles.map { srcFile => + val (srcCompactDir, srcBasePath) = splitCompactFilePath(srcFile.path) val dstFile = new Path(dstPath, srcBasePath) - FileUtil.copy(fs, new Path(srcPath), fs, new Path(dstPath, srcBasePath), false, taskContext.getConfiguration) + FileUtil.copy(fs, new Path(srcFile.path), fs, new Path(dstPath, srcBasePath), false, taskContext.getConfiguration) val status = fs.getFileStatus(dstFile) - DataFileInfo(partitionDesc, fs.makeQualified(dstFile).toString, "add", status.getLen, status.getModificationTime) + DataFileInfo(srcFile.range_partitions, fs.makeQualified(dstFile).toString, "add", status.getLen, status.getModificationTime) } new TaskCommitMessage(statuses) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala index 57248a58a..1d00a0c15 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/TransactionalWrite.scala @@ -12,6 +12,7 @@ import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.Dataset import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.datasources.LakeSoulFileWriter.COPY_FILE_WRITER_KEY import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, LakeSoulFileWriter, WriteJobStatsTracker} import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} import org.apache.spark.sql.functions.{col, when} @@ -103,7 +104,8 @@ trait TransactionalWrite { */ def writeFiles(oriData: Dataset[_], writeOptions: Option[LakeSoulOptions], - isCompaction: Boolean): (Seq[DataFileInfo], Path) = { + isCompaction: Boolean, + copyCompactedFile: Seq[DataFileInfo] = Seq.empty): (Seq[DataFileInfo], Path) = { val spark = oriData.sparkSession // LakeSoul always writes timestamp data with timezone=UTC spark.conf.set("spark.sql.session.timeZone", "UTC") @@ -160,7 +162,7 @@ trait TransactionalWrite { options.put("isBucketNumChanged", "false") } val cdcCol = snapshot.getTableInfo.configuration.get(LakeSoulTableProperties.lakeSoulCDCChangePropKey) - if (cdcCol.nonEmpty) { + if (cdcCol.nonEmpty && copyCompactedFile.isEmpty) { options.put("isCDC", "true") val cdcColName = cdcCol.get if (writeOptions.forall(_.options.getOrElse("fullCompaction", "true").equals("true"))) { @@ -214,10 +216,9 @@ trait TransactionalWrite { output.length < data.schema.size) } - val committer = if (writeOptions.exists(_.options.getOrElse("copyCompactedFile", "").nonEmpty)) { - val srcPath = writeOptions.get.options.get("copyCompactedFile") - options.put("copyCompactedFile", srcPath.get) - new DelayedCopyCommitProtocol("lakesoul", outputPath.toString, None) + val committer = if (copyCompactedFile.nonEmpty) { + options.put(COPY_FILE_WRITER_KEY, "true") + new DelayedCopyCommitProtocol(copyCompactedFile, "lakesoul", outputPath.toString, None) } else { getCommitter(outputPath) } diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulScanBuilder.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulScanBuilder.scala index eb47fa81d..830371192 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulScanBuilder.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/catalog/LakeSoulScanBuilder.scala @@ -116,7 +116,6 @@ case class LakeSoulScanBuilder(sparkSession: SparkSession, } val writableOptions = mutable.Map.empty[String, String] ++ options.asScala if (fileIndex.snapshotManagement.snapshot.getPartitionInfoArray.forall(p => p.commit_op.equals("CompactionCommit"))) { - println(s"set NATIVE_IO_IS_COMPACTED with ${fileIndex.snapshotManagement.snapshot.getPartitionInfoArray.mkString("Array(", ", ", ")")}") writableOptions.put(NATIVE_IO_IS_COMPACTED.key, "true") } val updatedOptions = new CaseInsensitiveStringMap(writableOptions.asJava) diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala index f74a35bf5..b0df3475c 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/commands/CompactionCommand.scala @@ -14,13 +14,14 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.PredicateHelper import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.execution.command.LeafRunnableCommand +import org.apache.spark.sql.execution.datasources.LakeSoulFileWriter.{MAX_FILE_SIZE_KEY, SNAPPY_COMPRESS_RATIO} import org.apache.spark.sql.execution.datasources.v2.merge.MergeDeltaParquetScan import org.apache.spark.sql.execution.datasources.v2.parquet.{NativeParquetScan, ParquetScan} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} -import org.apache.spark.sql.functions.{expr, forall} +import org.apache.spark.sql.functions.expr import org.apache.spark.sql.lakesoul.catalog.LakeSoulTableV2 import org.apache.spark.sql.lakesoul.exception.LakeSoulErrors -import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf.RENAME_COMPACTED_FILE +import org.apache.spark.sql.lakesoul.sources.LakeSoulSQLConf.{COMPACTION_TASK, SCAN_FILE_NUMBER_LIMIT} import org.apache.spark.sql.lakesoul.utils.TableInfo import org.apache.spark.sql.lakesoul.{BatchDataSoulFileIndexV2, LakeSoulOptions, SnapshotManagement, TransactionCommit} import org.apache.spark.sql.types.StructType @@ -29,9 +30,9 @@ import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.util.Utils import java.util.UUID +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConverters._ case class CompactionCommand(snapshotManagement: SnapshotManagement, conditionString: String, @@ -69,8 +70,7 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, files: Seq[DataFileInfo], readPartitionInfo: Array[PartitionInfoScala], compactionPath: String, - fullCompaction: Boolean, - copyCompactedFile: String = ""): List[DataCommitInfo] = { + copySrcFiles: Boolean = false): List[DataCommitInfo] = { if (newBucketNum.isEmpty && readPartitionInfo.forall(p => p.commit_op.equals("CompactionCommit") && p.read_files.length == 1)) { logInfo("=========== All Partitions Have Compacted, This Operation Will Cancel!!! ===========") return List.empty @@ -85,7 +85,11 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, Option(mergeOperatorInfo) ) val option = new CaseInsensitiveStringMap( - Map("basePath" -> tc.tableInfo.table_path_s.get, "isCompaction" -> "true").asJava) + Map("basePath" -> tc.tableInfo.table_path_s.get, + "isCompaction" -> "true", + SCAN_FILE_NUMBER_LIMIT.key -> fileNumLimit.getOrElse(Int.MaxValue).toString, + COMPACTION_TASK.key -> "true" + ).asJava) val partitionNames = readPartitionInfo.head.range_value.split(',').map(p => { p.split('=').head @@ -123,13 +127,19 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, val map = mutable.HashMap[String, String]() map.put("isCompaction", "true") map.put("compactionPath", compactionPath) - map.put("fullCompaction", fullCompaction.toString) - if (copyCompactedFile.nonEmpty) { - map.put("copyCompactedFile", copyCompactedFile) + + val copyCompactedFiles = if (copySrcFiles) { + files + } else { + Seq.empty } if (readPartitionInfo.nonEmpty) { map.put("partValue", readPartitionInfo.head.range_value) } + if (fileSizeLimit.isDefined) { + map.put("fullCompaction", "false") + map.put(MAX_FILE_SIZE_KEY, (fileSizeLimit.get * SNAPPY_COMPRESS_RATIO).toString) + } if (bucketNumChanged) { map.put("newBucketNum", newBucketNum.get.toString) } else if (tableInfo.hash_partition_columns.nonEmpty) { @@ -140,7 +150,7 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, } logInfo(s"write CompactData with Option=$map") - val (newFiles, path) = tc.writeFiles(compactDF, Some(new LakeSoulOptions(map.toMap, spark.sessionState.conf)), isCompaction = true) + val (newFiles, path) = tc.writeFiles(compactDF, Some(new LakeSoulOptions(map.toMap, spark.sessionState.conf)), isCompaction = true, copyCompactedFiles) tc.createDataCommitInfo(newFiles, Seq.empty, "", -1)._1 } @@ -195,63 +205,21 @@ case class CompactionCommand(snapshotManagement: SnapshotManagement, def compactSinglePartition(sparkSession: SparkSession, tc: TransactionCommit, files: Seq[DataFileInfo], sourcePartition: PartitionInfoScala): String = { logInfo(s"Compacting Single Partition=${sourcePartition} with ${files.length} files") - val bucketedFiles = if (tableInfo.hash_partition_columns.isEmpty || bucketNumChanged) { - Seq(-1 -> files) + val (copyFiles, scanFiles) = if (fileSizeLimit.isEmpty || bucketNumChanged || force) { + (Seq.empty, files) } else { - files.groupBy(_.file_bucket_id) + files.splitAt(files.indexWhere(_.size < fileSizeLimit.get * 0.5)) } - val compactionPath = newCompactPath - val allDataCommitInfo = bucketedFiles.flatMap(groupByBucketId => { - val (bucketId, files) = groupByBucketId - val groupedFiles = if (fileNumLimit.isDefined || fileSizeLimit.isDefined) { - val groupedFiles = new ArrayBuffer[Seq[DataFileInfo]] - var groupHead = 0 - var groupSize = 0L - var groupFileCount = 0 - for (i <- files.indices) { - // each group contains at least one file - if (i == groupHead) { - groupSize += files(i).size - groupFileCount += 1 - } else if (fileSizeLimit.exists(groupSize + files(i).size > _) || fileNumLimit.exists(groupFileCount + 1 > _)) { - // if the file size limit is reached, or the file count limit is reached, we need to start a new group - groupedFiles += files.slice(groupHead, i) - groupHead = i - groupSize = files(i).size - groupFileCount = 1 - } else { - // otherwise, we add the file to the current group - groupSize += files(i).size - groupFileCount += 1 - } - } - // add the last group to the groupedFiles - groupedFiles += files.slice(groupHead, files.length) - groupedFiles - } else { - Seq(files) - } - val fullCompaction = groupedFiles.size == 1 - groupedFiles.flatMap(files => { - lazy val incrementFiles = if (force || newBucketNum.isDefined) { - false - } else { - files.size == 1 && splitCompactFilePath(files.head.path)._1.nonEmpty - } - if (!incrementFiles) { - executeCompaction(sparkSession, tc, files, Array(sourcePartition), compactionPath, fullCompaction) - } else { - logInfo(s"== Partition ${sourcePartition.range_value} has no increment file.") - val origCompactedFile = files.head - if (sparkSession.sessionState.conf.getConf(RENAME_COMPACTED_FILE)) { - renameOldCompactedFile(tc, origCompactedFile, sourcePartition.range_value, compactionPath) - } else { - executeCompaction(sparkSession, tc, files, Array(sourcePartition), compactionPath, fullCompaction, origCompactedFile.path) - } - } - }) - }) + + val compactionPath = newCompactPath + val allDataCommitInfo = new ArrayBuffer[DataCommitInfo] + if (copyFiles.nonEmpty) { + allDataCommitInfo ++= executeCompaction(sparkSession, tc, copyFiles, Array(sourcePartition), compactionPath, true) + } + if (scanFiles.nonEmpty) { + allDataCommitInfo ++= executeCompaction(sparkSession, tc, scanFiles, Array(sourcePartition), compactionPath) + } if (allDataCommitInfo.nonEmpty) { val compactDataCommitInfoId = UUID.randomUUID diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/sources/LakeSoulSQLConf.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/sources/LakeSoulSQLConf.scala index 9bdc4893e..49e50b431 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/sources/LakeSoulSQLConf.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/lakesoul/sources/LakeSoulSQLConf.scala @@ -167,4 +167,23 @@ object LakeSoulSQLConf { """.stripMargin) .booleanConf .createWithDefault(false) + + val SCAN_FILE_NUMBER_LIMIT: ConfigEntry[Int] = + buildConf("scan.file.number.limit") + .doc( + """ + |If SCAN_FILE_NUMBER_LIMIT < Int.MaxValue, Scan will scan file with number less than SCAN_FILE_NUMBER_LIMIT per file group + """.stripMargin) + .intConf + .createWithDefault(Int.MaxValue) + + + val COMPACTION_TASK: ConfigEntry[Boolean] = + buildConf("scan.file.size.limit") + .doc( + """ + |If SCAN_FILE_NUMBER_LIMIT < Int.MaxValue, Scan will scan file with number less than SCAN_FILE_NUMBER_LIMIT per file group + """.stripMargin) + .booleanConf + .createWithDefault(false) } diff --git a/lakesoul-spark/src/main/scala/org/apache/spark/sql/vectorized/NativeIOUtils.scala b/lakesoul-spark/src/main/scala/org/apache/spark/sql/vectorized/NativeIOUtils.scala index 1f550ef7f..c01e7a61f 100644 --- a/lakesoul-spark/src/main/scala/org/apache/spark/sql/vectorized/NativeIOUtils.scala +++ b/lakesoul-spark/src/main/scala/org/apache/spark/sql/vectorized/NativeIOUtils.scala @@ -15,6 +15,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem import org.apache.hadoop.mapreduce.TaskAttemptContext import org.apache.parquet.hadoop.ParquetInputFormat import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.LakeSoulFileWriter.{HASH_BUCKET_ID_KEY, MAX_FILE_SIZE_KEY} import org.apache.spark.sql.execution.datasources.parquet.{ParquetReadSupport, ParquetWriteSupport} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType @@ -32,10 +33,11 @@ class NativeIOOptions(val s3Bucket: String, val s3Region: String, val fsUser: String, val defaultFS: String, - val virtual_path_style: Boolean + val virtual_path_style: Boolean, + val others: Map[String, String] = Map.empty ) -object NativeIOUtils{ +object NativeIOUtils { def asArrayColumnVector(vectorSchemaRoot: VectorSchemaRoot): Array[ColumnVector] = { asScalaIteratorConverter(vectorSchemaRoot.getFieldVectors.iterator()) @@ -62,6 +64,13 @@ object NativeIOUtils{ var defaultFS = taskAttemptContext.getConfiguration.get("fs.defaultFS") if (defaultFS == null) defaultFS = taskAttemptContext.getConfiguration.get("fs.default.name") val fileSystem = file.getFileSystem(taskAttemptContext.getConfiguration) + var otherOptions = Map[String, String]() + if (taskAttemptContext.getConfiguration.get(HASH_BUCKET_ID_KEY, "").nonEmpty) { + otherOptions += HASH_BUCKET_ID_KEY -> taskAttemptContext.getConfiguration.get(HASH_BUCKET_ID_KEY) + } + if (taskAttemptContext.getConfiguration.get(MAX_FILE_SIZE_KEY, "").nonEmpty) { + otherOptions += MAX_FILE_SIZE_KEY -> taskAttemptContext.getConfiguration.get(MAX_FILE_SIZE_KEY) + } if (hasS3AFileSystemClass) { fileSystem match { case s3aFileSystem: S3AFileSystem => @@ -71,11 +80,11 @@ object NativeIOUtils{ val s3aAccessKey = taskAttemptContext.getConfiguration.get("fs.s3a.access.key") val s3aSecretKey = taskAttemptContext.getConfiguration.get("fs.s3a.secret.key") val virtualPathStyle = taskAttemptContext.getConfiguration.getBoolean("fs.s3a.path.style.access", false) - return new NativeIOOptions(awsS3Bucket, s3aAccessKey, s3aSecretKey, s3aEndpoint, s3aRegion, user, defaultFS, virtualPathStyle) + return new NativeIOOptions(awsS3Bucket, s3aAccessKey, s3aSecretKey, s3aEndpoint, s3aRegion, user, defaultFS, virtualPathStyle, otherOptions) case _ => } } - new NativeIOOptions(null, null, null, null, null, user, defaultFS, false) + new NativeIOOptions(null, null, null, null, null, user, defaultFS, false, otherOptions) } def setNativeIOOptions(nativeIO: NativeIOBase, options: NativeIOOptions): Unit = { @@ -89,6 +98,7 @@ object NativeIOUtils{ options.defaultFS, options.virtual_path_style ) + options.others.foreach(options => nativeIO.setOption(options._1, options._2)) } def setParquetConfigurations(sparkSession: SparkSession, hadoopConf: Configuration, readDataSchema: StructType): Unit = { diff --git a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala index 7d9639ae2..87275f6cf 100644 --- a/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala +++ b/lakesoul-spark/src/test/scala/org/apache/spark/sql/lakesoul/commands/CompactionSuite.scala @@ -548,29 +548,30 @@ class CompactionSuite extends QueryTest // Get initial PartitionInfo count val initialFileCount = getFileList(tablePath).length - println(s"before compact initialPartitionInfoCount=$initialFileCount") + println(s"before ${c}th time compact file count=$initialFileCount") lakeSoulTable.toDF.show // Perform limited compaction (group every compactGroupSize PartitionInfo) lakeSoulTable.compaction(fileNumLimit = Some(compactGroupSize)) // Get PartitionInfo count after compaction - val compactedFileCount = getFileList(tablePath).length + val compactedFileList = getFileList(tablePath) + val compactedFileCount = compactedFileList.length - println(s"after compact compactedPartitionInfoCount=$compactedFileCount") + println(s"after ${c}th time compact file count=$compactedFileCount") lakeSoulTable.toDF.show // Verify results - assert(compactedFileCount < initialFileCount, - s"Compaction should reduce the number of files, but it changed from ${initialFileCount} to $compactedFileCount") - + assert(compactedFileCount <= hashBucketNum, + s"Compaction should have hashBucketNum files, but it has $compactedFileCount") - assert(compactedFileCount >= (initialFileCount - 1) / compactGroupSize + 1, - s"Compaction should produce files above a lower bound, but there are ${compactedFileCount} files") - assert(compactedFileCount <= (initialFileCount - 1) / compactGroupSize + 1 + hashBucketNum, - s"Compaction should produce files below a upper bound, but there are ${compactedFileCount} files") + // assert(compactedFileCount >= (initialFileCount - 1) / compactGroupSize + 1, + // s"Compaction should produce files above a lower bound, but there are ${compactedFileCount} files") + // + // assert(compactedFileCount <= (initialFileCount - 1) / compactGroupSize + 1 + hashBucketNum, + // s"Compaction should produce files below a upper bound, but there are ${compactedFileCount} files") } // Verify data integrity @@ -649,15 +650,15 @@ class CompactionSuite extends QueryTest lakeSoulTable.toDF.show // Verify results - assert(compactedFileCount < initialFileCount, - s"Compaction should reduce the number of files, but it changed from ${initialFileCount} to $compactedFileCount") + assert(compactedFileCount <= hashBucketNum, + s"Compaction should have hashBucketNum files, but it has $compactedFileCount") - assert(compactedFileCount >= (initialFileCount - 1) / compactGroupSize + 1, - s"Compaction should produce files above a lower bound, but there are ${compactedFileCount} files") - - assert(compactedFileCount <= (initialFileCount - 1) / compactGroupSize + 1 + hashBucketNum, - s"Compaction should produce files below a upper bound, but there are ${compactedFileCount} files") + // assert(compactedFileCount >= (initialFileCount - 1) / compactGroupSize + 1, + // s"Compaction should produce files above a lower bound, but there are ${compactedFileCount} files") + // + // assert(compactedFileCount <= (initialFileCount - 1) / compactGroupSize + 1 + hashBucketNum, + // s"Compaction should produce files below a upper bound, but there are ${compactedFileCount} files") } // Verify data integrity @@ -714,23 +715,26 @@ class CompactionSuite extends QueryTest // Get initial PartitionInfo count val initialMaxFileSize = getFileList(tablePath).map(_.size).max - println(s"before compact initialMaxFileSize=$initialMaxFileSize") - - // Perform limited compaction (group every compactGroupSize PartitionInfo) + println(s"before ${c}th compact initialMaxFileSize=$initialMaxFileSize") LakeSoulTable.uncached(tablePath) - lakeSoulTable.compaction(fileSizeLimit = Some(compactFileSize), force = false) + spark.time({ + // Perform limited compaction (group every compactGroupSize PartitionInfo) + lakeSoulTable.compaction(fileSizeLimit = Some(compactFileSize), force = false) + // lakeSoulTable.compaction(fileSizeLimit = Some(compactFileSize), force = true) + // lakeSoulTable.compaction() + }) // Get PartitionInfo count after compaction val compactedFiles = getFileList(tablePath) val compactedFileMax = compactedFiles.map(_.size).max - println(s"after compact compactedFileMax=$compactedFileMax") + println(s"after ${c}th compact compactedFileMax=$compactedFileMax") // Verify results - assert(compactedFileMax >= initialMaxFileSize, - s"Compaction should reduce the number of files, but it changed from ${initialMaxFileSize} to $compactedFileMax") + // assert(compactedFileMax >= initialMaxFileSize, + // s"Compaction should increase the max size of files, but it changed from ${initialMaxFileSize} to $compactedFileMax") - assert(compactedFileMax <= DBUtil.parseMemoryExpression(compactFileSize) * 1.2, + assert(compactedFileMax <= DBUtil.parseMemoryExpression(compactFileSize) * 1.1, s"Compaction should produce file with upper-bounded size, but there is a larger ${compactedFileMax} file size") val (compactDir, _) = splitCompactFilePath(compactedFiles.head.path) @@ -756,7 +760,7 @@ class CompactionSuite extends QueryTest val hashBucketNum = 4 val compactRounds = 5 val upsertPerRounds = 10 - val rowsPerUpsert = 1002 + val rowsPerUpsert = 1000 val compactFileSize = "10KB" // Create test data @@ -800,23 +804,26 @@ class CompactionSuite extends QueryTest // Get initial PartitionInfo count val initialMaxFileSize = getFileList(tablePath).map(_.size).max - println(s"before compact initialMaxFileSize=$initialMaxFileSize") + println(s"before ${c}th compact initialMaxFileSize=$initialMaxFileSize") // Perform limited compaction (group every compactGroupSize PartitionInfo) LakeSoulTable.uncached(tablePath) - lakeSoulTable.compaction(fileSizeLimit = Some(compactFileSize), force = false) + spark.time({ + lakeSoulTable.compaction(fileSizeLimit = Some(compactFileSize), force = false) + // lakeSoulTable.compaction(fileSizeLimit = Some(compactFileSize), force = true) + }) // Get PartitionInfo count after compaction val compactedFiles = getFileList(tablePath) val compactedFileMax = compactedFiles.map(_.size).max - println(s"after compact compactedFileMax=$compactedFileMax") + println(s"after ${c}th compact compactedFileMax=$compactedFileMax") // Verify results // assert(compactedFileMax >= initialMaxFileSize, // s"Compaction should reduce the number of files, but it changed from ${initialMaxFileSize} to $compactedFileMax") - assert(compactedFileMax <= DBUtil.parseMemoryExpression(compactFileSize) * 1.2, + assert(compactedFileMax <= DBUtil.parseMemoryExpression(compactFileSize) * 1.1, s"Compaction should produce file with upper-bounded size, but there is a larger ${compactedFileMax} file size") val (compactDir, _) = splitCompactFilePath(compactedFiles.head.path) @@ -826,7 +833,9 @@ class CompactionSuite extends QueryTest // Verify data integrity LakeSoulTable.uncached(tablePath) - val compactedData = lakeSoulTable.toDF.orderBy("id", "date").collect() + val finalData = lakeSoulTable.toDF.orderBy("id", "date") + // println(finalData.queryExecution) + val compactedData = finalData.collect() // println(compactedData.mkString("Array(", ", ", ")")) assert(compactedData.length == 6 + rowsPerUpsert * upsertPerRounds * compactRounds / 2, s"The compressed data should have ${6 + rowsPerUpsert * upsertPerRounds * compactRounds / 2} rows, but it actually has ${compactedData.length} rows") diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java index 5af7f9023..7a30b369a 100644 --- a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java @@ -166,6 +166,15 @@ public String getFilePath() { public String getFileExistCols() { return fileExistCols; } + + @Override + public String toString() { + return "FlushResult{" + + "filePath='" + filePath + '\'' + + ", fileSize=" + fileSize + + ", fileExistCols='" + fileExistCols + '\'' + + '}'; + } } public static FlushResult decodeFlushResult(String encoded) { diff --git a/rust/lakesoul-io/src/lakesoul_io_config.rs b/rust/lakesoul-io/src/lakesoul_io_config.rs index 8a87d71ed..948684ad2 100644 --- a/rust/lakesoul-io/src/lakesoul_io_config.rs +++ b/rust/lakesoul-io/src/lakesoul_io_config.rs @@ -172,6 +172,10 @@ impl LakeSoulIOConfig { self.option(OPTION_KEY_MEM_LIMIT).map(|x| x.parse().unwrap()) } + pub fn max_file_size_option(&self) -> Option { + self.option(OPTION_KEY_MAX_FILE_SIZE).map(|x| x.parse().unwrap()) + } + pub fn pool_size(&self) -> Option { self.option(OPTION_KEY_POOL_SIZE).map(|x| x.parse().unwrap()) } diff --git a/rust/lakesoul-io/src/lakesoul_writer.rs b/rust/lakesoul-io/src/lakesoul_writer.rs index 370a461d4..3d034c59b 100644 --- a/rust/lakesoul-io/src/lakesoul_writer.rs +++ b/rust/lakesoul-io/src/lakesoul_writer.rs @@ -41,6 +41,10 @@ impl SyncSendableMutableLakeSoulWriter { let writer = Self::create_writer(writer_config).await?; let schema = writer.schema(); + if let Some(max_file_size) = config.max_file_size_option() { + config.max_file_size = Some(max_file_size); + } + if let Some(mem_limit) = config.mem_limit() { if config.use_dynamic_partition { config.max_file_size = Some((mem_limit as f64 * 0.15) as u64); From 0371d6def189d4ad09995200f5680a420b6d69d8 Mon Sep 17 00:00:00 2001 From: ChenYunHey <1908166778@qq.com> Date: Mon, 4 Nov 2024 11:35:16 +0800 Subject: [PATCH 33/33] .. Signed-off-by: ChenYunHey <1908166778@qq.com> --- .../org/apache/flink/lakesoul/entry/sql/flink/ExecuteSql.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/ExecuteSql.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/ExecuteSql.java index 53b6f3bf0..b11ebefb9 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/ExecuteSql.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/ExecuteSql.java @@ -92,10 +92,11 @@ public static void executeSqlFileContent(String script, StreamTableEnvironment t // try get k8s cluster name String k8sClusterID = conf.getString("kubernetes.cluster-id", ""); - env.execute(k8sClusterID.isEmpty() ? null : k8sClusterID + "-job"); + env.execute(k8sClusterID.isEmpty() ? null : k8sClusterID); } else { System.out.println("There's no INSERT INTO statement, the program will terminate"); } + } public static List parseStatements(String script) {