From b0fe432e9988693c7fa572782c815a04bbefdc05 Mon Sep 17 00:00:00 2001 From: hawk9821 <39961809+hawk9821@users.noreply.github.com> Date: Mon, 22 Jul 2024 16:12:12 +0800 Subject: [PATCH 001/361] The isNullable attribute is true when the primary key field in the Paimon table converts the Column object. #7231 (#7242) --- .../connectors/seatunnel/paimon/catalog/PaimonCatalog.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java index 2c9fcd6f828..d896e015398 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java @@ -222,7 +222,8 @@ private CatalogTable toCatalogTable( BasicTypeDefine.builder() .name(dataField.name()) .comment(dataField.description()) - .nativeType(dataField.type()); + .nativeType(dataField.type()) + .nullable(dataField.type().isNullable()); Column column = SchemaUtil.toSeaTunnelType(typeDefineBuilder.build()); builder.column(column); }); From fe0c477b743e1d2b4c1dbc80b71e83b091306677 Mon Sep 17 00:00:00 2001 From: Eric Date: Tue, 23 Jul 2024 08:36:58 +0800 Subject: [PATCH 002/361] [Hotfix][Zeta] Fix taskgroup failed log lost (#7241) --- .../sink/inmemory/InMemorySinkFactory.java | 5 ++ .../e2e/sink/inmemory/InMemorySinkWriter.java | 8 ++ .../seatunnel/engine/e2e/ClusterIT.java | 73 +++++++++++++++++++ ...am_fake_to_inmemory_with_runtime_list.conf | 51 +++++++++++++ .../server/dag/physical/PhysicalVertex.java | 4 +- 5 files changed, 140 insertions(+), 1 deletion(-) create mode 100644 seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/stream_fake_to_inmemory_with_runtime_list.conf diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/sink/inmemory/InMemorySinkFactory.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/sink/inmemory/InMemorySinkFactory.java index 1c5b9fe398c..9ba1956dbe6 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/sink/inmemory/InMemorySinkFactory.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/sink/inmemory/InMemorySinkFactory.java @@ -28,6 +28,8 @@ import com.google.auto.service.AutoService; +import java.util.List; + import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkArgument; @AutoService(Factory.class) @@ -50,6 +52,9 @@ public class InMemorySinkFactory public static final Option ASSERT_OPTIONS_VALUE = Options.key("assert_options_value").stringType().noDefaultValue(); + public static final Option> THROW_RUNTIME_EXCEPTION_LIST = + Options.key("throw_runtime_exception_list").listType().noDefaultValue(); + @Override public String factoryIdentifier() { return "InMemory"; diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/sink/inmemory/InMemorySinkWriter.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/sink/inmemory/InMemorySinkWriter.java index a12b2ca5b99..81c8cf0af56 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/sink/inmemory/InMemorySinkWriter.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/sink/inmemory/InMemorySinkWriter.java @@ -39,6 +39,8 @@ public class InMemorySinkWriter // use a daemon thread to test classloader leak private static final Thread THREAD; + private static int restoreCount = -1; + static { // use the daemon thread to always hold the classloader ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); @@ -80,6 +82,12 @@ public void write(SeaTunnelRow element) throws IOException { if (config.get(InMemorySinkFactory.THROW_OUT_OF_MEMORY)) { throw new OutOfMemoryError(); } + + if (config.getOptional(InMemorySinkFactory.THROW_RUNTIME_EXCEPTION_LIST).isPresent()) { + restoreCount++; + throw new RuntimeException( + config.get(InMemorySinkFactory.THROW_RUNTIME_EXCEPTION_LIST).get(restoreCount)); + } } @Override diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterIT.java index ced1065731a..76b4f6fc820 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterIT.java @@ -18,8 +18,14 @@ package org.apache.seatunnel.engine.e2e; import org.apache.seatunnel.engine.client.SeaTunnelClient; +import org.apache.seatunnel.engine.client.job.ClientJobExecutionEnvironment; +import org.apache.seatunnel.engine.client.job.ClientJobProxy; import org.apache.seatunnel.engine.common.config.ConfigProvider; +import org.apache.seatunnel.engine.common.config.JobConfig; import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; +import org.apache.seatunnel.engine.common.utils.PassiveCompletableFuture; +import org.apache.seatunnel.engine.core.job.JobResult; +import org.apache.seatunnel.engine.core.job.JobStatus; import org.apache.seatunnel.engine.server.SeaTunnelServerStarter; import org.awaitility.Awaitility; @@ -31,6 +37,7 @@ import lombok.extern.slf4j.Slf4j; import java.util.Map; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @Slf4j @@ -89,4 +96,70 @@ public void getClusterHealthMetrics() { } } } + + @Test + public void testTaskGroupErrorMsgLost() throws Exception { + HazelcastInstanceImpl node1 = null; + SeaTunnelClient engineClient = null; + + String testClusterName = "Test_TaskGroupErrorMsgLost"; + + SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); + seaTunnelConfig + .getHazelcastConfig() + .setClusterName(TestUtils.getClusterName(testClusterName)); + seaTunnelConfig.getEngineConfig().setClassloaderCacheMode(true); + + try { + node1 = SeaTunnelServerStarter.createHazelcastInstance(seaTunnelConfig); + HazelcastInstanceImpl finalNode = node1; + Awaitility.await() + .atMost(10000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> + Assertions.assertEquals( + 1, finalNode.getCluster().getMembers().size())); + + ClientConfig clientConfig = ConfigProvider.locateAndGetClientConfig(); + clientConfig.setClusterName(TestUtils.getClusterName(testClusterName)); + engineClient = new SeaTunnelClient(clientConfig); + + String filePath = + TestUtils.getResource("stream_fake_to_inmemory_with_runtime_list.conf"); + JobConfig jobConfig = new JobConfig(); + jobConfig.setName(testClusterName); + ClientJobExecutionEnvironment jobExecutionEnv = + engineClient.createExecutionContext(filePath, jobConfig, seaTunnelConfig); + + final ClientJobProxy clientJobProxy = jobExecutionEnv.execute(); + + CompletableFuture> objectCompletableFuture = + CompletableFuture.supplyAsync(clientJobProxy::doWaitForJobComplete); + + Awaitility.await() + .atMost(120000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Thread.sleep(2000); + Assertions.assertTrue(objectCompletableFuture.isDone()); + + PassiveCompletableFuture + jobResultPassiveCompletableFuture = + objectCompletableFuture.get(); + JobResult jobResult = jobResultPassiveCompletableFuture.get(); + Assertions.assertEquals(JobStatus.FAILED, jobResult.getStatus()); + Assertions.assertTrue( + jobResult.getError().contains("runtime error 4")); + }); + + } finally { + if (engineClient != null) { + engineClient.close(); + } + + if (node1 != null) { + node1.shutdown(); + } + } + } } diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/stream_fake_to_inmemory_with_runtime_list.conf b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/stream_fake_to_inmemory_with_runtime_list.conf new file mode 100644 index 00000000000..b3a93adcc8b --- /dev/null +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/stream_fake_to_inmemory_with_runtime_list.conf @@ -0,0 +1,51 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + result_table_name = "fake" + row.num = 100 + split.num = 5 + schema = { + fields { + name = "string" + age = "int" + } + } + parallelism = 1 + } +} + +transform { +} + +sink { + InMemory { + source_table_name="fake" + throw_runtime_exception_list=["runtime error1", "runtime error 2", "runtime error 3", "runtime error 4"] + } +} \ No newline at end of file diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalVertex.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalVertex.java index 4fbcfa4fa3f..b6ec234bf25 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalVertex.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalVertex.java @@ -214,7 +214,7 @@ public PassiveCompletableFuture initStateFuture() { } } else if (ExecutionState.DEPLOYING.equals(currExecutionState)) { if (!checkTaskGroupIsExecuting(taskGroupLocation)) { - updateTaskState(ExecutionState.RUNNING); + updateTaskState(ExecutionState.FAILING); } } return new PassiveCompletableFuture<>(this.taskFuture); @@ -485,6 +485,8 @@ private void resetExecutionState() { () -> { updateStateTimestamps(ExecutionState.CREATED); runningJobStateIMap.set(taskGroupLocation, ExecutionState.CREATED); + // reset the errorByPhysicalVertex + errorByPhysicalVertex = new AtomicReference<>(); return null; }, new RetryUtils.RetryMaterial( From cc5949988b7a851c49d594389fd3b96dbe0f9508 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Tue, 23 Jul 2024 14:27:07 +0800 Subject: [PATCH 003/361] [Improve][Core] Move MultiTableSink to seatunnel-api module (#7243) * [Improve][Core] Move MultiTableSink to seatunnel-api module * [Improve][Core] Move MultiTableSink to seatunnel-api module --- .../MultiTableAggregatedCommitInfo.java | 2 +- .../sink}/multitablesink/MultiTableCommitInfo.java | 2 +- .../api/sink}/multitablesink/MultiTableSink.java | 6 ++++-- .../MultiTableSinkAggregatedCommitter.java | 2 +- .../sink}/multitablesink/MultiTableSinkCommitter.java | 2 +- .../sink}/multitablesink/MultiTableSinkFactory.java | 2 +- .../sink}/multitablesink/MultiTableSinkWriter.java | 2 +- .../api/sink}/multitablesink/MultiTableState.java | 2 +- .../multitablesink/MultiTableWriterRunnable.java | 2 +- .../api/sink}/multitablesink/SinkContextProxy.java | 2 +- .../api/sink}/multitablesink/SinkIdentifier.java | 2 +- .../seatunnel/api/table/factory/FactoryUtil.java | 3 ++- .../seatunnel/engine/server/master/JobMaster.java | 11 +++-------- 13 files changed, 19 insertions(+), 21 deletions(-) rename {seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common => seatunnel-api/src/main/java/org/apache/seatunnel/api/sink}/multitablesink/MultiTableAggregatedCommitInfo.java (93%) rename {seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common => seatunnel-api/src/main/java/org/apache/seatunnel/api/sink}/multitablesink/MultiTableCommitInfo.java (93%) rename {seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common => seatunnel-api/src/main/java/org/apache/seatunnel/api/sink}/multitablesink/MultiTableSink.java (97%) rename {seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common => seatunnel-api/src/main/java/org/apache/seatunnel/api/sink}/multitablesink/MultiTableSinkAggregatedCommitter.java (99%) rename {seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common => seatunnel-api/src/main/java/org/apache/seatunnel/api/sink}/multitablesink/MultiTableSinkCommitter.java (98%) rename {seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common => seatunnel-api/src/main/java/org/apache/seatunnel/api/sink}/multitablesink/MultiTableSinkFactory.java (96%) rename {seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common => seatunnel-api/src/main/java/org/apache/seatunnel/api/sink}/multitablesink/MultiTableSinkWriter.java (99%) rename {seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common => seatunnel-api/src/main/java/org/apache/seatunnel/api/sink}/multitablesink/MultiTableState.java (93%) rename {seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common => seatunnel-api/src/main/java/org/apache/seatunnel/api/sink}/multitablesink/MultiTableWriterRunnable.java (97%) rename {seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common => seatunnel-api/src/main/java/org/apache/seatunnel/api/sink}/multitablesink/SinkContextProxy.java (95%) rename {seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common => seatunnel-api/src/main/java/org/apache/seatunnel/api/sink}/multitablesink/SinkIdentifier.java (94%) diff --git a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableAggregatedCommitInfo.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableAggregatedCommitInfo.java similarity index 93% rename from seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableAggregatedCommitInfo.java rename to seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableAggregatedCommitInfo.java index 5d378140e94..585a8f4e068 100644 --- a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableAggregatedCommitInfo.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableAggregatedCommitInfo.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.common.multitablesink; +package org.apache.seatunnel.api.sink.multitablesink; import lombok.AllArgsConstructor; import lombok.Getter; diff --git a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableCommitInfo.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableCommitInfo.java similarity index 93% rename from seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableCommitInfo.java rename to seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableCommitInfo.java index 21faf0c7edc..8b12fa07c5b 100644 --- a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableCommitInfo.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableCommitInfo.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.common.multitablesink; +package org.apache.seatunnel.api.sink.multitablesink; import lombok.AllArgsConstructor; import lombok.Getter; diff --git a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableSink.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSink.java similarity index 97% rename from seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableSink.java rename to seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSink.java index 7abb176117d..bb04283ca68 100644 --- a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableSink.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSink.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.common.multitablesink; +package org.apache.seatunnel.api.sink.multitablesink; import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.serialization.DefaultSerializer; @@ -28,6 +28,8 @@ import org.apache.seatunnel.api.table.factory.MultiTableFactoryContext; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import lombok.Getter; + import java.io.IOException; import java.util.Collection; import java.util.HashMap; @@ -44,7 +46,7 @@ public class MultiTableSink MultiTableCommitInfo, MultiTableAggregatedCommitInfo> { - private final Map sinks; + @Getter private final Map sinks; private final int replicaNum; public MultiTableSink(MultiTableFactoryContext context) { diff --git a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableSinkAggregatedCommitter.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkAggregatedCommitter.java similarity index 99% rename from seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableSinkAggregatedCommitter.java rename to seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkAggregatedCommitter.java index 31dd91f1eec..6ed04d871bf 100644 --- a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableSinkAggregatedCommitter.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkAggregatedCommitter.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.common.multitablesink; +package org.apache.seatunnel.api.sink.multitablesink; import org.apache.seatunnel.api.sink.MultiTableResourceManager; import org.apache.seatunnel.api.sink.SinkAggregatedCommitter; diff --git a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableSinkCommitter.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkCommitter.java similarity index 98% rename from seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableSinkCommitter.java rename to seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkCommitter.java index ed52fafb002..113e269fd07 100644 --- a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableSinkCommitter.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkCommitter.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.common.multitablesink; +package org.apache.seatunnel.api.sink.multitablesink; import org.apache.seatunnel.api.sink.SinkCommitter; diff --git a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableSinkFactory.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkFactory.java similarity index 96% rename from seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableSinkFactory.java rename to seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkFactory.java index 00e1e1ab133..08db91b7c8e 100644 --- a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableSinkFactory.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkFactory.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.common.multitablesink; +package org.apache.seatunnel.api.sink.multitablesink; import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.table.connector.TableSink; diff --git a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableSinkWriter.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkWriter.java similarity index 99% rename from seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableSinkWriter.java rename to seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkWriter.java index 12163676d7d..3c73435fafb 100644 --- a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableSinkWriter.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkWriter.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.common.multitablesink; +package org.apache.seatunnel.api.sink.multitablesink; import org.apache.seatunnel.api.sink.MultiTableResourceManager; import org.apache.seatunnel.api.sink.SinkWriter; diff --git a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableState.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableState.java similarity index 93% rename from seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableState.java rename to seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableState.java index 43f5d8bd996..ac7db893ba0 100644 --- a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableState.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableState.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.common.multitablesink; +package org.apache.seatunnel.api.sink.multitablesink; import lombok.AllArgsConstructor; import lombok.Getter; diff --git a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableWriterRunnable.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableWriterRunnable.java similarity index 97% rename from seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableWriterRunnable.java rename to seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableWriterRunnable.java index ce22e0e2e20..3026dc778b8 100644 --- a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/MultiTableWriterRunnable.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableWriterRunnable.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.common.multitablesink; +package org.apache.seatunnel.api.sink.multitablesink; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.table.type.SeaTunnelRow; diff --git a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/SinkContextProxy.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/SinkContextProxy.java similarity index 95% rename from seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/SinkContextProxy.java rename to seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/SinkContextProxy.java index f7691ddedff..3a97bb27bc9 100644 --- a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/SinkContextProxy.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/SinkContextProxy.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.common.multitablesink; +package org.apache.seatunnel.api.sink.multitablesink; import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.event.EventListener; diff --git a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/SinkIdentifier.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/SinkIdentifier.java similarity index 94% rename from seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/SinkIdentifier.java rename to seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/SinkIdentifier.java index 18f7484853d..50eac7c0d9d 100644 --- a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/multitablesink/SinkIdentifier.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/SinkIdentifier.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.common.multitablesink; +package org.apache.seatunnel.api.sink.multitablesink; import lombok.EqualsAndHashCode; import lombok.Getter; diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java index 668ff2a43c8..79c0c18706f 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.env.ParsingMode; import org.apache.seatunnel.api.sink.SeaTunnelSink; +import org.apache.seatunnel.api.sink.multitablesink.MultiTableSinkFactory; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SourceOptions; import org.apache.seatunnel.api.source.SourceSplit; @@ -151,7 +152,7 @@ SeaTunnelSink createMultiTableSi ClassLoader classLoader) { try { TableSinkFactory factory = - discoverFactory(classLoader, TableSinkFactory.class, "MultiTableSink"); + new MultiTableSinkFactory(); MultiTableFactoryContext context = new MultiTableFactoryContext(options, classLoader, sinks); ConfigValidator.of(context.getOptions()).validate(factory.optionRule()); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java index e9928a018a1..aa74460b056 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java @@ -26,9 +26,9 @@ import org.apache.seatunnel.api.sink.SaveModeHandler; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.sink.SupportSaveMode; +import org.apache.seatunnel.api.sink.multitablesink.MultiTableSink; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.common.utils.ExceptionUtils; -import org.apache.seatunnel.common.utils.ReflectionUtils; import org.apache.seatunnel.common.utils.RetryUtils; import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.engine.checkpoint.storage.exception.CheckpointStorageException; @@ -375,13 +375,8 @@ public static void handleSaveMode(SeaTunnelSink sink) { throw new SeaTunnelRuntimeException(HANDLE_SAVE_MODE_FAILED, e); } } - } else if (sink.getClass() - .getName() - .equals( - "org.apache.seatunnel.connectors.seatunnel.common.multitablesink.MultiTableSink")) { - // TODO we should not use class name to judge the sink type - Map sinks = - (Map) ReflectionUtils.getField(sink, "sinks").get(); + } else if (sink instanceof MultiTableSink) { + Map sinks = ((MultiTableSink) sink).getSinks(); for (SeaTunnelSink seaTunnelSink : sinks.values()) { handleSaveMode(seaTunnelSink); } From 0d08b200619bdc2017236c46d4134e56f095e738 Mon Sep 17 00:00:00 2001 From: dailai Date: Thu, 25 Jul 2024 10:00:13 +0800 Subject: [PATCH 004/361] [Improve][Connector-v2] Optimize the count table rows for jdbc-oracle and oracle-cdc (#7248) --- docs/en/connector-v2/source/Jdbc.md | 198 +++++------------- docs/en/connector-v2/source/Oracle-CDC.md | 40 ++++ .../cdc/oracle/config/OracleSourceConfig.java | 9 + .../config/OracleSourceConfigFactory.java | 16 ++ .../source/OracleIncrementalSource.java | 2 + .../OracleIncrementalSourceFactory.java | 2 + .../oracle/source/OracleSourceOptions.java | 12 ++ .../source/eumerator/OracleChunkSplitter.java | 6 +- .../cdc/oracle/utils/OracleUtils.java | 55 +++-- .../jdbc/config/JdbcSourceOptions.java | 12 ++ .../jdbc/config/JdbcSourceTableConfig.java | 8 + .../dialect/oracle/OracleDialect.java | 39 ++-- .../jdbc/source/JdbcSourceFactory.java | 4 + .../jdbc/source/JdbcSourceTable.java | 2 + .../jdbc/utils/JdbcCatalogUtils.java | 2 + .../seatunnel/cdc/oracle/OracleCDCIT.java | 31 ++- ..._console.conf => oraclecdc_to_oracle.conf} | 0 .../oraclecdc_to_oracle_skip_analysis.conf | 66 ++++++ .../oraclecdc_to_oracle_use_select_count.conf | 66 ++++++ .../seatunnel/jdbc/JdbcOracleIT.java | 28 ++- ...dbc_oracle_source_to_sink_use_select1.conf | 58 +++++ ...dbc_oracle_source_to_sink_use_select2.conf | 58 +++++ ...dbc_oracle_source_to_sink_use_select3.conf | 59 ++++++ 23 files changed, 594 insertions(+), 179 deletions(-) rename seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/{oraclecdc_to_console.conf => oraclecdc_to_oracle.conf} (100%) create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_skip_analysis.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_use_select_count.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select1.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select2.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select3.conf diff --git a/docs/en/connector-v2/source/Jdbc.md b/docs/en/connector-v2/source/Jdbc.md index 7655083172a..7fab8d50b25 100644 --- a/docs/en/connector-v2/source/Jdbc.md +++ b/docs/en/connector-v2/source/Jdbc.md @@ -39,104 +39,32 @@ supports query SQL and can achieve projection effect. ## Options -| name | type | required | default value | -|--------------------------------------------|--------|----------|-----------------| -| url | String | Yes | - | -| driver | String | Yes | - | -| user | String | No | - | -| password | String | No | - | -| query | String | No | - | -| compatible_mode | String | No | - | -| connection_check_timeout_sec | Int | No | 30 | -| partition_column | String | No | - | -| partition_upper_bound | Long | No | - | -| partition_lower_bound | Long | No | - | -| partition_num | Int | No | job parallelism | -| fetch_size | Int | No | 0 | -| properties | Map | No | - | -| table_path | String | No | - | -| table_list | Array | No | - | -| where_condition | String | No | - | -| split.size | Int | No | 8096 | -| split.even-distribution.factor.lower-bound | Double | No | 0.05 | -| split.even-distribution.factor.upper-bound | Double | No | 100 | -| split.sample-sharding.threshold | Int | No | 1000 | -| split.inverse-sampling.rate | Int | No | 1000 | -| common-options | | No | - | - -### driver [string] - -The jdbc class name used to connect to the remote data source, if you use MySQL the value is `com.mysql.cj.jdbc.Driver`. - -### user [string] - -userName - -### password [string] - -password - -### url [string] - -The URL of the JDBC connection. Refer to a case: jdbc:postgresql://localhost/test - -### query [string] - -Query statement - -### compatible_mode [string] - -The compatible mode of database, required when the database supports multiple compatible modes. For example, when using OceanBase database, you need to set it to 'mysql' or 'oracle'. - -### connection_check_timeout_sec [int] - -The time in seconds to wait for the database operation used to validate the connection to complete. - -### fetch_size [int] - -For queries that return a large number of objects, you can configure the row fetch size used in the query to -improve performance by reducing the number database hits required to satisfy the selection criteria. Zero means use jdbc default value. - -### properties - -Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. - -### table_path - -The path to the full path of table, you can use this configuration instead of `query`. - -examples: -- mysql: "testdb.table1" -- oracle: "test_schema.table1" -- sqlserver: "testdb.test_schema.table1" -- postgresql: "testdb.test_schema.table1" -- iris: "test_schema.table1" - -### table_list - -The list of tables to be read, you can use this configuration instead of `table_path` - -example - -```hocon -table_list = [ - { - table_path = "testdb.table1" - } - { - table_path = "testdb.table2" - query = "select * from testdb.table2 where id > 100" - } -] -``` - -### where_condition - -Common row filter conditions for all tables/queries, must start with `where`. for example `where id > 100` - -### common options - -Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details. +| name | type | required | default value | description | +|--------------------------------------------|---------|----------|-----------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:postgresql://localhost/test | +| driver | String | Yes | - | The jdbc class name used to connect to the remote data source, if you use MySQL the value is `com.mysql.cj.jdbc.Driver`. | +| user | String | No | - | userName | +| password | String | No | - | password | +| query | String | No | - | Query statement | +| compatible_mode | String | No | - | The compatible mode of database, required when the database supports multiple compatible modes. For example, when using OceanBase database, you need to set it to 'mysql' or 'oracle'. | +| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. | +| partition_column | String | No | - | The column name for split data. | +| partition_upper_bound | Long | No | - | The partition_column max value for scan, if not set SeaTunnel will query database get max value. | +| partition_lower_bound | Long | No | - | The partition_column min value for scan, if not set SeaTunnel will query database get min value. | +| partition_num | Int | No | job parallelism | Not recommended for use, The correct approach is to control the number of split through `split.size`
How many splits do we need to split into, only support positive integer. default value is job parallelism. | +| use_select_count | Boolean | No | false | Use select count for table count rather then other methods in dynamic chunk split stage. This is currently only available for jdbc-oracle.In this scenario, select count directly is used when it is faster to update statistics using sql from analysis table | +| skip_analyze | Boolean | No | false | Skip the analysis of table count in dynamic chunk split stage. This is currently only available for jdbc-oracle.In this scenario, you schedule analysis table sql to update related table statistics periodically or your table data does not change frequently | +| fetch_size | Int | No | 0 | For queries that return a large number of objects, you can configure the row fetch size used in the query to improve performance by reducing the number database hits required to satisfy the selection criteria. Zero means use jdbc default value. | +| properties | Map | No | - | Additional connection configuration parameters,when properties and URL have the same parameters, the priority is determined by the
specific implementation of the driver. For example, in MySQL, properties take precedence over the URL. | +| table_path | String | No | - | The path to the full path of table, you can use this configuration instead of `query`.
examples:
`- mysql: "testdb.table1" `
`- oracle: "test_schema.table1" `
`- sqlserver: "testdb.test_schema.table1"`
`- postgresql: "testdb.test_schema.table1"`
`- iris: "test_schema.table1"` | +| table_list | Array | No | - | The list of tables to be read, you can use this configuration instead of `table_path` | +| where_condition | String | No | - | Common row filter conditions for all tables/queries, must start with `where`. for example `where id > 100` | +| split.size | Int | No | 8096 | How many rows in one split, captured tables are split into multiple splits when read of table. | +| split.even-distribution.factor.lower-bound | Double | No | 0.05 | Not recommended for use.
The lower bound of the chunk key distribution factor. This factor is used to determine whether the table data is evenly distributed. If the distribution factor is calculated to be greater than or equal to this lower bound (i.e., (MAX(id) - MIN(id) + 1) / row count), the table chunks would be optimized for even distribution. Otherwise, if the distribution factor is less, the table will be considered as unevenly distributed and the sampling-based sharding strategy will be used if the estimated shard count exceeds the value specified by `sample-sharding.threshold`. The default value is 0.05. | +| split.even-distribution.factor.upper-bound | Double | No | 100 | Not recommended for use.
The upper bound of the chunk key distribution factor. This factor is used to determine whether the table data is evenly distributed. If the distribution factor is calculated to be less than or equal to this upper bound (i.e., (MAX(id) - MIN(id) + 1) / row count), the table chunks would be optimized for even distribution. Otherwise, if the distribution factor is greater, the table will be considered as unevenly distributed and the sampling-based sharding strategy will be used if the estimated shard count exceeds the value specified by `sample-sharding.threshold`. The default value is 100.0. | +| split.sample-sharding.threshold | Int | No | 1000 | This configuration specifies the threshold of estimated shard count to trigger the sample sharding strategy. When the distribution factor is outside the bounds specified by `chunk-key.even-distribution.factor.upper-bound` and `chunk-key.even-distribution.factor.lower-bound`, and the estimated shard count (calculated as approximate row count / chunk size) exceeds this threshold, the sample sharding strategy will be used. This can help to handle large datasets more efficiently. The default value is 1000 shards. | +| split.inverse-sampling.rate | Int | No | 1000 | The inverse of the sampling rate used in the sample sharding strategy. For example, if this value is set to 1000, it means a 1/1000 sampling rate is applied during the sampling process. This option provides flexibility in controlling the granularity of the sampling, thus affecting the final number of shards. It's especially useful when dealing with very large datasets where a lower sampling rate is preferred. The default value is 1000. | +| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details. | ## Parallel Reader @@ -152,50 +80,6 @@ The JDBC Source connector supports parallel reading of data from tables. SeaTunn * Number(int, bigint, decimal, ...) * Date -### Options Related To Split - -#### split.size - -How many rows in one split, captured tables are split into multiple splits when read of table. - -#### split.even-distribution.factor.lower-bound - -> Not recommended for use - -The lower bound of the chunk key distribution factor. This factor is used to determine whether the table data is evenly distributed. If the distribution factor is calculated to be greater than or equal to this lower bound (i.e., (MAX(id) - MIN(id) + 1) / row count), the table chunks would be optimized for even distribution. Otherwise, if the distribution factor is less, the table will be considered as unevenly distributed and the sampling-based sharding strategy will be used if the estimated shard count exceeds the value specified by `sample-sharding.threshold`. The default value is 0.05. - -#### split.even-distribution.factor.upper-bound - -> Not recommended for use - -The upper bound of the chunk key distribution factor. This factor is used to determine whether the table data is evenly distributed. If the distribution factor is calculated to be less than or equal to this upper bound (i.e., (MAX(id) - MIN(id) + 1) / row count), the table chunks would be optimized for even distribution. Otherwise, if the distribution factor is greater, the table will be considered as unevenly distributed and the sampling-based sharding strategy will be used if the estimated shard count exceeds the value specified by `sample-sharding.threshold`. The default value is 100.0. - -#### split.sample-sharding.threshold - -This configuration specifies the threshold of estimated shard count to trigger the sample sharding strategy. When the distribution factor is outside the bounds specified by `chunk-key.even-distribution.factor.upper-bound` and `chunk-key.even-distribution.factor.lower-bound`, and the estimated shard count (calculated as approximate row count / chunk size) exceeds this threshold, the sample sharding strategy will be used. This can help to handle large datasets more efficiently. The default value is 1000 shards. - -#### split.inverse-sampling.rate - -The inverse of the sampling rate used in the sample sharding strategy. For example, if this value is set to 1000, it means a 1/1000 sampling rate is applied during the sampling process. This option provides flexibility in controlling the granularity of the sampling, thus affecting the final number of shards. It's especially useful when dealing with very large datasets where a lower sampling rate is preferred. The default value is 1000. - -#### partition_column [string] - -The column name for split data. - -#### partition_upper_bound [BigDecimal] - -The partition_column max value for scan, if not set SeaTunnel will query database get max value. - -#### partition_lower_bound [BigDecimal] - -The partition_column min value for scan, if not set SeaTunnel will query database get min value. - -#### partition_num [int] - -> Not recommended for use, The correct approach is to control the number of split through `split.size` - -How many splits do we need to split into, only support positive integer. default value is job parallelism. - ## tips > If the table can not be split(for example, table have no Primary Key or Unique Index, and `partition_column` is not set), it will run in single concurrency. @@ -235,6 +119,35 @@ there are some reference value for params above. ### simple +#### Case 1 + +``` +Jdbc { + url = "jdbc:mysql://localhost/test?serverTimezone=GMT%2b8" + driver = "com.mysql.cj.jdbc.Driver" + connection_check_timeout_sec = 100 + user = "root" + password = "123456" + query = "select * from type_bin" +} +``` + +#### Case 2 Use the select count(*) instead of analysis table for count table rows in dynamic chunk split stage + +``` +Jdbc { + url = "jdbc:mysql://localhost/test?serverTimezone=GMT%2b8" + driver = "com.mysql.cj.jdbc.Driver" + connection_check_timeout_sec = 100 + user = "root" + password = "123456" + use_select_count = true + query = "select * from type_bin" +} +``` + +#### Case 3 Use the select NUM_ROWS from all_tables for the table rows but skip the analyze table. + ``` Jdbc { url = "jdbc:mysql://localhost/test?serverTimezone=GMT%2b8" @@ -242,6 +155,7 @@ Jdbc { connection_check_timeout_sec = 100 user = "root" password = "123456" + skip_analyze = true query = "select * from type_bin" } ``` diff --git a/docs/en/connector-v2/source/Oracle-CDC.md b/docs/en/connector-v2/source/Oracle-CDC.md index 2dfffedc66d..cedbda141f6 100644 --- a/docs/en/connector-v2/source/Oracle-CDC.md +++ b/docs/en/connector-v2/source/Oracle-CDC.md @@ -244,6 +244,8 @@ exit; | sample-sharding.threshold | Integer | No | 1000 | This configuration specifies the threshold of estimated shard count to trigger the sample sharding strategy. When the distribution factor is outside the bounds specified by `chunk-key.even-distribution.factor.upper-bound` and `chunk-key.even-distribution.factor.lower-bound`, and the estimated shard count (calculated as approximate row count / chunk size) exceeds this threshold, the sample sharding strategy will be used. This can help to handle large datasets more efficiently. The default value is 1000 shards. | | inverse-sampling.rate | Integer | No | 1000 | The inverse of the sampling rate used in the sample sharding strategy. For example, if this value is set to 1000, it means a 1/1000 sampling rate is applied during the sampling process. This option provides flexibility in controlling the granularity of the sampling, thus affecting the final number of shards. It's especially useful when dealing with very large datasets where a lower sampling rate is preferred. The default value is 1000. | | exactly_once | Boolean | No | false | Enable exactly once semantic. | +| use_select_count | Boolean | No | false | Use select count for table count rather then other methods in full stage.In this scenario, select count directly is used when it is faster to update statistics using sql from analysis table | +| skip_analyze | Boolean | No | false | Skip the analysis of table count in full stage.In this scenario, you schedule analysis table sql to update related table statistics periodically or your table data does not change frequently | | format | Enum | No | DEFAULT | Optional output format for Oracle CDC, valid enumerations are `DEFAULT`、`COMPATIBLE_DEBEZIUM_JSON`. | | debezium | Config | No | - | Pass-through [Debezium's properties](https://github.com/debezium/debezium/blob/v1.9.8.Final/documentation/modules/ROOT/pages/connectors/oracle.adoc#connector-properties) to Debezium Embedded Engine which is used to capture data changes from Oracle server. | | common-options | | no | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details | @@ -270,6 +272,44 @@ source { } ``` +> Use the select count(*) instead of analysis table for count table rows in full stage +> +> ```conf +> source { +> # This is a example source plugin **only for test and demonstrate the feature source plugin** +> Oracle-CDC { +> result_table_name = "customers" +> use_select_count = true +> username = "system" +> password = "oracle" +> database-names = ["XE"] +> schema-names = ["DEBEZIUM"] +> table-names = ["XE.DEBEZIUM.FULL_TYPES"] +> base-url = "jdbc:oracle:thin:system/oracle@oracle-host:1521:xe" +> source.reader.close.timeout = 120000 +> } +> } +> ``` +> +> Use the select NUM_ROWS from all_tables for the table rows but skip the analyze table. +> +> ```conf +> source { +> # This is a example source plugin **only for test and demonstrate the feature source plugin** +> Oracle-CDC { +> result_table_name = "customers" +> skip_analyze = true +> username = "system" +> password = "oracle" +> database-names = ["XE"] +> schema-names = ["DEBEZIUM"] +> table-names = ["XE.DEBEZIUM.FULL_TYPES"] +> base-url = "jdbc:oracle:thin:system/oracle@oracle-host:1521:xe" +> source.reader.close.timeout = 120000 +> } +> } +> ``` + ### Support custom primary key for table ``` diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfig.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfig.java index 5cdf1e9eecd..32bcb41f78f 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfig.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfig.java @@ -24,6 +24,7 @@ import io.debezium.config.Configuration; import io.debezium.connector.oracle.OracleConnectorConfig; import io.debezium.relational.RelationalTableFilters; +import lombok.Getter; import java.util.List; import java.util.Properties; @@ -32,11 +33,17 @@ * Describes the connection information of the Oracle database and the configuration information for * performing snapshotting and streaming reading, such as splitSize. */ +@Getter public class OracleSourceConfig extends JdbcSourceConfig { private static final long serialVersionUID = 1L; + private final Boolean useSelectCount; + private final Boolean skipAnalyze; + public OracleSourceConfig( + Boolean useSelectCount, + Boolean skipAnalyze, StartupConfig startupConfig, StopConfig stopConfig, List databaseList, @@ -82,6 +89,8 @@ public OracleSourceConfig( connectMaxRetries, connectionPoolSize, exactlyOnce); + this.useSelectCount = useSelectCount; + this.skipAnalyze = skipAnalyze; } @Override diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfigFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfigFactory.java index 8e175bd7fe4..d6018083c29 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfigFactory.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/config/OracleSourceConfigFactory.java @@ -38,6 +38,10 @@ public class OracleSourceConfigFactory extends JdbcSourceConfigFactory { private static final String DRIVER_CLASS_NAME = "oracle.jdbc.driver.OracleDriver"; private List schemaList; + + private Boolean useSelectCount; + + private Boolean skipAnalyze; /** * An optional list of regular expressions that match schema names to be monitored; any schema * name not included in the whitelist will be excluded from monitoring. By default all @@ -48,6 +52,16 @@ public JdbcSourceConfigFactory schemaList(List schemaList) { return this; } + public JdbcSourceConfigFactory useSelectCount(Boolean useSelectCount) { + this.useSelectCount = useSelectCount; + return this; + } + + public JdbcSourceConfigFactory skipAnalyze(Boolean skipAnalyze) { + this.skipAnalyze = skipAnalyze; + return this; + } + /** Creates a new {@link OracleSourceConfig} for the given subtask {@code subtaskId}. */ public OracleSourceConfig create(int subtask) { @@ -123,6 +137,8 @@ public OracleSourceConfig create(int subtask) { } return new OracleSourceConfig( + useSelectCount, + skipAnalyze, startupConfig, stopConfig, databaseList, diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSource.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSource.java index f3357f46e07..a1bbd0cb25c 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSource.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSource.java @@ -88,6 +88,8 @@ public SourceConfig.Factory createSourceConfigFactory(Readonly configFactory.startupOptions(startupConfig); configFactory.stopOptions(stopConfig); configFactory.schemaList(config.get(OracleSourceOptions.SCHEMA_NAMES)); + configFactory.useSelectCount(config.get(OracleSourceOptions.USE_SELECT_COUNT)); + configFactory.skipAnalyze(config.get(OracleSourceOptions.SKIP_ANALYZE)); configFactory.originUrl(config.get(JdbcCatalogOptions.BASE_URL)); return configFactory; } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSourceFactory.java index 2a0dc6b2907..21e08c2af7f 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSourceFactory.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleIncrementalSourceFactory.java @@ -61,6 +61,8 @@ public OptionRule optionRule() { JdbcCatalogOptions.BASE_URL, JdbcSourceOptions.DATABASE_NAMES, OracleSourceOptions.SCHEMA_NAMES, + OracleSourceOptions.USE_SELECT_COUNT, + OracleSourceOptions.SKIP_ANALYZE, JdbcSourceOptions.SERVER_TIME_ZONE, JdbcSourceOptions.CONNECT_TIMEOUT_MS, JdbcSourceOptions.CONNECT_MAX_RETRIES, diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleSourceOptions.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleSourceOptions.java index e6bbd77a410..f87ea1ccf5c 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleSourceOptions.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleSourceOptions.java @@ -53,4 +53,16 @@ public class OracleSourceOptions { .listType() .noDefaultValue() .withDescription("Schema name of the database to monitor."); + + public static final Option USE_SELECT_COUNT = + Options.key("use_select_count") + .booleanType() + .defaultValue(false) + .withDescription("Use select count for table count in full stage"); + + public static final Option SKIP_ANALYZE = + Options.key("skip_analyze") + .booleanType() + .defaultValue(false) + .withDescription("Skip the analysis of table count in full stage"); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/eumerator/OracleChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/eumerator/OracleChunkSplitter.java index 6525c3a2dbe..21cfebcd470 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/eumerator/OracleChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/eumerator/OracleChunkSplitter.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.AbstractJdbcSourceChunkSplitter; import org.apache.seatunnel.connectors.cdc.base.utils.ObjectUtils; +import org.apache.seatunnel.connectors.seatunnel.cdc.oracle.config.OracleSourceConfig; import org.apache.seatunnel.connectors.seatunnel.cdc.oracle.utils.OracleTypeUtils; import org.apache.seatunnel.connectors.seatunnel.cdc.oracle.utils.OracleUtils; @@ -41,8 +42,11 @@ @Slf4j public class OracleChunkSplitter extends AbstractJdbcSourceChunkSplitter { + private final OracleSourceConfig oracleSourceConfig; + public OracleChunkSplitter(JdbcSourceConfig sourceConfig, JdbcDataSourceDialect dialect) { super(sourceConfig, dialect); + this.oracleSourceConfig = (OracleSourceConfig) sourceConfig; } @Override @@ -80,7 +84,7 @@ public Object queryNextChunkMax( @Override public Long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId) throws SQLException { - return OracleUtils.queryApproximateRowCnt(jdbc, tableId); + return OracleUtils.queryApproximateRowCnt(oracleSourceConfig, jdbc, tableId); } @Override diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleUtils.java index 1994bd6e03f..fbb3664be0a 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleUtils.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.connectors.cdc.base.utils.SourceRecordUtils; +import org.apache.seatunnel.connectors.seatunnel.cdc.oracle.config.OracleSourceConfig; import org.apache.seatunnel.connectors.seatunnel.cdc.oracle.source.offset.RedoLogOffset; import org.apache.kafka.connect.source.SourceRecord; @@ -81,27 +82,41 @@ public static Object[] queryMinMax(JdbcConnection jdbc, TableId tableId, String }); } - public static long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId) + public static long queryApproximateRowCnt( + OracleSourceConfig oracleSourceConfig, JdbcConnection jdbc, TableId tableId) throws SQLException { - final String analyzeTable = - String.format( - "analyze table %s compute statistics for table", - quoteSchemaAndTable(tableId)); - final String rowCountQuery = - String.format( - "select NUM_ROWS from all_tables where TABLE_NAME = '%s'", tableId.table()); - return jdbc.execute(analyzeTable) - .queryAndMap( - rowCountQuery, - rs -> { - if (!rs.next()) { - throw new SQLException( - String.format( - "No result returned after running query [%s]", - rowCountQuery)); - } - return rs.getLong(1); - }); + Boolean useSelectCount = oracleSourceConfig.getUseSelectCount(); + String rowCountQuery; + if (useSelectCount) { + rowCountQuery = String.format("select count(*) from %s", quoteSchemaAndTable(tableId)); + } else { + rowCountQuery = + String.format( + "select NUM_ROWS from all_tables where TABLE_NAME = '%s'", + tableId.table()); + Boolean skipAnalyze = oracleSourceConfig.getSkipAnalyze(); + if (!skipAnalyze) { + final String analyzeTable = + String.format( + "analyze table %s compute statistics for table", + quoteSchemaAndTable(tableId)); + // not skip analyze + log.info("analyze table sql: {}", analyzeTable); + jdbc.execute(analyzeTable); + } + } + log.info("row count query: {}", rowCountQuery); + return jdbc.queryAndMap( + rowCountQuery, + rs -> { + if (!rs.next()) { + throw new SQLException( + String.format( + "No result returned after running query [%s]", + rowCountQuery)); + } + return rs.getLong(1); + }); } public static Object queryMin( diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceOptions.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceOptions.java index 14ea5873350..6647d9c8eb1 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceOptions.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceOptions.java @@ -93,4 +93,16 @@ public interface JdbcSourceOptions { + "The value represents the denominator of the sampling rate fraction. " + "For example, a value of 1000 means a sampling rate of 1/1000. " + "This parameter is used when the sample sharding strategy is triggered."); + + Option USE_SELECT_COUNT = + Options.key("use_select_count") + .booleanType() + .defaultValue(false) + .withDescription("Use select count for table count"); + + Option SKIP_ANALYZE = + Options.key("skip_analyze") + .booleanType() + .defaultValue(false) + .withDescription("Skip the analysis of table count"); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceTableConfig.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceTableConfig.java index b4a6e890dfc..d217a0b745a 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceTableConfig.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceTableConfig.java @@ -58,6 +58,12 @@ public class JdbcSourceTableConfig implements Serializable { @JsonProperty("partition_upper_bound") private BigDecimal partitionEnd; + @JsonProperty("use_select_count") + private Boolean useSelectCount; + + @JsonProperty("skip_analyze") + private Boolean skipAnalyze; + @Tolerate public JdbcSourceTableConfig() {} @@ -79,6 +85,8 @@ public static List of(ReadonlyConfig connectorConfig) { .partitionNumber(connectorConfig.get(JdbcOptions.PARTITION_NUM)) .partitionStart(connectorConfig.get(JdbcOptions.PARTITION_LOWER_BOUND)) .partitionEnd(connectorConfig.get(JdbcOptions.PARTITION_UPPER_BOUND)) + .useSelectCount(connectorConfig.get(JdbcSourceOptions.USE_SELECT_COUNT)) + .skipAnalyze(connectorConfig.get(JdbcSourceOptions.SKIP_ANALYZE)) .build(); tableList = Collections.singletonList(tableProperty); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java index 1bf14669490..e1aee7f7d88 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java @@ -180,34 +180,47 @@ public String tableIdentifier(TablePath tablePath) { public Long approximateRowCntStatement(Connection connection, JdbcSourceTable table) throws SQLException { - // 1. If no query is configured, use TABLE STATUS. - // 2. If a query is configured but does not contain a WHERE clause and tablePath is + // 1. Use select count + // 2. If no query is configured, use TABLE STATUS. + // 3. If a query is configured but does not contain a WHERE clause and tablePath is // configured, use TABLE STATUS. - // 3. If a query is configured with a WHERE clause, or a query statement is configured but + // 4. If a query is configured with a WHERE clause, or a query statement is configured but // tablePath is TablePath.DEFAULT, use COUNT(*). + String query = table.getQuery(); + boolean useTableStats = - StringUtils.isBlank(table.getQuery()) - || (!table.getQuery().toLowerCase().contains("where") + StringUtils.isBlank(query) + || (!query.toLowerCase().contains("where") && table.getTablePath() != null && !TablePath.DEFAULT .getFullName() .equals(table.getTablePath().getFullName())); + if (table.getUseSelectCount()) { + useTableStats = false; + if (StringUtils.isBlank(query)) { + query = "SELECT * FROM " + tableIdentifier(table.getTablePath()); + } + } + if (useTableStats) { TablePath tablePath = table.getTablePath(); - String analyzeTable = - String.format( - "analyze table %s compute statistics for table", - tableIdentifier(tablePath)); String rowCountQuery = String.format( "select NUM_ROWS from all_tables where OWNER = '%s' AND TABLE_NAME = '%s' ", tablePath.getSchemaName(), tablePath.getTableName()); - try (Statement stmt = connection.createStatement()) { - log.info("Split Chunk, approximateRowCntStatement: {}", analyzeTable); - stmt.execute(analyzeTable); + String analyzeTable = + String.format( + "analyze table %s compute statistics for table", + tableIdentifier(tablePath)); + if (!table.getSkipAnalyze()) { + log.info("Split Chunk, approximateRowCntStatement: {}", analyzeTable); + stmt.execute(analyzeTable); + } else { + log.warn("Skip analyze, approximateRowCntStatement: {}", analyzeTable); + } log.info("Split Chunk, approximateRowCntStatement: {}", rowCountQuery); try (ResultSet rs = stmt.executeQuery(rowCountQuery)) { if (!rs.next()) { @@ -220,7 +233,7 @@ public Long approximateRowCntStatement(Connection connection, JdbcSourceTable ta } } } - return SQLUtils.countForSubquery(connection, table.getQuery()); + return SQLUtils.countForSubquery(connection, query); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java index 54e8d5173b3..b9ca90ed538 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java @@ -46,6 +46,7 @@ import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions.QUERY; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions.URL; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions.USER; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.SKIP_ANALYZE; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.SPLIT_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.SPLIT_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.SPLIT_INVERSE_SAMPLING_RATE; @@ -53,6 +54,7 @@ import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.SPLIT_SIZE; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.TABLE_LIST; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.TABLE_PATH; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.USE_SELECT_COUNT; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceOptions.WHERE_CONDITION; @Slf4j @@ -94,6 +96,8 @@ public OptionRule optionRule() { COMPATIBLE_MODE, PROPERTIES, QUERY, + USE_SELECT_COUNT, + SKIP_ANALYZE, TABLE_PATH, WHERE_CONDITION, TABLE_LIST, diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceTable.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceTable.java index fea73824720..8aad94c8b69 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceTable.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceTable.java @@ -37,5 +37,7 @@ public class JdbcSourceTable implements Serializable { private final Integer partitionNumber; private final BigDecimal partitionStart; private final BigDecimal partitionEnd; + private final Boolean useSelectCount; + private final Boolean skipAnalyze; private final CatalogTable catalogTable; } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java index 83d5bfa7692..a6896322065 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java @@ -90,6 +90,8 @@ public static Map getTables( .partitionNumber(tableConfig.getPartitionNumber()) .partitionStart(tableConfig.getPartitionStart()) .partitionEnd(tableConfig.getPartitionEnd()) + .useSelectCount(tableConfig.getUseSelectCount()) + .skipAnalyze(tableConfig.getSkipAnalyze()) .catalogTable(catalogTable) .build(); tables.put(tablePath, jdbcSourceTable); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCIT.java index 125d57915c2..0192fae3f70 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/OracleCDCIT.java @@ -137,7 +137,22 @@ public void startUp() throws Exception { @TestTemplate public void testOracleCdcCheckDataE2e(TestContainer container) throws Exception { + checkDataForTheJob(container, "/oraclecdc_to_oracle.conf", false); + } + + @TestTemplate + public void testOracleCdcCheckDataE2eForUseSelectCount(TestContainer container) + throws Exception { + checkDataForTheJob(container, "/oraclecdc_to_oracle_use_select_count.conf", false); + } + @TestTemplate + public void testOracleCdcCheckDataE2eForSkipAnalysis(TestContainer container) throws Exception { + checkDataForTheJob(container, "/oraclecdc_to_oracle_skip_analysis.conf", true); + } + + private void checkDataForTheJob( + TestContainer container, String jobConfPath, Boolean skipAnalysis) throws Exception { clearTable(DATABASE, SOURCE_TABLE1); clearTable(DATABASE, SOURCE_TABLE2); clearTable(DATABASE, SINK_TABLE1); @@ -145,10 +160,24 @@ public void testOracleCdcCheckDataE2e(TestContainer container) throws Exception insertSourceTable(DATABASE, SOURCE_TABLE1); + if (skipAnalysis) { + // analyzeTable before execute job + String analyzeTable = + String.format( + "analyze table " + + "\"DEBEZIUM\".\"FULL_TYPES\" " + + "compute statistics for table"); + log.info("analyze table {}", analyzeTable); + try (Connection connection = testConnection(ORACLE_CONTAINER); + Statement statement = connection.createStatement()) { + statement.execute(analyzeTable); + } + } + CompletableFuture.supplyAsync( () -> { try { - container.executeJob("/oraclecdc_to_console.conf"); + container.executeJob(jobConfPath); } catch (Exception e) { log.error("Commit task exception :" + e.getMessage()); throw new RuntimeException(e); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_console.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle.conf similarity index 100% rename from seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_console.conf rename to seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle.conf diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_skip_analysis.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_skip_analysis.conf new file mode 100644 index 00000000000..233fc735ef6 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_skip_analysis.conf @@ -0,0 +1,66 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Oracle-CDC { + result_table_name = "customers" + skip_analyze = true + username = "system" + password = "top_secret" + database-names = ["ORCLCDB"] + schema-names = ["DEBEZIUM"] + table-names = ["ORCLCDB.DEBEZIUM.FULL_TYPES"] + base-url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + source.reader.close.timeout = 120000 + connection.pool.size = 1 + debezium { + # log.mining.strategy = "online_catalog" + # log.mining.continuous.mine = true + database.oracle.jdbc.timezoneAsRegion = "false" + } + } +} + +transform { +} + +sink { +Jdbc { + source_table_name = "customers" + driver = "oracle.jdbc.driver.OracleDriver" + url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + user = "system" + password = "top_secret" + generate_sink_sql = true + database = "ORCLCDB" + table = "DEBEZIUM.SINK_FULL_TYPES" + batch_size = 1 + primary_keys = ["ID"] + connection.pool.size = 1 +} +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_use_select_count.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_use_select_count.conf new file mode 100644 index 00000000000..dd93cd4f9e1 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-oracle-e2e/src/test/resources/oraclecdc_to_oracle_use_select_count.conf @@ -0,0 +1,66 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Oracle-CDC { + result_table_name = "customers" + use_select_count = true + username = "system" + password = "top_secret" + database-names = ["ORCLCDB"] + schema-names = ["DEBEZIUM"] + table-names = ["ORCLCDB.DEBEZIUM.FULL_TYPES"] + base-url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + source.reader.close.timeout = 120000 + connection.pool.size = 1 + debezium { + # log.mining.strategy = "online_catalog" + # log.mining.continuous.mine = true + database.oracle.jdbc.timezoneAsRegion = "false" + } + } +} + +transform { +} + +sink { +Jdbc { + source_table_name = "customers" + driver = "oracle.jdbc.driver.OracleDriver" + url = "jdbc:oracle:thin:@oracle-host:1521/ORCLCDB" + user = "system" + password = "top_secret" + generate_sink_sql = true + database = "ORCLCDB" + table = "DEBEZIUM.SINK_FULL_TYPES" + batch_size = 1 + primary_keys = ["ID"] + connection.pool.size = 1 +} +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java index 70c9d39cf45..9d3597c435b 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java @@ -28,6 +28,7 @@ import org.apache.commons.lang3.tuple.Pair; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.testcontainers.containers.GenericContainer; import org.testcontainers.containers.OracleContainer; @@ -40,6 +41,7 @@ import java.math.BigDecimal; import java.sql.Date; +import java.sql.Statement; import java.sql.Timestamp; import java.time.LocalDate; import java.time.LocalDateTime; @@ -63,7 +65,11 @@ public class JdbcOracleIT extends AbstractJdbcIT { private static final String SINK_TABLE = "E2E_TABLE_SINK"; private static final String CATALOG_TABLE = "E2E_TABLE_CATALOG"; private static final List CONFIG_FILE = - Lists.newArrayList("/jdbc_oracle_source_to_sink.conf"); + Lists.newArrayList( + "/jdbc_oracle_source_to_sink.conf", + "/jdbc_oracle_source_to_sink_use_select1.conf", + "/jdbc_oracle_source_to_sink_use_select2.conf", + "/jdbc_oracle_source_to_sink_use_select3.conf"); private static final String CREATE_SQL = "create table %s\n" @@ -163,7 +169,7 @@ String driverUrl() { @Override Pair> initTestData() { List rows = new ArrayList<>(); - for (int i = 0; i < 100; i++) { + for (int i = 0; i < 20000; i++) { SeaTunnelRow row = new SeaTunnelRow( new Object[] { @@ -237,4 +243,22 @@ protected void initCatalog() { SCHEMA); catalog.open(); } + + @BeforeAll + @Override + public void startUp() { + super.startUp(); + // analyzeTable before execute job + String analyzeTable = + String.format( + "analyze table " + + quoteIdentifier(SOURCE_TABLE) + + " compute statistics for table"); + log.info("analyze table {}", analyzeTable); + try (Statement stmt = connection.createStatement()) { + stmt.execute(analyzeTable); + } catch (Exception e) { + log.error("Error when analyze table", e); + } + } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select1.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select1.conf new file mode 100644 index 00000000000..8a0c8310443 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select1.conf @@ -0,0 +1,58 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Jdbc { + driver = oracle.jdbc.driver.OracleDriver + url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" + user = testUser + password = testPassword + use_select_count = true + query = "SELECT VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_3_SF_2_DP,NUMBER_7_SF_N2_DP,INTEGER_COL,FLOAT_COL,REAL_COL,BINARY_FLOAT_COL,BINARY_DOUBLE_COL,DATE_COL,TIMESTAMP_WITH_3_FRAC_SEC_COL,TIMESTAMP_WITH_LOCAL_TZ,XML_TYPE_COL FROM E2E_TABLE_SOURCE" + properties { + database.oracle.jdbc.timezoneAsRegion = "false" + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of source plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource +} + +sink { + Jdbc { + driver = oracle.jdbc.driver.OracleDriver + url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" + user = testUser + password = testPassword + query = "INSERT INTO E2E_TABLE_SINK (VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_3_SF_2_DP,NUMBER_7_SF_N2_DP,INTEGER_COL,FLOAT_COL,REAL_COL,BINARY_FLOAT_COL,BINARY_DOUBLE_COL,DATE_COL,TIMESTAMP_WITH_3_FRAC_SEC_COL,TIMESTAMP_WITH_LOCAL_TZ,XML_TYPE_COL) VALUES(?,?,?,?,?,?,?,?,?,?,?,?,?,?)" + properties { + database.oracle.jdbc.timezoneAsRegion = "false" + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/sink/Jdbc +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select2.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select2.conf new file mode 100644 index 00000000000..ebebdb55051 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select2.conf @@ -0,0 +1,58 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Jdbc { + driver = oracle.jdbc.driver.OracleDriver + url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" + user = testUser + password = testPassword + use_select_count = true + table_path = TESTUSER.E2E_TABLE_SOURCE + properties { + database.oracle.jdbc.timezoneAsRegion = "false" + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of source plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource +} + +sink { + Jdbc { + driver = oracle.jdbc.driver.OracleDriver + url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" + user = testUser + password = testPassword + query = "INSERT INTO E2E_TABLE_SINK (VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_3_SF_2_DP,NUMBER_7_SF_N2_DP,INTEGER_COL,FLOAT_COL,REAL_COL,BINARY_FLOAT_COL,BINARY_DOUBLE_COL,DATE_COL,TIMESTAMP_WITH_3_FRAC_SEC_COL,TIMESTAMP_WITH_LOCAL_TZ,XML_TYPE_COL) VALUES(?,?,?,?,?,?,?,?,?,?,?,?,?,?)" + properties { + database.oracle.jdbc.timezoneAsRegion = "false" + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/sink/Jdbc +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select3.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select3.conf new file mode 100644 index 00000000000..d00ce9b6434 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_use_select3.conf @@ -0,0 +1,59 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Jdbc { + driver = oracle.jdbc.driver.OracleDriver + url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" + user = testUser + password = testPassword + use_select_count = false + skip_analyze = true + table_path = TESTUSER.E2E_TABLE_SOURCE + properties { + database.oracle.jdbc.timezoneAsRegion = "false" + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of source plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource +} + +sink { + Jdbc { + driver = oracle.jdbc.driver.OracleDriver + url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" + user = testUser + password = testPassword + query = "INSERT INTO E2E_TABLE_SINK (VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_3_SF_2_DP,NUMBER_7_SF_N2_DP,INTEGER_COL,FLOAT_COL,REAL_COL,BINARY_FLOAT_COL,BINARY_DOUBLE_COL,DATE_COL,TIMESTAMP_WITH_3_FRAC_SEC_COL,TIMESTAMP_WITH_LOCAL_TZ,XML_TYPE_COL) VALUES(?,?,?,?,?,?,?,?,?,?,?,?,?,?)" + properties { + database.oracle.jdbc.timezoneAsRegion = "false" + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/sink/Jdbc +} From 876d2f08d28313c963e520374b1d043f86d53302 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Thu, 25 Jul 2024 10:24:55 +0800 Subject: [PATCH 005/361] [Fix][Dist] Remove aws jar dependency in release package (#7257) * [Fix][Dist] Remove aws jar dependency in release package * [Fix][Dist] Remove aws jar dependency in release package --- seatunnel-dist/pom.xml | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index a5dd203f837..c96bf0b612b 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -702,7 +702,7 @@ com.amazonaws aws-java-sdk-bundle - 1.11.271 + ${aws-java-sdk.version} provided @@ -883,12 +883,6 @@ ${project.version} provided - - com.amazonaws - aws-java-sdk-bundle - ${aws-java-sdk.version} - provided - org.apache.seatunnel From d003bd85b65b307e77867781986cc7d58c3ba80a Mon Sep 17 00:00:00 2001 From: zhangdonghao <39961809+hawk9821@users.noreply.github.com> Date: Fri, 26 Jul 2024 10:44:09 +0800 Subject: [PATCH 006/361] [Feature][Zeta] Added the metrics information of table statistics in multi-table mode (#7212) --- .../sink/multitablesink/MultiTableSink.java | 5 + .../engine/e2e/MultiTableMetricsIT.java | 125 ++++++++++++++++++ .../batch_fake_multi_table_to_console.conf | 64 +++++++++ .../engine/client/SeaTunnelClientTest.java | 114 ++++++++++++++++ .../batch_fake_multi_table_to_console.conf | 66 +++++++++ .../rest/RestHttpGetCommandProcessor.java | 74 ++++++++++- .../server/task/SeaTunnelSourceCollector.java | 53 ++++++-- .../server/task/SourceSeaTunnelTask.java | 13 +- .../server/task/flow/SinkFlowLifeCycle.java | 38 ++++++ 9 files changed, 537 insertions(+), 15 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java create mode 100644 seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf create mode 100644 seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fake_multi_table_to_console.conf diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSink.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSink.java index bb04283ca68..923ecff8b88 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSink.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSink.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.api.sink.SinkCommitter; import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.factory.MultiTableFactoryContext; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -149,6 +150,10 @@ public Optional> getCommitInfoSerializer() { return Optional.of(new MultiTableSinkAggregatedCommitter(aggCommitters)); } + public List getSinkTables() { + return sinks.keySet().stream().map(TablePath::of).collect(Collectors.toList()); + } + @Override public Optional> getAggregatedCommitInfoSerializer() { diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java new file mode 100644 index 00000000000..59942eb4cc8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.e2e; + +import org.apache.seatunnel.engine.client.SeaTunnelClient; +import org.apache.seatunnel.engine.client.job.ClientJobExecutionEnvironment; +import org.apache.seatunnel.engine.client.job.ClientJobProxy; +import org.apache.seatunnel.engine.common.config.ConfigProvider; +import org.apache.seatunnel.engine.common.config.JobConfig; +import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; +import org.apache.seatunnel.engine.core.job.JobStatus; +import org.apache.seatunnel.engine.server.SeaTunnelServerStarter; +import org.apache.seatunnel.engine.server.rest.RestConstant; + +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import com.hazelcast.client.config.ClientConfig; +import com.hazelcast.instance.impl.HazelcastInstanceImpl; + +import java.util.Collections; +import java.util.concurrent.TimeUnit; + +import static io.restassured.RestAssured.given; +import static org.hamcrest.Matchers.equalTo; + +public class MultiTableMetricsIT { + + private static final String HOST = "http://localhost:"; + + private static ClientJobProxy batchJobProxy; + + private static HazelcastInstanceImpl node1; + + private static SeaTunnelClient engineClient; + + @BeforeEach + void beforeClass() throws Exception { + String testClusterName = TestUtils.getClusterName("RestApiIT"); + SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); + seaTunnelConfig.getHazelcastConfig().setClusterName(testClusterName); + node1 = SeaTunnelServerStarter.createHazelcastInstance(seaTunnelConfig); + + ClientConfig clientConfig = ConfigProvider.locateAndGetClientConfig(); + clientConfig.setClusterName(testClusterName); + engineClient = new SeaTunnelClient(clientConfig); + + String batchFilePath = TestUtils.getResource("batch_fake_multi_table_to_console.conf"); + JobConfig batchConf = new JobConfig(); + batchConf.setName("batch_fake_multi_table_to_console"); + ClientJobExecutionEnvironment batchJobExecutionEnv = + engineClient.createExecutionContext(batchFilePath, batchConf, seaTunnelConfig); + batchJobProxy = batchJobExecutionEnv.execute(); + Awaitility.await() + .atMost(2, TimeUnit.MINUTES) + .untilAsserted( + () -> + Assertions.assertEquals( + JobStatus.FINISHED, batchJobProxy.getJobStatus())); + } + + @Test + public void multiTableMetrics() { + Collections.singletonList(node1) + .forEach( + instance -> { + given().get( + HOST + + instance.getCluster() + .getLocalMember() + .getAddress() + .getPort() + + RestConstant.JOB_INFO_URL + + "/" + + batchJobProxy.getJobId()) + .then() + .statusCode(200) + .body("jobName", equalTo("batch_fake_multi_table_to_console")) + .body("jobStatus", equalTo("FINISHED")) + .body("metrics.SourceReceivedCount", equalTo("50")) + .body("metrics.SinkWriteCount", equalTo("50")) + .body( + "metrics.TableSourceReceivedCount.'fake.table1'", + equalTo("20")) + .body( + "metrics.TableSourceReceivedCount.'fake.public.table2'", + equalTo("30")) + .body( + "metrics.TableSinkWriteCount.'fake.table1'", + equalTo("20")) + .body( + "metrics.TableSinkWriteCount.'fake.public.table2'", + equalTo("30")); + }); + } + + @AfterEach + void afterClass() { + if (engineClient != null) { + engineClient.close(); + } + + if (node1 != null) { + node1.shutdown(); + } + } +} diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf new file mode 100644 index 00000000000..c51929a0edb --- /dev/null +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf @@ -0,0 +1,64 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + result_table_name = "fake1" + row.num = 20 + schema = { + table = "fake.table1" + fields { + name = "string" + age = "int" + } + } + } + + FakeSource { + result_table_name = "fake2" + row.num = 30 + schema = { + table = "fake.public.table2" + fields { + name = "string" + age = "int" + sex = "int" + } + } + } +} + +transform { +} + +sink { + console { + source_table_name = "fake1" + } + console { + source_table_name = "fake2" + } +} diff --git a/seatunnel-engine/seatunnel-engine-client/src/test/java/org/apache/seatunnel/engine/client/SeaTunnelClientTest.java b/seatunnel-engine/seatunnel-engine-client/src/test/java/org/apache/seatunnel/engine/client/SeaTunnelClientTest.java index d7e55db4ec2..100aa0b3203 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/test/java/org/apache/seatunnel/engine/client/SeaTunnelClientTest.java +++ b/seatunnel-engine/seatunnel-engine-client/src/test/java/org/apache/seatunnel/engine/client/SeaTunnelClientTest.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.engine.client; +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.seatunnel.common.config.Common; @@ -51,10 +53,14 @@ import java.net.URL; import java.util.ArrayList; import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Spliterators; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_COUNT; import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_QPS; @@ -548,6 +554,114 @@ public void testSavePointAndRestoreWithSavePoint() throws Exception { } } + @Test + public void testGetMultiTableJobMetrics() { + Common.setDeployMode(DeployMode.CLIENT); + String filePath = TestUtils.getResource("/batch_fake_multi_table_to_console.conf"); + JobConfig jobConfig = new JobConfig(); + jobConfig.setName("testGetMultiTableJobMetrics"); + + SeaTunnelClient seaTunnelClient = createSeaTunnelClient(); + JobClient jobClient = seaTunnelClient.getJobClient(); + + try { + ClientJobExecutionEnvironment jobExecutionEnv = + seaTunnelClient.createExecutionContext(filePath, jobConfig, SEATUNNEL_CONFIG); + + final ClientJobProxy clientJobProxy = jobExecutionEnv.execute(); + CompletableFuture objectCompletableFuture = + CompletableFuture.supplyAsync( + () -> { + return clientJobProxy.waitForJobComplete(); + }); + long jobId = clientJobProxy.getJobId(); + + await().atMost(30000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> + Assertions.assertTrue( + jobClient.getJobDetailStatus(jobId).contains("FINISHED") + && jobClient + .listJobStatus(true) + .contains("FINISHED"))); + + String jobMetrics = jobClient.getJobMetrics(jobId); + + Assertions.assertTrue(jobMetrics.contains(SOURCE_RECEIVED_COUNT + "#fake.table1")); + Assertions.assertTrue( + jobMetrics.contains(SOURCE_RECEIVED_COUNT + "#fake.public.table2")); + Assertions.assertTrue(jobMetrics.contains(SINK_WRITE_COUNT + "#fake.table1")); + Assertions.assertTrue(jobMetrics.contains(SINK_WRITE_COUNT + "#fake.public.table2")); + + log.info("jobMetrics : {}", jobMetrics); + JsonNode jobMetricsStr = new ObjectMapper().readTree(jobMetrics); + List metricNameList = + StreamSupport.stream( + Spliterators.spliteratorUnknownSize( + jobMetricsStr.fieldNames(), 0), + false) + .filter( + metricName -> + metricName.startsWith(SOURCE_RECEIVED_COUNT) + || metricName.startsWith(SINK_WRITE_COUNT)) + .collect(Collectors.toList()); + + Map totalCount = + metricNameList.stream() + .filter(metrics -> !metrics.contains("#")) + .collect( + Collectors.toMap( + metrics -> metrics, + metrics -> + StreamSupport.stream( + jobMetricsStr + .get(metrics) + .spliterator(), + false) + .mapToLong( + value -> + value.get("value") + .asLong()) + .sum())); + + Map tableCount = + metricNameList.stream() + .filter(metrics -> metrics.contains("#")) + .collect( + Collectors.toMap( + metrics -> metrics, + metrics -> + StreamSupport.stream( + jobMetricsStr + .get(metrics) + .spliterator(), + false) + .mapToLong( + value -> + value.get("value") + .asLong()) + .sum())); + + Assertions.assertEquals( + totalCount.get(SOURCE_RECEIVED_COUNT), + tableCount.entrySet().stream() + .filter(e -> e.getKey().startsWith(SOURCE_RECEIVED_COUNT)) + .mapToLong(Map.Entry::getValue) + .sum()); + Assertions.assertEquals( + totalCount.get(SINK_WRITE_COUNT), + tableCount.entrySet().stream() + .filter(e -> e.getKey().startsWith(SINK_WRITE_COUNT)) + .mapToLong(Map.Entry::getValue) + .sum()); + + } catch (ExecutionException | InterruptedException | JsonProcessingException e) { + throw new RuntimeException(e); + } finally { + seaTunnelClient.close(); + } + } + @AfterAll public static void after() { INSTANCE.shutdown(); diff --git a/seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fake_multi_table_to_console.conf b/seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fake_multi_table_to_console.conf new file mode 100644 index 00000000000..df7ae51fe6e --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fake_multi_table_to_console.conf @@ -0,0 +1,66 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + result_table_name = "fake1" + row.num = 20 + schema = { + table = "fake.table1" + fields { + name = "string" + age = "int" + } + } + parallelism = 1 + } + + FakeSource { + result_table_name = "fake2" + row.num = 30 + schema = { + table = "fake.public.table2" + fields { + name = "string" + age = "int" + sex = "int" + } + } + parallelism = 1 + } +} + +transform { +} + +sink { + console { + source_table_name = "fake1" + } + console { + source_table_name = "fake2" + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java index 6081b0f2eaf..d5d60b7cbb4 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java @@ -22,6 +22,7 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.seatunnel.api.common.metrics.JobMetrics; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.common.utils.DateTimeUtils; import org.apache.seatunnel.common.utils.JsonUtils; import org.apache.seatunnel.engine.common.Constant; @@ -64,8 +65,10 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; +import java.util.Spliterators; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import static com.hazelcast.internal.ascii.rest.HttpStatusCode.SC_500; import static org.apache.seatunnel.engine.server.rest.RestConstant.FINISHED_JOBS_INFO; @@ -79,7 +82,9 @@ public class RestHttpGetCommandProcessor extends HttpCommandProcessor { private static final String SOURCE_RECEIVED_COUNT = "SourceReceivedCount"; + private static final String TABLE_SOURCE_RECEIVED_COUNT = "TableSourceReceivedCount"; private static final String SINK_WRITE_COUNT = "SinkWriteCount"; + private static final String TABLE_SINK_WRITE_COUNT = "TableSinkWriteCount"; private final Log4j2HttpGetCommandProcessor original; private NodeEngine nodeEngine; @@ -362,12 +367,31 @@ private void getRunningThread(HttpGetCommand command) { .collect(JsonArray::new, JsonArray::add, JsonArray::add)); } - private Map getJobMetrics(String jobMetrics) { - Map metricsMap = new HashMap<>(); + private Map getJobMetrics(String jobMetrics) { + Map metricsMap = new HashMap<>(); long sourceReadCount = 0L; long sinkWriteCount = 0L; + Map tableSourceReceivedCountMap = new HashMap<>(); + Map tableSinkWriteCountMap = new HashMap<>(); try { JsonNode jobMetricsStr = new ObjectMapper().readTree(jobMetrics); + StreamSupport.stream( + Spliterators.spliteratorUnknownSize(jobMetricsStr.fieldNames(), 0), + false) + .filter(metricName -> metricName.contains("#")) + .forEach( + metricName -> { + String tableName = + TablePath.of(metricName.split("#")[1]).getFullName(); + if (metricName.startsWith(SOURCE_RECEIVED_COUNT)) { + tableSourceReceivedCountMap.put( + tableName, jobMetricsStr.get(metricName)); + } + if (metricName.startsWith(SOURCE_RECEIVED_COUNT)) { + tableSinkWriteCountMap.put( + tableName, jobMetricsStr.get(metricName)); + } + }); JsonNode sourceReceivedCountJson = jobMetricsStr.get(SOURCE_RECEIVED_COUNT); JsonNode sinkWriteCountJson = jobMetricsStr.get(SINK_WRITE_COUNT); for (int i = 0; i < jobMetricsStr.get(SOURCE_RECEIVED_COUNT).size(); i++) { @@ -379,9 +403,36 @@ private Map getJobMetrics(String jobMetrics) { } catch (JsonProcessingException | NullPointerException e) { return metricsMap; } + + Map tableSourceReceivedCount = + tableSourceReceivedCountMap.entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> + StreamSupport.stream( + entry.getValue().spliterator(), + false) + .mapToLong( + node -> node.get("value").asLong()) + .sum())); + Map tableSinkWriteCount = + tableSinkWriteCountMap.entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> + StreamSupport.stream( + entry.getValue().spliterator(), + false) + .mapToLong( + node -> node.get("value").asLong()) + .sum())); + metricsMap.put(SOURCE_RECEIVED_COUNT, sourceReadCount); metricsMap.put(SINK_WRITE_COUNT, sinkWriteCount); - + metricsMap.put(TABLE_SOURCE_RECEIVED_COUNT, tableSourceReceivedCount); + metricsMap.put(TABLE_SINK_WRITE_COUNT, tableSinkWriteCount); return metricsMap; } @@ -475,11 +526,24 @@ private JsonObject convertToJson(JobInfo jobInfo, long jobId) { .add( RestConstant.IS_START_WITH_SAVE_POINT, jobImmutableInformation.isStartWithSavePoint()) - .add(RestConstant.METRICS, JsonUtil.toJsonObject(getJobMetrics(jobMetrics))); + .add(RestConstant.METRICS, toJsonObject(getJobMetrics(jobMetrics))); return jobInfoJson; } + private JsonObject toJsonObject(Map jobMetrics) { + JsonObject members = new JsonObject(); + jobMetrics.forEach( + (key, value) -> { + if (value instanceof Map) { + members.add(key, toJsonObject((Map) value)); + } else { + members.add(key, value.toString()); + } + }); + return members; + } + private JsonObject getJobInfoJson(JobState jobState, String jobMetrics, JobDAGInfo jobDAGInfo) { return new JsonObject() .add(RestConstant.JOB_ID, String.valueOf(jobState.getJobId())) @@ -498,6 +562,6 @@ private JsonObject getJobInfoJson(JobState jobState, String jobMetrics, JobDAGIn DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS)) .add(RestConstant.JOB_DAG, JsonUtils.toJsonString(jobDAGInfo)) .add(RestConstant.PLUGIN_JARS_URLS, new JsonArray()) - .add(RestConstant.METRICS, JsonUtil.toJsonObject(getJobMetrics(jobMetrics))); + .add(RestConstant.METRICS, toJsonObject(getJobMetrics(jobMetrics))); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java index f5d4aed1ab4..62612d0617a 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.common.metrics.Meter; import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.event.SchemaChangeEvent; import org.apache.seatunnel.api.table.event.handler.DataTypeChangeEventDispatcher; import org.apache.seatunnel.api.table.event.handler.DataTypeChangeEventHandler; @@ -34,12 +35,17 @@ import org.apache.seatunnel.engine.common.exception.SeaTunnelEngineException; import org.apache.seatunnel.engine.server.task.flow.OneInputFlowLifeCycle; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.StringUtils; + import lombok.extern.slf4j.Slf4j; import java.io.IOException; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES; @@ -54,12 +60,16 @@ public class SeaTunnelSourceCollector implements Collector { private final List>> outputs; + private final MetricsContext metricsContext; + private final AtomicBoolean schemaChangeBeforeCheckpointSignal = new AtomicBoolean(false); private final AtomicBoolean schemaChangeAfterCheckpointSignal = new AtomicBoolean(false); private final Counter sourceReceivedCount; + private final Map sourceReceivedCountPerTable = new ConcurrentHashMap<>(); + private final Meter sourceReceivedQPS; private final Counter sourceReceivedBytes; @@ -77,17 +87,24 @@ public SeaTunnelSourceCollector( List>> outputs, MetricsContext metricsContext, FlowControlStrategy flowControlStrategy, - SeaTunnelDataType rowType) { + SeaTunnelDataType rowType, + List tablePaths) { this.checkpointLock = checkpointLock; this.outputs = outputs; this.rowType = rowType; + this.metricsContext = metricsContext; if (rowType instanceof MultipleRowType) { ((MultipleRowType) rowType) .iterator() - .forEachRemaining( - type -> { - this.rowTypeMap.put(type.getKey(), type.getValue()); - }); + .forEachRemaining(type -> this.rowTypeMap.put(type.getKey(), type.getValue())); + } + if (CollectionUtils.isNotEmpty(tablePaths)) { + tablePaths.forEach( + tablePath -> + sourceReceivedCountPerTable.put( + getFullName(tablePath), + metricsContext.counter( + SOURCE_RECEIVED_COUNT + "#" + getFullName(tablePath)))); } sourceReceivedCount = metricsContext.counter(SOURCE_RECEIVED_COUNT); sourceReceivedQPS = metricsContext.meter(SOURCE_RECEIVED_QPS); @@ -100,14 +117,12 @@ public SeaTunnelSourceCollector( public void collect(T row) { try { if (row instanceof SeaTunnelRow) { + String tableId = ((SeaTunnelRow) row).getTableId(); int size; if (rowType instanceof SeaTunnelRowType) { size = ((SeaTunnelRow) row).getBytesSize((SeaTunnelRowType) rowType); } else if (rowType instanceof MultipleRowType) { - size = - ((SeaTunnelRow) row) - .getBytesSize( - rowTypeMap.get(((SeaTunnelRow) row).getTableId())); + size = ((SeaTunnelRow) row).getBytesSize(rowTypeMap.get(tableId)); } else { throw new SeaTunnelEngineException( "Unsupported row type: " + rowType.getClass().getName()); @@ -115,6 +130,18 @@ public void collect(T row) { sourceReceivedBytes.inc(size); sourceReceivedBytesPerSeconds.markEvent(size); flowControlGate.audit((SeaTunnelRow) row); + if (StringUtils.isNotEmpty(tableId)) { + String tableName = getFullName(TablePath.of(tableId)); + Counter sourceTableCounter = sourceReceivedCountPerTable.get(tableName); + if (Objects.nonNull(sourceTableCounter)) { + sourceTableCounter.inc(); + } else { + Counter counter = + metricsContext.counter(SOURCE_RECEIVED_COUNT + "#" + tableName); + counter.inc(); + sourceReceivedCountPerTable.put(tableName, counter); + } + } } sendRecordToNext(new Record<>(row)); emptyThisPollNext = false; @@ -205,4 +232,12 @@ public void sendRecordToNext(Record record) throws IOException { } } } + + private String getFullName(TablePath tablePath) { + if (StringUtils.isBlank(tablePath.getTableName())) { + tablePath = + TablePath.of(tablePath.getDatabaseName(), tablePath.getSchemaName(), "default"); + } + return tablePath.getFullName(); + } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSeaTunnelTask.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSeaTunnelTask.java index 53171d40315..dbcde3e9d6e 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSeaTunnelTask.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSeaTunnelTask.java @@ -22,6 +22,8 @@ import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.core.starter.flowcontrol.FlowControlStrategy; import org.apache.seatunnel.engine.core.dag.actions.SourceAction; @@ -37,9 +39,11 @@ import lombok.Getter; import lombok.NonNull; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; public class SourceSeaTunnelTask extends SeaTunnelTask { @@ -76,10 +80,16 @@ public void init() throws Exception { + startFlowLifeCycle.getClass().getName()); } else { SeaTunnelDataType sourceProducedType; + List tablePaths = new ArrayList<>(); try { List producedCatalogTables = sourceFlow.getAction().getSource().getProducedCatalogTables(); sourceProducedType = CatalogTableUtil.convertToDataType(producedCatalogTables); + tablePaths = + producedCatalogTables.stream() + .map(CatalogTable::getTableId) + .map(TableIdentifier::toTablePath) + .collect(Collectors.toList()); } catch (UnsupportedOperationException e) { // TODO remove it when all connector use `getProducedCatalogTables` sourceProducedType = sourceFlow.getAction().getSource().getProducedType(); @@ -90,7 +100,8 @@ public void init() throws Exception { outputs, this.getMetricsContext(), FlowControlStrategy.fromMap(envOption), - sourceProducedType); + sourceProducedType, + tablePaths); ((SourceFlowLifeCycle) startFlowLifeCycle).setCollector(collector); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java index 48c530a0c36..516e1c97c41 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java @@ -26,6 +26,8 @@ import org.apache.seatunnel.api.sink.SinkCommitter; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportResourceShare; +import org.apache.seatunnel.api.sink.multitablesink.MultiTableSink; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.event.SchemaChangeEvent; import org.apache.seatunnel.api.table.type.Record; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -43,6 +45,8 @@ import org.apache.seatunnel.engine.server.task.operation.sink.SinkRegisterOperation; import org.apache.seatunnel.engine.server.task.record.Barrier; +import org.apache.commons.lang3.StringUtils; + import com.hazelcast.cluster.Address; import lombok.extern.slf4j.Slf4j; @@ -52,9 +56,11 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; @@ -92,6 +98,8 @@ public class SinkFlowLifeCycle sinkWriteCountPerTable = new ConcurrentHashMap<>(); + private Meter sinkWriteQPS; private Counter sinkWriteBytes; @@ -125,6 +133,15 @@ public SinkFlowLifeCycle( sinkWriteQPS = metricsContext.meter(SINK_WRITE_QPS); sinkWriteBytes = metricsContext.counter(SINK_WRITE_BYTES); sinkWriteBytesPerSeconds = metricsContext.meter(SINK_WRITE_BYTES_PER_SECONDS); + if (sinkAction.getSink() instanceof MultiTableSink) { + List sinkTables = ((MultiTableSink) sinkAction.getSink()).getSinkTables(); + sinkTables.forEach( + tablePath -> + sinkWriteCountPerTable.put( + getFullName(tablePath), + metricsContext.counter( + SINK_WRITE_COUNT + "#" + getFullName(tablePath)))); + } } @Override @@ -256,6 +273,19 @@ public void received(Record record) { long size = ((SeaTunnelRow) record.getData()).getBytesSize(); sinkWriteBytes.inc(size); sinkWriteBytesPerSeconds.markEvent(size); + String tableId = ((SeaTunnelRow) record.getData()).getTableId(); + if (StringUtils.isNotBlank(tableId)) { + String tableName = getFullName(TablePath.of(tableId)); + Counter sinkTableCounter = sinkWriteCountPerTable.get(tableName); + if (Objects.nonNull(sinkTableCounter)) { + sinkTableCounter.inc(); + } else { + Counter counter = + metricsContext.counter(SINK_WRITE_COUNT + "#" + tableName); + counter.inc(); + sinkWriteCountPerTable.put(tableName, counter); + } + } } } } catch (Exception e) { @@ -315,4 +345,12 @@ public void restoreState(List actionStateList) throws Except ((SupportResourceShare) this.writer).setMultiTableResourceManager(resourceManager, 0); } } + + private String getFullName(TablePath tablePath) { + if (StringUtils.isBlank(tablePath.getTableName())) { + tablePath = + TablePath.of(tablePath.getDatabaseName(), tablePath.getSchemaName(), "default"); + } + return tablePath.getFullName(); + } } From 31e59cdf82e727a68aab57235eb99dec129cefea Mon Sep 17 00:00:00 2001 From: Xuzz <90698333+Xuzhengz@users.noreply.github.com> Date: Fri, 26 Jul 2024 10:44:50 +0800 Subject: [PATCH 007/361] [Feature][Jdbc] Support hive compatibleMode add inceptor dialect (#7262) --- .../internal/dialect/DatabaseIdentifier.java | 1 + .../dialect/hive/HiveDialectFactory.java | 10 ++ .../dialect/inceptor/InceptorDialect.java | 35 +++++ .../inceptor/InceptorJdbcRowConverter.java | 141 ++++++++++++++++++ .../dialect/hive/HiveDialectFactoryTest.java | 36 +++++ 5 files changed, 223 insertions(+) create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorDialect.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorJdbcRowConverter.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactoryTest.java diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java index 17608392ff1..bf00298a742 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java @@ -40,4 +40,5 @@ public class DatabaseIdentifier { public static final String TIDB = "TiDB"; public static final String XUGU = "XUGU"; public static final String IRIS = "IRIS"; + public static final String INCEPTOR = "Inceptor"; } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java index 56bd81b7f83..3ddf3bfab86 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectFactory; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.inceptor.InceptorDialect; import com.google.auto.service.AutoService; @@ -33,6 +34,15 @@ public boolean acceptsURL(String url) { @Override public JdbcDialect create() { + throw new UnsupportedOperationException( + "Can't create JdbcDialect without compatible mode for Hive"); + } + + @Override + public JdbcDialect create(String compatibleMode, String fieldId) { + if ("inceptor".equals(compatibleMode)) { + return new InceptorDialect(); + } return new HiveDialect(); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorDialect.java new file mode 100644 index 00000000000..9770fb63bdf --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorDialect.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 org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.inceptor; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.JdbcRowConverter; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.hive.HiveDialect; + +public class InceptorDialect extends HiveDialect { + + @Override + public String dialectName() { + return DatabaseIdentifier.INCEPTOR; + } + + @Override + public JdbcRowConverter getRowConverter() { + return new InceptorJdbcRowConverter(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorJdbcRowConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorJdbcRowConverter.java new file mode 100644 index 00000000000..806788b30eb --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/inceptor/InceptorJdbcRowConverter.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.inceptor; + +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.ArrayType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.hive.HiveJdbcRowConverter; + +import org.apache.commons.lang3.StringUtils; + +import java.math.BigDecimal; +import java.sql.PreparedStatement; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; + +public class InceptorJdbcRowConverter extends HiveJdbcRowConverter { + + @Override + public String converterName() { + return DatabaseIdentifier.INCEPTOR; + } + + @Override + public PreparedStatement toExternal( + TableSchema tableSchema, SeaTunnelRow row, PreparedStatement statement) { + SeaTunnelRowType rowType = tableSchema.toPhysicalRowDataType(); + for (int fieldIndex = 0; fieldIndex < rowType.getTotalFields(); fieldIndex++) { + try { + SeaTunnelDataType seaTunnelDataType = rowType.getFieldType(fieldIndex); + int statementIndex = fieldIndex + 1; + Object fieldValue = row.getField(fieldIndex); + if (fieldValue == null) { + statement.setObject(statementIndex, StringUtils.EMPTY); + continue; + } + switch (seaTunnelDataType.getSqlType()) { + case STRING: + statement.setString(statementIndex, (String) row.getField(fieldIndex)); + break; + case BOOLEAN: + statement.setBoolean(statementIndex, (Boolean) row.getField(fieldIndex)); + break; + case TINYINT: + statement.setByte(statementIndex, (Byte) row.getField(fieldIndex)); + break; + case SMALLINT: + statement.setShort(statementIndex, (Short) row.getField(fieldIndex)); + break; + case INT: + statement.setInt(statementIndex, (Integer) row.getField(fieldIndex)); + break; + case BIGINT: + statement.setLong(statementIndex, (Long) row.getField(fieldIndex)); + break; + case FLOAT: + statement.setFloat(statementIndex, (Float) row.getField(fieldIndex)); + break; + case DOUBLE: + statement.setDouble(statementIndex, (Double) row.getField(fieldIndex)); + break; + case DECIMAL: + statement.setBigDecimal( + statementIndex, (BigDecimal) row.getField(fieldIndex)); + break; + case DATE: + LocalDate localDate = (LocalDate) row.getField(fieldIndex); + statement.setDate(statementIndex, java.sql.Date.valueOf(localDate)); + break; + case TIME: + writeTime(statement, statementIndex, (LocalTime) row.getField(fieldIndex)); + break; + case TIMESTAMP: + LocalDateTime localDateTime = (LocalDateTime) row.getField(fieldIndex); + statement.setTimestamp( + statementIndex, java.sql.Timestamp.valueOf(localDateTime)); + break; + case BYTES: + statement.setBytes(statementIndex, (byte[]) row.getField(fieldIndex)); + break; + case NULL: + statement.setNull(statementIndex, java.sql.Types.NULL); + break; + case ARRAY: + SeaTunnelDataType elementType = + ((ArrayType) seaTunnelDataType).getElementType(); + Object[] array = (Object[]) row.getField(fieldIndex); + if (array == null) { + statement.setNull(statementIndex, java.sql.Types.ARRAY); + break; + } + if (SqlType.TINYINT.equals(elementType.getSqlType())) { + Short[] shortArray = new Short[array.length]; + for (int i = 0; i < array.length; i++) { + shortArray[i] = Short.valueOf(array[i].toString()); + } + statement.setObject(statementIndex, shortArray); + } else { + statement.setObject(statementIndex, array); + } + break; + case MAP: + case ROW: + default: + throw new JdbcConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + "Unexpected value: " + seaTunnelDataType); + } + } catch (Exception e) { + throw new JdbcConnectorException( + JdbcConnectorErrorCode.DATA_TYPE_CAST_FAILED, + "error field:" + rowType.getFieldNames()[fieldIndex], + e); + } + } + return statement; + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactoryTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactoryTest.java new file mode 100644 index 00000000000..169f51b6aea --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.hive; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.inceptor.InceptorDialect; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class HiveDialectFactoryTest { + + @Test + public void testWithCompatibleMode() { + HiveDialectFactory hiveDialectFactory = new HiveDialectFactory(); + JdbcDialect inceptorDialect = hiveDialectFactory.create("inceptor", ""); + Assertions.assertTrue(inceptorDialect instanceof InceptorDialect); + JdbcDialect hiveDialect = hiveDialectFactory.create("", ""); + Assertions.assertTrue(hiveDialect instanceof HiveDialect); + } +} From d886495584c48392c2c09d2fb5d3d1780fc33244 Mon Sep 17 00:00:00 2001 From: Guangdong Liu <804167098@qq.com> Date: Fri, 26 Jul 2024 10:45:32 +0800 Subject: [PATCH 008/361] [Fix][Connector-V2] Fix doris TRANSFER_ENCODING header error (#7267) --- .../org/apache/seatunnel/connectors/doris/util/HttpUtil.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/HttpUtil.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/HttpUtil.java index 46d1126c2fe..aa70d1f290a 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/HttpUtil.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/HttpUtil.java @@ -21,6 +21,7 @@ import org.apache.http.impl.client.DefaultRedirectStrategy; import org.apache.http.impl.client.HttpClientBuilder; import org.apache.http.impl.client.HttpClients; +import org.apache.http.protocol.RequestContent; /** util to build http client. */ public class HttpUtil { @@ -32,7 +33,8 @@ public class HttpUtil { protected boolean isRedirectable(String method) { return true; } - }); + }) + .addInterceptorLast(new RequestContent(true));; public CloseableHttpClient getHttpClient() { return httpClientBuilder.build(); From 3ccc6a8bd1e381a23e21eebcd9fca0d87694ba7c Mon Sep 17 00:00:00 2001 From: hailin0 Date: Mon, 29 Jul 2024 10:41:09 +0800 Subject: [PATCH 009/361] [Hotfix][Zeta] Fix task cannot be stopped when system is busy (#7280) --- .../seatunnel/engine/server/TaskExecutionService.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java index 19878545edc..94f0fa324fc 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java @@ -626,7 +626,7 @@ private void updateMetricsContextInImap() { if (localMap.size() > 0) { try { if (!metricsImap.tryLock( - Constant.IMAP_RUNNING_JOB_METRICS_KEY, 2, TimeUnit.SECONDS)) { + Constant.IMAP_RUNNING_JOB_METRICS_KEY, 5, TimeUnit.SECONDS)) { logger.warning("try lock failed in update metrics"); return; } @@ -640,7 +640,11 @@ private void updateMetricsContextInImap() { "The Imap acquisition failed due to the hazelcast node being offline or restarted, and will be retried next time", e); } finally { - metricsImap.unlock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); + try { + metricsImap.unlock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); + } catch (Throwable e) { + logger.warning("unlock imap failed in update metrics", e); + } } } this.printTaskExecutionRuntimeInfo(); From 21c4f5245e8777a9b3a967b6038d00d3f20d9fb8 Mon Sep 17 00:00:00 2001 From: corgy-w <73771213+corgy-w@users.noreply.github.com> Date: Mon, 29 Jul 2024 17:28:10 +0800 Subject: [PATCH 010/361] [Improve][Core] Improve base on plugin name of lookup strategy (#7278) * [bug][plugin-discovery] fix multi plugin discovery * [bug][plugin-discovery] optimize code --------- Co-authored-by: wangchao --- .../discovery/AbstractPluginDiscovery.java | 122 ++++++++++++++++-- .../SeaTunnelSourcePluginDiscoveryTest.java | 29 ++++- .../connectors/plugin-mapping.properties | 8 +- 3 files changed, 143 insertions(+), 16 deletions(-) diff --git a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java index 175ba435ed6..d4bd43c3d1c 100644 --- a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java +++ b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java @@ -54,11 +54,13 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.ServiceLoader; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.function.BiConsumer; import java.util.stream.Collectors; @@ -430,17 +432,16 @@ public boolean accept(File pathname) { if (ArrayUtils.isEmpty(targetPluginFiles)) { return Optional.empty(); } - if (targetPluginFiles.length > 1) { - throw new IllegalArgumentException( - "Found multiple plugin jar: " - + Arrays.stream(targetPluginFiles) - .map(File::getPath) - .collect(Collectors.joining(",")) - + " for pluginIdentifier: " - + pluginIdentifier); - } try { - URL pluginJarPath = targetPluginFiles[0].toURI().toURL(); + URL pluginJarPath; + if (targetPluginFiles.length == 1) { + pluginJarPath = targetPluginFiles[0].toURI().toURL(); + } else { + pluginJarPath = + findMostSimlarPluginJarFile(targetPluginFiles, pluginJarPrefix) + .toURI() + .toURL(); + } log.info("Discovery plugin jar for: {} at: {}", pluginIdentifier, pluginJarPath); return Optional.of(pluginJarPath); } catch (MalformedURLException e) { @@ -451,4 +452,105 @@ public boolean accept(File pathname) { return Optional.empty(); } } + + private static File findMostSimlarPluginJarFile( + File[] targetPluginFiles, String pluginJarPrefix) { + String splitRegex = "\\-|\\_|\\."; + double maxSimlarity = -Integer.MAX_VALUE; + int mostSimlarPluginJarFileIndex = -1; + for (int i = 0; i < targetPluginFiles.length; i++) { + File file = targetPluginFiles[i]; + String fileName = file.getName(); + double similarity = + CosineSimilarityUtil.cosineSimilarity(pluginJarPrefix, fileName, splitRegex); + if (similarity > maxSimlarity) { + maxSimlarity = similarity; + mostSimlarPluginJarFileIndex = i; + } + } + return targetPluginFiles[mostSimlarPluginJarFileIndex]; + } + + static class CosineSimilarityUtil { + public static double cosineSimilarity(String textA, String textB, String splitRegrex) { + Set words1 = + new HashSet<>(Arrays.asList(textA.toLowerCase().split(splitRegrex))); + Set words2 = + new HashSet<>(Arrays.asList(textB.toLowerCase().split(splitRegrex))); + int[] termFrequency1 = calculateTermFrequencyVector(textA, words1, splitRegrex); + int[] termFrequency2 = calculateTermFrequencyVector(textB, words2, splitRegrex); + return calculateCosineSimilarity(termFrequency1, termFrequency2); + } + + private static int[] calculateTermFrequencyVector( + String text, Set words, String splitRegrex) { + int[] termFrequencyVector = new int[words.size()]; + String[] textArray = text.toLowerCase().split(splitRegrex); + List orderedWords = new ArrayList(); + words.clear(); + for (String word : textArray) { + if (!words.contains(word)) { + orderedWords.add(word); + words.add(word); + } + } + for (String word : textArray) { + if (words.contains(word)) { + int index = 0; + for (String w : orderedWords) { + if (w.equals(word)) { + termFrequencyVector[index]++; + break; + } + index++; + } + } + } + return termFrequencyVector; + } + + private static double calculateCosineSimilarity(int[] vectorA, int[] vectorB) { + double dotProduct = 0.0; + double magnitudeA = 0.0; + double magnitudeB = 0.0; + int vectorALength = vectorA.length; + int vectorBLength = vectorB.length; + if (vectorALength < vectorBLength) { + int[] vectorTemp = new int[vectorBLength]; + for (int i = 0; i < vectorB.length; i++) { + if (i <= vectorALength - 1) { + vectorTemp[i] = vectorA[i]; + } else { + vectorTemp[i] = 0; + } + } + vectorA = vectorTemp; + } + if (vectorALength > vectorBLength) { + int[] vectorTemp = new int[vectorALength]; + for (int i = 0; i < vectorA.length; i++) { + if (i <= vectorBLength - 1) { + vectorTemp[i] = vectorB[i]; + } else { + vectorTemp[i] = 0; + } + } + vectorB = vectorTemp; + } + for (int i = 0; i < vectorA.length; i++) { + dotProduct += vectorA[i] * vectorB[i]; + magnitudeA += Math.pow(vectorA[i], 2); + magnitudeB += Math.pow(vectorB[i], 2); + } + + magnitudeA = Math.sqrt(magnitudeA); + magnitudeB = Math.sqrt(magnitudeB); + + if (magnitudeA == 0 || magnitudeB == 0) { + return 0.0; // Avoid dividing by 0 + } else { + return dotProduct / (magnitudeA * magnitudeB); + } + } + } } diff --git a/seatunnel-plugin-discovery/src/test/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSourcePluginDiscoveryTest.java b/seatunnel-plugin-discovery/src/test/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSourcePluginDiscoveryTest.java index 81333d4b4df..88fd76d73be 100644 --- a/seatunnel-plugin-discovery/src/test/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSourcePluginDiscoveryTest.java +++ b/seatunnel-plugin-discovery/src/test/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSourcePluginDiscoveryTest.java @@ -32,10 +32,13 @@ import com.google.common.collect.Lists; import java.io.IOException; +import java.net.URL; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; @DisabledOnOs(OS.WINDOWS) class SeaTunnelSourcePluginDiscoveryTest { @@ -47,7 +50,10 @@ class SeaTunnelSourcePluginDiscoveryTest { private static final List pluginJars = Lists.newArrayList( Paths.get(seatunnelHome, "connectors", "connector-http-jira.jar"), - Paths.get(seatunnelHome, "connectors", "connector-http.jar")); + Paths.get(seatunnelHome, "connectors", "connector-http.jar"), + Paths.get(seatunnelHome, "connectors", "connector-kafka.jar"), + Paths.get(seatunnelHome, "connectors", "connector-kafka-alcs.jar"), + Paths.get(seatunnelHome, "connectors", "connector-kafka-blcs.jar")); @BeforeEach public void before() throws IOException { @@ -67,12 +73,25 @@ void getPluginBaseClass() { List pluginIdentifiers = Lists.newArrayList( PluginIdentifier.of("seatunnel", PluginType.SOURCE.getType(), "HttpJira"), - PluginIdentifier.of("seatunnel", PluginType.SOURCE.getType(), "HttpBase")); + PluginIdentifier.of("seatunnel", PluginType.SOURCE.getType(), "HttpBase"), + PluginIdentifier.of("seatunnel", PluginType.SOURCE.getType(), "Kafka"), + PluginIdentifier.of("seatunnel", PluginType.SINK.getType(), "Kafka-Blcs")); SeaTunnelSourcePluginDiscovery seaTunnelSourcePluginDiscovery = new SeaTunnelSourcePluginDiscovery(); - Assertions.assertThrows( - IllegalArgumentException.class, - () -> seaTunnelSourcePluginDiscovery.getPluginJarPaths(pluginIdentifiers)); + Assertions.assertIterableEquals( + Stream.of( + Paths.get(seatunnelHome, "connectors", "connector-http-jira.jar") + .toString(), + Paths.get(seatunnelHome, "connectors", "connector-http.jar") + .toString(), + Paths.get(seatunnelHome, "connectors", "connector-kafka.jar") + .toString(), + Paths.get(seatunnelHome, "connectors", "connector-kafka-blcs.jar") + .toString()) + .collect(Collectors.toList()), + seaTunnelSourcePluginDiscovery.getPluginJarPaths(pluginIdentifiers).stream() + .map(URL::getPath) + .collect(Collectors.toList())); } @AfterEach diff --git a/seatunnel-plugin-discovery/src/test/resources/duplicate/connectors/plugin-mapping.properties b/seatunnel-plugin-discovery/src/test/resources/duplicate/connectors/plugin-mapping.properties index be38939a7f0..ea20ad05b0f 100644 --- a/seatunnel-plugin-discovery/src/test/resources/duplicate/connectors/plugin-mapping.properties +++ b/seatunnel-plugin-discovery/src/test/resources/duplicate/connectors/plugin-mapping.properties @@ -18,4 +18,10 @@ seatunnel.source.HttpBase = connector-http seatunnel.sink.HttpBase = connector-http seatunnel.source.HttpJira = connector-http-jira -seatunnel.sink.HttpJira = connector-http-jira \ No newline at end of file +seatunnel.sink.HttpJira = connector-http-jira +seatunnel.source.Kafka = connector-kafka +seatunnel.sink.Kafka = connector-kafka +seatunnel.source.Kafka-Alcs = connector-kafka-alcs +seatunnel.sink.Kafka-Alcs = connector-kafka-alcs +seatunnel.source.Kafka-Blcs = connector-kafka-blcs +seatunnel.sink.Kafka-Blcs = connector-kafka-blcs \ No newline at end of file From 48974917082ccc08f196e892697f4c518a8b0ebf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=8D=A2=E5=AE=97=E6=9F=B1?= Date: Mon, 29 Jul 2024 17:29:23 +0800 Subject: [PATCH 011/361] fix [Bug] Unable to create a source for identifier 'Iceberg'. #7182 (#7279) Co-authored-by: luzongzhu --- seatunnel-connectors-v2/connector-doris/pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/seatunnel-connectors-v2/connector-doris/pom.xml b/seatunnel-connectors-v2/connector-doris/pom.xml index fbaf9e1128b..85aafc97ad8 100644 --- a/seatunnel-connectors-v2/connector-doris/pom.xml +++ b/seatunnel-connectors-v2/connector-doris/pom.xml @@ -75,6 +75,12 @@ seatunnel-thrift-service ${project.version} optional + + + org.apache.thrift + libthrift + + org.apache.seatunnel From f012b2a6f093bcad49fbd59946e75ce6d6a97838 Mon Sep 17 00:00:00 2001 From: dailai Date: Mon, 29 Jul 2024 21:51:59 +0800 Subject: [PATCH 012/361] [Improve][Connector-v2] Optimize the way of databases and tables are checked for existence (#7261) --- .../common/exception/CommonError.java | 8 +++ .../common/exception/CommonErrorCode.java | 1 + .../jdbc/catalog/AbstractJdbcCatalog.java | 70 +++++++++++++++++-- .../jdbc/catalog/dm/DamengCatalog.java | 30 ++++---- .../jdbc/catalog/iris/IrisCatalog.java | 29 ++++---- .../jdbc/catalog/mysql/MySqlCatalog.java | 17 +++++ .../oceanbase/OceanBaseOracleCatalog.java | 33 +++++---- .../jdbc/catalog/oracle/OracleCatalog.java | 39 ++++++----- .../jdbc/catalog/psql/PostgresCatalog.java | 35 +++++----- .../catalog/redshift/RedshiftCatalog.java | 40 +++++------ .../jdbc/catalog/saphana/SapHanaCatalog.java | 30 ++++---- .../catalog/sqlserver/SqlServerCatalog.java | 29 ++++---- .../jdbc/catalog/xugu/XuguCatalog.java | 32 ++++----- .../jdbc/catalog/mysql/MySqlCatalogTest.java | 15 +++- .../catalog/oracle/OracleCatalogTest.java | 33 +++++++-- .../catalog/psql/PostgresCatalogTest.java | 32 +++++++-- .../sqlserver/SqlServerCatalogTest.java | 14 ++-- ...l_source_and_sink_with_multiple_tables.sql | 4 +- 18 files changed, 309 insertions(+), 182 deletions(-) diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonError.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonError.java index 782a071d011..e9adf4d70a1 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonError.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonError.java @@ -285,4 +285,12 @@ public static SeaTunnelRuntimeException formatDateError(String date, String fiel params.put("field", field); return new SeaTunnelRuntimeException(CommonErrorCode.FORMAT_DATE_ERROR, params); } + + public static SeaTunnelRuntimeException unsupportedMethod( + String identifier, String methodName) { + Map params = new HashMap<>(); + params.put("identifier", identifier); + params.put("methodName", methodName); + return new SeaTunnelRuntimeException(CommonErrorCode.UNSUPPORTED_METHOD, params); + } } diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java index 58939248482..79621c42168 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java @@ -77,6 +77,7 @@ public enum CommonErrorCode implements SeaTunnelErrorCode { FORMAT_DATETIME_ERROR( "COMMON-33", "The datetime format '' of field '' is not supported. Please check the datetime format."), + UNSUPPORTED_METHOD("COMMON-34", "'' unsupported the method ''"), ; private final String code; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java index a033d0eaac7..8d0301b492e 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java @@ -44,6 +44,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import lombok.extern.slf4j.Slf4j; + import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.DriverManager; @@ -63,11 +65,14 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.seatunnel.common.exception.CommonErrorCode.UNSUPPORTED_METHOD; +@Slf4j public abstract class AbstractJdbcCatalog implements Catalog { private static final Logger LOG = LoggerFactory.getLogger(AbstractJdbcCatalog.class); protected static final Set SYS_DATABASES = new HashSet<>(); + protected static final Set EXCLUDED_SCHEMAS = new HashSet<>(); protected final String catalogName; protected final String defaultDatabase; @@ -259,6 +264,10 @@ protected String getListDatabaseSql() { throw new UnsupportedOperationException(); } + protected String getDatabaseWithConditionSql(String databaseName) { + throw CommonError.unsupportedMethod(this.catalogName, "getDatabaseWithConditionSql"); + } + @Override public List listDatabases() throws CatalogException { try { @@ -277,15 +286,35 @@ public List listDatabases() throws CatalogException { @Override public boolean databaseExists(String databaseName) throws CatalogException { - checkArgument(StringUtils.isNotBlank(databaseName)); - - return listDatabases().contains(databaseName); + if (StringUtils.isBlank(databaseName)) { + return false; + } + if (SYS_DATABASES.contains(databaseName)) { + return false; + } + try { + return querySQLResultExists( + getUrlFromDatabaseName(databaseName), + getDatabaseWithConditionSql(databaseName)); + } catch (SeaTunnelRuntimeException e) { + if (e.getSeaTunnelErrorCode().getCode().equals(UNSUPPORTED_METHOD.getCode())) { + log.warn( + "The catalog: {} is not supported the getDatabaseWithConditionSql for databaseExists", + this.catalogName); + return listDatabases().contains(databaseName); + } + throw e; + } } protected String getListTableSql(String databaseName) { throw new UnsupportedOperationException(); } + protected String getTableWithConditionSql(TablePath tablePath) { + throw CommonError.unsupportedMethod(this.catalogName, "getTableWithConditionSql"); + } + protected String getTableName(ResultSet rs) throws SQLException { String schemaName = rs.getString(1); String tableName = rs.getString(2); @@ -317,12 +346,28 @@ public List listTables(String databaseName) @Override public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - return databaseExists(tablePath.getDatabaseName()) - && listTables(tablePath.getDatabaseName()).contains(getTableName(tablePath)); - } catch (DatabaseNotExistException e) { + String databaseName = tablePath.getDatabaseName(); + if (EXCLUDED_SCHEMAS.contains(tablePath.getSchemaName())) { return false; } + try { + return querySQLResultExists( + this.getUrlFromDatabaseName(databaseName), getTableWithConditionSql(tablePath)); + } catch (SeaTunnelRuntimeException e1) { + if (e1.getSeaTunnelErrorCode().getCode().equals(UNSUPPORTED_METHOD.getCode())) { + log.warn( + "The catalog: {} is not supported the getTableWithConditionSql for tableExists ", + this.catalogName); + try { + return databaseExists(tablePath.getDatabaseName()) + && listTables(tablePath.getDatabaseName()) + .contains(getTableName(tablePath)); + } catch (DatabaseNotExistException e2) { + return false; + } + } + throw e1; + } } @Override @@ -528,6 +573,17 @@ protected List queryString(String url, String sql, ResultSetConsumer listTables() { List databases = listDatabases(); return listTables(databases.get(0)); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCatalog.java index 40f08dc50b5..02e58ea8573 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCatalog.java @@ -57,7 +57,7 @@ public class IrisCatalog extends AbstractJdbcCatalog { private static final String LIST_TABLES_SQL_TEMPLATE = - "SELECT TABLE_SCHEMA,TABLE_NAME FROM INFORMATION_SCHEMA.Tables WHERE TABLE_SCHEMA='%s' and TABLE_TYPE != 'SYSTEM TABLE' and TABLE_TYPE != 'SYSTEM VIEW';"; + "SELECT TABLE_SCHEMA,TABLE_NAME FROM INFORMATION_SCHEMA.Tables WHERE TABLE_SCHEMA='%s' and TABLE_TYPE != 'SYSTEM TABLE' and TABLE_TYPE != 'SYSTEM VIEW'"; public IrisCatalog( String catalogName, String username, String password, JdbcUrlUtil.UrlInfo urlInfo) { @@ -101,13 +101,6 @@ protected String getTableName(ResultSet rs) throws SQLException { return schemaName + "." + tableName; } - // @Override - // protected String getSelectColumnsSql(TablePath tablePath) { - // return String.format( - // SELECT_COLUMNS_SQL_TEMPLATE, tablePath.getSchemaName(), - // tablePath.getTableName()); - // } - @Override protected Column buildColumn(ResultSet resultSet) throws SQLException { String columnName = resultSet.getString("COLUMN_NAME"); @@ -144,12 +137,24 @@ public boolean databaseExists(String databaseName) throws CatalogException { @Override public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - return listTables(tablePath.getSchemaName()) - .contains(tablePath.getSchemaAndTableName()); - } catch (DatabaseNotExistException e) { + if (EXCLUDED_SCHEMAS.contains(tablePath.getSchemaName())) { return false; } + return querySQLResultExists( + this.getUrlFromDatabaseName(tablePath.getDatabaseName()), + getTableWithConditionSql(tablePath)); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return String.format( + getListTableSql(tablePath.getSchemaName()) + " and TABLE_NAME = '%s'", + tablePath.getTableName()); + } + + @Override + protected String getUrlFromDatabaseName(String databaseName) { + return defaultUrl; } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java index 6b263b0fd46..e2df8ab24b9 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java @@ -51,6 +51,12 @@ public class MySqlCatalog extends AbstractJdbcCatalog { private static final String SELECT_COLUMNS_SQL_TEMPLATE = "SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_SCHEMA = '%s' AND TABLE_NAME ='%s' ORDER BY ORDINAL_POSITION ASC"; + private static final String SELECT_DATABASE_EXISTS = + "SELECT SCHEMA_NAME FROM information_schema.schemata WHERE SCHEMA_NAME = '%s'"; + + private static final String SELECT_TABLE_EXISTS = + "SELECT TABLE_SCHEMA,TABLE_NAME FROM information_schema.tables WHERE table_schema = '%s' AND table_name = '%s'"; + static { SYS_DATABASES.add("information_schema"); SYS_DATABASES.add("mysql"); @@ -68,6 +74,17 @@ public MySqlCatalog( this.typeConverter = new MySqlTypeConverter(version); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + return String.format(SELECT_DATABASE_EXISTS, databaseName); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return String.format( + SELECT_TABLE_EXISTS, tablePath.getDatabaseName(), tablePath.getTableName()); + } + @Override protected String getListDatabaseSql() { return "SHOW DATABASES;"; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseOracleCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseOracleCatalog.java index b4ece7db9c2..b98f4c4c2b2 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseOracleCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseOracleCatalog.java @@ -25,8 +25,6 @@ import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleCatalog; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkNotNull; @@ -34,9 +32,10 @@ public class OceanBaseOracleCatalog extends OracleCatalog { static { - EXCLUDED_SCHEMAS = - Collections.unmodifiableList( - Arrays.asList("oceanbase", "LBACSYS", "ORAAUDITOR", "SYS")); + EXCLUDED_SCHEMAS.add("oceanbase"); + EXCLUDED_SCHEMAS.add("LBACSYS"); + EXCLUDED_SCHEMAS.add("ORAAUDITOR"); + EXCLUDED_SCHEMAS.add("SYS"); } public OceanBaseOracleCatalog( @@ -53,6 +52,21 @@ protected String getListDatabaseSql() { throw new UnsupportedOperationException(); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean tableExists(TablePath tablePath) throws CatalogException { + if (EXCLUDED_SCHEMAS.contains(tablePath.getSchemaName())) { + return false; + } + return querySQLResultExists( + this.getUrlFromDatabaseName(tablePath.getDatabaseName()), + getTableWithConditionSql(tablePath)); + } + @Override public List listTables(String databaseName) throws CatalogException, DatabaseNotExistException { @@ -65,15 +79,6 @@ public List listTables(String databaseName) } } - @Override - public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - return listTables(tablePath.getDatabaseName()).contains(getTableName(tablePath)); - } catch (DatabaseNotExistException e) { - return false; - } - } - @Override public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreIfExists) throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java index b51369e3f58..1430cb387af 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java @@ -21,8 +21,6 @@ import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; import org.apache.seatunnel.api.table.catalog.TablePath; -import org.apache.seatunnel.api.table.catalog.exception.CatalogException; -import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; @@ -30,8 +28,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oracle.OracleTypeConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oracle.OracleTypeMapper; -import org.apache.commons.lang3.StringUtils; - import lombok.extern.slf4j.Slf4j; import java.sql.Connection; @@ -46,7 +42,7 @@ @Slf4j public class OracleCatalog extends AbstractJdbcCatalog { - protected static List EXCLUDED_SCHEMAS = + protected static List EXCLUDED_SCHEMAS_ALL = Collections.unmodifiableList( Arrays.asList( "APPQOSSYS", @@ -101,6 +97,10 @@ public class OracleCatalog extends AbstractJdbcCatalog { + "ORDER BY \n" + " cols.column_id \n"; + static { + EXCLUDED_SCHEMAS.addAll(EXCLUDED_SCHEMAS_ALL); + } + public OracleCatalog( String catalogName, String username, @@ -110,6 +110,21 @@ public OracleCatalog( super(catalogName, username, pwd, urlInfo, defaultSchema); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + return String.format(getListDatabaseSql() + " where name = '%s'", databaseName); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return getListTableSql(tablePath.getDatabaseName()) + + " and OWNER = '" + + tablePath.getSchemaName() + + "' and table_name = '" + + tablePath.getTableName() + + "'"; + } + @Override protected String getListDatabaseSql() { return "SELECT name FROM v$database"; @@ -191,20 +206,6 @@ protected String getOptionTableName(TablePath tablePath) { return tablePath.getSchemaAndTableName(); } - @Override - public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - if (StringUtils.isNotBlank(tablePath.getDatabaseName())) { - return databaseExists(tablePath.getDatabaseName()) - && listTables(tablePath.getDatabaseName()) - .contains(tablePath.getSchemaAndTableName()); - } - return listTables().contains(tablePath.getSchemaAndTableName()); - } catch (DatabaseNotExistException e) { - return false; - } - } - private List listTables() { List databases = listDatabases(); return listTables(databases.get(0)); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java index 4697d1999ef..d5261e16d59 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java @@ -21,7 +21,6 @@ import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.exception.CatalogException; -import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; @@ -30,7 +29,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.psql.PostgresTypeMapper; import org.apache.commons.collections4.CollectionUtils; -import org.apache.commons.lang3.StringUtils; import lombok.extern.slf4j.Slf4j; @@ -104,14 +102,28 @@ public PostgresCatalog( super(catalogName, username, pwd, urlInfo, defaultSchema); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + return String.format(getListDatabaseSql() + " where datname = '%s'", databaseName); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return String.format( + getListTableSql(tablePath.getDatabaseName()) + + " where table_schema = '%s' and table_name= '%s'", + tablePath.getSchemaName(), + tablePath.getTableName()); + } + @Override protected String getListDatabaseSql() { - return "select datname from pg_database;"; + return "select datname from pg_database"; } @Override protected String getListTableSql(String databaseName) { - return "SELECT table_schema, table_name FROM information_schema.tables;"; + return "SELECT table_schema, table_name FROM information_schema.tables"; } @Override @@ -231,21 +243,6 @@ protected void dropDatabaseInternal(String databaseName) throws CatalogException super.dropDatabaseInternal(databaseName); } - @Override - public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - if (StringUtils.isNotBlank(tablePath.getDatabaseName())) { - return databaseExists(tablePath.getDatabaseName()) - && listTables(tablePath.getDatabaseName()) - .contains(tablePath.getSchemaAndTableName()); - } - - return listTables(defaultDatabase).contains(tablePath.getSchemaAndTableName()); - } catch (DatabaseNotExistException e) { - return false; - } - } - @Override public CatalogTable getTable(String sqlQuery) throws SQLException { Connection defaultConnection = getConnection(defaultUrl); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalog.java index 7b29bbb8ea6..064b2473371 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalog.java @@ -23,7 +23,6 @@ import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.exception.CatalogException; -import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; @@ -31,23 +30,17 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.redshift.RedshiftTypeConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.redshift.RedshiftTypeMapper; -import org.apache.commons.lang3.StringUtils; - import lombok.extern.slf4j.Slf4j; import java.sql.Connection; import java.sql.ResultSet; import java.sql.SQLException; -import java.util.HashSet; import java.util.Map; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @Slf4j public class RedshiftCatalog extends AbstractJdbcCatalog { - protected static final Set EXCLUDED_SCHEMAS = new HashSet<>(4); - private final String SELECT_COLUMNS = "SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_SCHEMA = '%s' AND TABLE_NAME ='%s' ORDER BY ordinal_position ASC"; @@ -80,6 +73,20 @@ public RedshiftCatalog( this.connectionMap = new ConcurrentHashMap<>(); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + return String.format(getListDatabaseSql() + " where datname = '%s'", databaseName); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return String.format( + getListTableSql(tablePath.getDatabaseName()) + + " where table_schema = '%s' and table_name = '%s'", + tablePath.getSchemaName(), + tablePath.getTableName()); + } + @Override public void close() throws CatalogException { for (Map.Entry entry : connectionMap.entrySet()) { @@ -95,12 +102,12 @@ public void close() throws CatalogException { @Override protected String getListDatabaseSql() { - return "select datname from pg_database;"; + return "select datname from pg_database"; } @Override protected String getListTableSql(String databaseName) { - return "SELECT table_schema, table_name FROM information_schema.tables;"; + return "SELECT table_schema, table_name FROM information_schema.tables"; } @Override @@ -144,21 +151,6 @@ protected String getDropDatabaseSql(String databaseName) { return String.format("DROP DATABASE `%s`;", databaseName); } - @Override - public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - if (StringUtils.isNotBlank(tablePath.getDatabaseName())) { - return databaseExists(tablePath.getDatabaseName()) - && listTables(tablePath.getDatabaseName()) - .contains(tablePath.getSchemaAndTableName().toLowerCase()); - } - return listTables(defaultDatabase) - .contains(tablePath.getSchemaAndTableName().toLowerCase()); - } catch (DatabaseNotExistException e) { - return false; - } - } - @Override protected String getSelectColumnsSql(TablePath tablePath) { return String.format(SELECT_COLUMNS, tablePath.getSchemaName(), tablePath.getTableName()); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java index df6f4b3c248..19b8f668af9 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java @@ -22,8 +22,6 @@ import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; import org.apache.seatunnel.api.table.catalog.TablePath; -import org.apache.seatunnel.api.table.catalog.exception.CatalogException; -import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; @@ -31,8 +29,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.saphana.SapHanaTypeConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.saphana.SapHanaTypeMapper; -import org.apache.commons.lang3.StringUtils; - import lombok.extern.slf4j.Slf4j; import java.sql.Connection; @@ -113,6 +109,18 @@ public SapHanaCatalog( super(catalogName, username, pwd, urlInfo, defaultSchema); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + return String.format(getListDatabaseSql() + " where SCHEMA_NAME = '%s'", databaseName); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return String.format( + getListTableSql(tablePath.getDatabaseName()) + " and TABLE_NAME = '%s'", + tablePath.getTableName()); + } + @Override protected String getListDatabaseSql() { return "SELECT SCHEMA_NAME FROM SCHEMAS"; @@ -203,20 +211,6 @@ protected String getOptionTableName(TablePath tablePath) { return tablePath.getTableName(); } - @Override - public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - if (StringUtils.isNotBlank(tablePath.getDatabaseName())) { - return databaseExists(tablePath.getDatabaseName()) - && listTables(tablePath.getDatabaseName()) - .contains(tablePath.getTableName()); - } - return listTables().contains(tablePath.getSchemaAndTableName()); - } catch (DatabaseNotExistException e) { - return false; - } - } - private List listTables() { List databases = listDatabases(); return listTables(databases.get(0)); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java index 55660b36a2c..e4c63515220 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java @@ -22,7 +22,6 @@ import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.exception.CatalogException; -import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; @@ -69,6 +68,20 @@ public SqlServerCatalog( super(catalogName, username, pwd, urlInfo, defaultSchema); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + return String.format(getListDatabaseSql() + " where name = '%s'", databaseName); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return String.format( + getListTableSql(tablePath.getDatabaseName()) + + " and TABLE_SCHEMA = '%s' and TABLE_NAME = '%s'", + tablePath.getSchemaName(), + tablePath.getTableName()); + } + @Override protected String getListDatabaseSql() { return "SELECT NAME FROM sys.databases"; @@ -147,20 +160,6 @@ protected String getUrlFromDatabaseName(String databaseName) { return baseUrl + ";databaseName=" + databaseName + ";" + suffix; } - @Override - public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - if (StringUtils.isNotBlank(tablePath.getDatabaseName())) { - return databaseExists(tablePath.getDatabaseName()) - && listTables(tablePath.getDatabaseName()) - .contains(tablePath.getSchemaAndTableName()); - } - return listTables(defaultDatabase).contains(tablePath.getSchemaAndTableName()); - } catch (DatabaseNotExistException e) { - return false; - } - } - @Override public CatalogTable getTable(String sqlQuery) throws SQLException { Connection defaultConnection = getConnection(defaultUrl); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java index 462e109c76a..a0b28e49abd 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java @@ -21,8 +21,6 @@ import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; import org.apache.seatunnel.api.table.catalog.TablePath; -import org.apache.seatunnel.api.table.catalog.exception.CatalogException; -import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; @@ -30,8 +28,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu.XuguTypeConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu.XuguTypeMapper; -import org.apache.commons.lang3.StringUtils; - import lombok.extern.slf4j.Slf4j; import java.sql.Connection; @@ -128,6 +124,20 @@ public XuguCatalog( super(catalogName, username, pwd, urlInfo, defaultSchema); } + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + return String.format(getListDatabaseSql() + " where DB_NAME = '%s'", databaseName); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return String.format( + getListTableSql(tablePath.getDatabaseName()) + + " where user_name = '%s' and table_name = '%s'", + tablePath.getSchemaName(), + tablePath.getTableName()); + } + @Override protected String getListDatabaseSql() { return "SELECT DB_NAME FROM dba_databases"; @@ -210,20 +220,6 @@ protected String getOptionTableName(TablePath tablePath) { return tablePath.getSchemaAndTableName(); } - @Override - public boolean tableExists(TablePath tablePath) throws CatalogException { - try { - if (StringUtils.isNotBlank(tablePath.getDatabaseName())) { - return databaseExists(tablePath.getDatabaseName()) - && listTables(tablePath.getDatabaseName()) - .contains(tablePath.getSchemaAndTableName()); - } - return listTables().contains(tablePath.getSchemaAndTableName()); - } catch (DatabaseNotExistException e) { - return false; - } - } - private List listTables() { List databases = listDatabases(); return listTables(databases.get(0)); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogTest.java index daf87b3693a..bc89d4c8c39 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogTest.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.sqlserver.SqlServerURLParser; import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.MethodOrderer; @@ -39,7 +40,8 @@ class MySqlCatalogTest { static JdbcUrlUtil.UrlInfo sqlParse = SqlServerURLParser.parse("jdbc:sqlserver://127.0.0.1:1434;database=TestDB"); static JdbcUrlUtil.UrlInfo MysqlUrlInfo = - JdbcUrlUtil.getUrlInfo("jdbc:mysql://127.0.0.1:33061/liuliTest?useSSL=false"); + JdbcUrlUtil.getUrlInfo( + "jdbc:mysql://127.0.0.1:3306/test?useSSL=false&allowPublicKeyRetrieval=true"); static JdbcUrlUtil.UrlInfo pg = JdbcUrlUtil.getUrlInfo("jdbc:postgresql://127.0.0.1:5432/liulitest"); static TablePath tablePathSQL; @@ -74,13 +76,22 @@ static void before() { tablePathPG = TablePath.of(databaseName, "pg_to_mysql"); tablePathOracle = TablePath.of(databaseName, "oracle_to_mysql"); sqlServerCatalog = new SqlServerCatalog("sqlserver", "sa", "root@123", sqlParse, null); - mySqlCatalog = new MySqlCatalog("mysql", "root", "root@123", MysqlUrlInfo); + mySqlCatalog = new MySqlCatalog("mysql", "root", "123456", MysqlUrlInfo); postgresCatalog = new PostgresCatalog("postgres", "postgres", "postgres", pg, null); mySqlCatalog.open(); sqlServerCatalog.open(); postgresCatalog.open(); } + @Test + void exists() { + Assertions.assertTrue(mySqlCatalog.databaseExists("test")); + Assertions.assertTrue(mySqlCatalog.tableExists(TablePath.of("test", "MY_TABLE"))); + Assertions.assertTrue(mySqlCatalog.tableExists(TablePath.of("test", "my_table"))); + Assertions.assertFalse(mySqlCatalog.tableExists(TablePath.of("test", "test"))); + Assertions.assertFalse(mySqlCatalog.databaseExists("mysql")); + } + @Test @Order(1) void getTable() { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java index 1c5fb5a2b22..75b22ec24dc 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java @@ -20,6 +20,8 @@ import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TablePath; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; @@ -27,17 +29,24 @@ @Disabled("Please Test it in your local environment") class OracleCatalogTest { - @Test - void testCatalog() { - OracleCatalog catalog = + + static OracleCatalog catalog; + + @BeforeAll + static void before() { + catalog = new OracleCatalog( "oracle", - "test", - "oracle", - OracleURLParser.parse("jdbc:oracle:thin:@127.0.0.1:1521:xe"), + "c##gguser", + "testdb", + OracleURLParser.parse("jdbc:oracle:thin:@127.0.0.1:1521/CDC_PDB"), null); catalog.open(); + } + + @Test + void testCatalog() { List strings = catalog.listDatabases(); @@ -45,4 +54,16 @@ void testCatalog() { catalog.createTable(new TablePath("XE", "TEST", "TEST003"), table, false); } + + @Test + void exist() { + Assertions.assertTrue(catalog.databaseExists("ORCLCDB")); + Assertions.assertTrue(catalog.tableExists(TablePath.of("ORCLCDB", "C##GGUSER", "myTable"))); + Assertions.assertFalse(catalog.databaseExists("ORCL")); + Assertions.assertTrue( + catalog.tableExists( + TablePath.of("ORCLCDB", "CDC_PDB", "ads_index_public_health_data"))); + Assertions.assertTrue( + catalog.tableExists(TablePath.of("ORCLCDB", "CDC_PDB", "ADS_INDEX_DISEASE_DATA"))); + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogTest.java index c04c1941b0b..05a013ef691 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogTest.java @@ -22,6 +22,8 @@ import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.mysql.MySqlCatalog; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; @@ -31,15 +33,23 @@ @Slf4j class PostgresCatalogTest { - @Test - void testCatalog() { - JdbcUrlUtil.UrlInfo urlInfo = - JdbcUrlUtil.getUrlInfo("jdbc:postgresql://127.0.0.1:5432/liulitest"); - PostgresCatalog catalog = - new PostgresCatalog("postgres", "postgres", "postgres", urlInfo, null); + static PostgresCatalog catalog; + + @BeforeAll + static void before() { + catalog = + new PostgresCatalog( + "postgres", + "pg", + "pg#2024", + JdbcUrlUtil.getUrlInfo("jdbc:postgresql://127.0.0.1:5432/postgres"), + null); catalog.open(); + } + @Test + void testCatalog() { MySqlCatalog mySqlCatalog = new MySqlCatalog( "mysql", @@ -59,4 +69,14 @@ void testCatalog() { catalog.createTable( new TablePath("liulitest", "public", "all_types_table_02"), table, false); } + + @Test + void exists() { + Assertions.assertFalse(catalog.databaseExists("postgres")); + Assertions.assertFalse( + catalog.tableExists(TablePath.of("postgres", "pg_catalog", "pg_aggregate"))); + Assertions.assertTrue(catalog.databaseExists("zdykdb")); + Assertions.assertTrue( + catalog.tableExists(TablePath.of("zdykdb", "pg_catalog", "pg_class"))); + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogTest.java index ea305ca0c1f..a18cc4abd9d 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogTest.java @@ -24,6 +24,7 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresCatalog; import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.MethodOrderer; @@ -38,7 +39,7 @@ class SqlServerCatalogTest { static JdbcUrlUtil.UrlInfo sqlParse = - SqlServerURLParser.parse("jdbc:sqlserver://127.0.0.1:1434;database=TestDB"); + SqlServerURLParser.parse("jdbc:sqlserver://127.0.0.1:1433;database=master"); static JdbcUrlUtil.UrlInfo MysqlUrlInfo = JdbcUrlUtil.getUrlInfo("jdbc:mysql://127.0.0.1:33061/liuliTest?useSSL=false"); static JdbcUrlUtil.UrlInfo pg = @@ -84,9 +85,14 @@ void listTables() { } @Test - void tableExists() { - - // boolean b = sqlServerCatalog.tableExists(tablePath); + void exists() { + Assertions.assertTrue(sqlServerCatalog.databaseExists("master")); + Assertions.assertTrue( + sqlServerCatalog.tableExists( + TablePath.of("master", "dbo", "MSreplication_options"))); + Assertions.assertTrue( + sqlServerCatalog.tableExists(TablePath.of("master", "dbo", "spt_fallback_db"))); + Assertions.assertFalse(sqlServerCatalog.tableExists(TablePath.of("master", "dbo", "xxx"))); } @Test diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_source_and_sink_with_multiple_tables.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_source_and_sink_with_multiple_tables.sql index a9b02e2ae3a..8c624959f87 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_source_and_sink_with_multiple_tables.sql +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_mysql_source_and_sink_with_multiple_tables.sql @@ -55,10 +55,10 @@ CREATE TABLE sink_table WITH ( 'user' = 'root', 'password' = 'Abc!@#135_seatunnel', 'generate_sink_sql' = 'true', - 'database' = 'sink' + 'database' = 'sink', 'table' = '${table_name}' ); -- If it's multi-table synchronization, there's no need to set select columns. -- You can directly use the syntax 'INSERT INTO sink_table SELECT source_table'. -INSERT INTO sink_table SELECT source_table; \ No newline at end of file +INSERT INTO sink_table SELECT source_table; From ec1c3198bcffb8ff6b41014c36d4ddacedb03868 Mon Sep 17 00:00:00 2001 From: "Nothing." Date: Mon, 29 Jul 2024 21:53:01 +0800 Subject: [PATCH 013/361] [Docs]translate event-listener doc into chinese (#7274) --- docs/zh/concept/event-listener.md | 114 ++++++++++++++++++++++++++++++ 1 file changed, 114 insertions(+) create mode 100644 docs/zh/concept/event-listener.md diff --git a/docs/zh/concept/event-listener.md b/docs/zh/concept/event-listener.md new file mode 100644 index 00000000000..69972cbfc56 --- /dev/null +++ b/docs/zh/concept/event-listener.md @@ -0,0 +1,114 @@ +# 事件监听器 + +## 介绍 + +SeaTunnel提供了丰富的事件监听器功能,用于管理数据同步时的状态。此功能在需要监听任务运行状态时十分重要(`org.apache.seatunnel.api.event`)。本文档将指导您如何使用这些参数并有效地利用他们。 + +## 支持的引擎 + +> SeaTunnel Zeta
+> Flink
+> Spark
+ +## API + +事件(event)API的定义在 `org.apache.seatunnel.api.event`包中。 + +### Event Data API + +- `org.apache.seatunnel.api.event.Event` - 事件数据的接口。 +- `org.apache.seatunnel.api.event.EventType` - 事件数据的枚举值。 + +### Event Listener API + +您可以自定义事件处理器,例如将事件发送到外部系统。 + +- `org.apache.seatunnel.api.event.EventHandler` - 事件处理器的接口,SPI将会自动从类路径中加载子类。 + +### Event Collect API + +- `org.apache.seatunnel.api.source.SourceSplitEnumerator` - 在`SourceSplitEnumerator`加载事件监听器。 + +```java +package org.apache.seatunnel.api.source; + +public interface SourceSplitEnumerator { + + interface Context { + + /** + * Get the {@link org.apache.seatunnel.api.event.EventListener} of this enumerator. + * + * @return + */ + EventListener getEventListener(); + } +} +``` + +- `org.apache.seatunnel.api.source.SourceReader` - 在`SourceReader`加载事件监听器。 + +```java +package org.apache.seatunnel.api.source; + +public interface SourceReader { + + interface Context { + + /** + * Get the {@link org.apache.seatunnel.api.event.EventListener} of this reader. + * + * @return + */ + EventListener getEventListener(); + } +} +``` + +- `org.apache.seatunnel.api.sink.SinkWriter` - 在`SinkWriter`加载事件监听器。 + +```java +package org.apache.seatunnel.api.sink; + +public interface SinkWriter { + + interface Context { + + /** + * Get the {@link org.apache.seatunnel.api.event.EventListener} of this writer. + * + * @return + */ + EventListener getEventListener(); + } +} +``` + +## 设置监听器 + +您需要设置引擎配置以使用事件监听器功能。 + +### Zeta 引擎 + +配置样例(seatunnel.yaml): + +``` +seatunnel: + engine: + event-report-http: + url: "http://example.com:1024/event/report" + headers: + Content-Type: application/json +``` + +### Flink 引擎 + +您可以定义 `org.apache.seatunnel.api.event.EventHandler` 接口并添加到类路径,SPI会自动加载。 + +支持的flink版本: 1.14.0+ + +样例: `org.apache.seatunnel.api.event.LoggingEventHandler` + +### Spark 引擎 + +您可以定义 `org.apache.seatunnel.api.event.EventHandler` 接口并添加到类路径,SPI会自动加载。 From b9acb573b75055931ab5dc911aa05a04ea830c4f Mon Sep 17 00:00:00 2001 From: hailin0 Date: Tue, 30 Jul 2024 16:12:57 +0800 Subject: [PATCH 014/361] [Feature][Transforms] Support cast to bytes function of sql (#7284) --- docs/en/transform-v2/sql-functions.md | 2 +- docs/zh/transform-v2/sql-functions.md | 2 +- .../src/test/resources/sql_transform/func_system.conf | 11 ++++++++++- .../seatunnel/transform/sql/zeta/ZetaSQLType.java | 4 ++++ .../transform/sql/zeta/functions/SystemFunction.java | 3 +++ 5 files changed, 19 insertions(+), 3 deletions(-) diff --git a/docs/en/transform-v2/sql-functions.md b/docs/en/transform-v2/sql-functions.md index e1c541ef1c9..3438a24de9c 100644 --- a/docs/en/transform-v2/sql-functions.md +++ b/docs/en/transform-v2/sql-functions.md @@ -889,7 +889,7 @@ CALL FROM_UNIXTIME(1672502400, 'yyyy-MM-dd HH:mm:ss','UTC+6') Converts a value to another data type. -Supported data types: STRING | VARCHAR, INT | INTEGER, LONG | BIGINT, BYTE, FLOAT, DOUBLE, DECIMAL(p,s), TIMESTAMP, DATE, TIME +Supported data types: STRING | VARCHAR, INT | INTEGER, LONG | BIGINT, BYTE, FLOAT, DOUBLE, DECIMAL(p,s), TIMESTAMP, DATE, TIME, BYTES Example: diff --git a/docs/zh/transform-v2/sql-functions.md b/docs/zh/transform-v2/sql-functions.md index cd90b948674..57c440a39b3 100644 --- a/docs/zh/transform-v2/sql-functions.md +++ b/docs/zh/transform-v2/sql-functions.md @@ -880,7 +880,7 @@ CALL FROM_UNIXTIME(1672502400, 'yyyy-MM-dd HH:mm:ss','UTC+6') 将一个值转换为另一个数据类型。 -支持的数据类型有:STRING | VARCHAR,INT | INTEGER,LONG | BIGINT,BYTE,FLOAT,DOUBLE,DECIMAL(p,s),TIMESTAMP,DATE,TIME +支持的数据类型有:STRING | VARCHAR,INT | INTEGER,LONG | BIGINT,BYTE,FLOAT,DOUBLE,DECIMAL(p,s),TIMESTAMP,DATE,TIME,BYTES 示例: diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf index 558d0cceb38..14f41665e34 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf @@ -49,7 +49,7 @@ transform { Sql { source_table_name = "fake" result_table_name = "fake1" - query = "select cast(id as STRING) as id, cast(id as INT) as id2, cast(id as DOUBLE) as id3 , cast(c1 as double) as c1_1, cast(c1 as DECIMAL(10,2)) as c1_2, cast(c2 as DATE) as c2_1, coalesce(c3,'Unknown') c3_1, ifnull(c3,'Unknown') c3_2, ifnull(nullif(name,'Joy Ding'),'NULL') name1, nullif(name,'Joy Ding_') name2, cast(c4 as timestamp) as c4_1, cast(c4 as decimal(17,4)) as c4_2, cast(c5 as date) as c5, cast(c6 as time) as c6 from fake" + query = "select cast(id as STRING) as id, cast(id as INT) as id2, cast(id as DOUBLE) as id3 , cast(c1 as double) as c1_1, cast(c1 as DECIMAL(10,2)) as c1_2, cast(c2 as DATE) as c2_1, coalesce(c3,'Unknown') c3_1, ifnull(c3,'Unknown') c3_2, ifnull(nullif(name,'Joy Ding'),'NULL') name1, nullif(name,'Joy Ding_') name2, cast(c4 as timestamp) as c4_1, cast(c4 as decimal(17,4)) as c4_2, cast(c5 as date) as c5, cast(c6 as time) as c6, cast(name as bytes) as c7 from fake" } } @@ -155,6 +155,15 @@ sink { field_value = [ {equals_to = "23:51:09"} ] + }, + { + field_name = "c7" + field_type = "bytes" + field_value = [ + { + rule_type = NOT_NULL + } + ] } ] } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java index 934cd883080..45b269bae67 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.table.type.DecimalType; import org.apache.seatunnel.api.table.type.LocalTimeType; import org.apache.seatunnel.api.table.type.MapType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.api.table.type.SqlType; @@ -69,6 +70,7 @@ public class ZetaSQLType { public static final String BIGINT = "BIGINT"; public static final String LONG = "LONG"; public static final String BYTE = "BYTE"; + public static final String BYTES = "BYTES"; public static final String DOUBLE = "DOUBLE"; public static final String FLOAT = "FLOAT"; public static final String TIMESTAMP = "TIMESTAMP"; @@ -311,6 +313,8 @@ private SeaTunnelDataType getCastType(CastExpression castExpression) { return BasicType.LONG_TYPE; case BYTE: return BasicType.BYTE_TYPE; + case BYTES: + return PrimitiveByteArrayType.INSTANCE; case DOUBLE: return BasicType.DOUBLE_TYPE; case FLOAT: diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java index 0039f0cade9..0b616b0fbe8 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java @@ -24,6 +24,7 @@ import java.math.BigDecimal; import java.math.RoundingMode; +import java.nio.charset.StandardCharsets; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; @@ -98,6 +99,8 @@ public static Object castAs(List args) { return Long.parseLong(v1.toString()); case "BYTE": return Byte.parseByte(v1.toString()); + case "BYTES": + return v1.toString().getBytes(StandardCharsets.UTF_8); case "DOUBLE": return Double.parseDouble(v1.toString()); case "FLOAT": From 9df557cb12d39831a737658994e2fe2662230e1a Mon Sep 17 00:00:00 2001 From: lizhenglei <127465317+jackyyyyyssss@users.noreply.github.com> Date: Tue, 30 Jul 2024 21:37:04 +0800 Subject: [PATCH 015/361] [Improve][Transform] Improve DynamicCompile transform (#7264) --- docs/en/transform-v2/dynamic-compile.md | 30 ++++++- .../e2e/common/container/TestContainer.java | 2 + .../flink/AbstractTestFlinkContainer.java | 7 ++ .../ConnectorPackageServiceContainer.java | 6 ++ .../seatunnel/SeaTunnelContainer.java | 6 ++ .../spark/AbstractTestSparkContainer.java | 7 ++ .../e2e/transform/TestDynamicCompileIT.java | 34 ++++++-- ...dynamic_groovy_java_compile_transform.conf | 2 + ...iple_dynamic_groovy_compile_transform.conf | 2 + ...ltiple_dynamic_java_compile_transform.conf | 2 + ...ngle_dynamic_groovy_compile_transform.conf | 1 + ...single_dynamic_java_compile_transform.conf | 1 + .../conf/single_groovy_path_compile.conf | 85 ++++++++++++++++++ .../conf/single_java_path_compile.conf | 86 +++++++++++++++++++ .../dynamic_compile/source_file/GroovyFile | 42 +++++++++ .../dynamic_compile/source_file/JavaFile | 39 +++++++++ .../dynamiccompile/CompilePattern.java | 23 +++++ .../DynamicCompileTransform.java | 30 +++++-- .../DynamicCompileTransformConfig.java | 12 +++ .../DynamicCompileTransformFactory.java | 8 ++ .../dynamiccompile/parse/AbstractParse.java | 2 +- .../{ParseUtil.java => AbstractParser.java} | 2 +- .../parse/GroovyClassParse.java | 4 +- ...yClassUtil.java => GroovyClassParser.java} | 4 +- .../dynamiccompile/parse/JavaClassParse.java | 4 +- ...avaClassUtil.java => JavaClassParser.java} | 27 +++--- 26 files changed, 432 insertions(+), 36 deletions(-) rename seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/{ => conf}/mixed_dynamic_groovy_java_compile_transform.conf (98%) rename seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/{ => conf}/multiple_dynamic_groovy_compile_transform.conf (98%) rename seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/{ => conf}/multiple_dynamic_java_compile_transform.conf (98%) rename seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/{ => conf}/single_dynamic_groovy_compile_transform.conf (98%) rename seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/{ => conf}/single_dynamic_java_compile_transform.conf (99%) create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_groovy_path_compile.conf create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_java_path_compile.conf create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/GroovyFile create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/JavaFile create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/CompilePattern.java rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/{ParseUtil.java => AbstractParser.java} (97%) rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/{GroovyClassUtil.java => GroovyClassParser.java} (89%) rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/{JavaClassUtil.java => JavaClassParser.java} (72%) diff --git a/docs/en/transform-v2/dynamic-compile.md b/docs/en/transform-v2/dynamic-compile.md index 5bfbbadbe08..4a772e8cbf0 100644 --- a/docs/en/transform-v2/dynamic-compile.md +++ b/docs/en/transform-v2/dynamic-compile.md @@ -11,8 +11,10 @@ If the conversion is too complex, it may affect performance | name | type | required | default value | |------------------|--------|----------|---------------| -| source_code | string | yes | | -| compile_language | string | yes | | +| source_code | string | no | | +| compile_language | Enum | yes | | +| compile_pattern | Enum | no | SOURCE_CODE | +| absolute_path | string | no | | ### source_code [string] @@ -24,11 +26,20 @@ If there are third-party dependency packages, please place them in ${SEATUNNEL_H Transform plugin common parameters, please refer to [Transform Plugin](common-options.md) for details -### compile_language [string] +### compile_language [Enum] Some syntax in Java may not be supported, please refer https://github.com/janino-compiler/janino GROOVY,JAVA +### compile_pattern [Enum] + +SOURCE_CODE,ABSOLUTE_PATH +If it is a SOURCE-CODE enumeration; the SOURCE-CODE attribute is required, and the ABSOLUTE_PATH enumeration;ABSOLUTE_PATH attribute is required + +### absolute_path [string] + +The absolute path of Java or Groovy files on the server + ## Example The data read from source is a table like this: @@ -46,6 +57,7 @@ transform { source_table_name = "fake" result_table_name = "fake1" compile_language="GROOVY" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor @@ -82,6 +94,7 @@ transform { source_table_name = "fake" result_table_name = "fake1" compile_language="JAVA" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; @@ -113,6 +126,17 @@ transform { } } + + transform { + DynamicCompile { + source_table_name = "fake" + result_table_name = "fake1" + compile_language="GROOVY" + compile_pattern="ABSOLUTE_PATH" + absolute_path="""/tmp/GroovyFile""" + + } +} ``` Then the data in result table `fake1` will like this diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/TestContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/TestContainer.java index 33b196eeba0..07fef2c295a 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/TestContainer.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/TestContainer.java @@ -55,4 +55,6 @@ default Container.ExecResult restoreJob(String confFile, String jobId) } String getServerLogs(); + + void copyFileToContainer(String path, String targetPath); } diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/flink/AbstractTestFlinkContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/flink/AbstractTestFlinkContainer.java index 7145da6242d..ff16c0c7541 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/flink/AbstractTestFlinkContainer.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/flink/AbstractTestFlinkContainer.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.e2e.common.container.AbstractTestContainer; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.util.ContainerUtil; import org.testcontainers.containers.Container; import org.testcontainers.containers.GenericContainer; @@ -168,4 +169,10 @@ public String executeJobManagerInnerCommand(String command) throws IOException, InterruptedException { return jobManager.execInContainer("bash", "-c", command).getStdout(); } + + @Override + public void copyFileToContainer(String path, String targetPath) { + ContainerUtil.copyFileIntoContainers( + ContainerUtil.getResourcesFile(path).toPath(), targetPath, jobManager); + } } diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/ConnectorPackageServiceContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/ConnectorPackageServiceContainer.java index 4f5ea990295..3a27d78d423 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/ConnectorPackageServiceContainer.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/ConnectorPackageServiceContainer.java @@ -234,4 +234,10 @@ public Container.ExecResult executeJob(String confFile, List variables) public String getServerLogs() { return server1.getLogs(); } + + @Override + public void copyFileToContainer(String path, String targetPath) { + ContainerUtil.copyFileIntoContainers( + ContainerUtil.getResourcesFile(path).toPath(), targetPath, server1); + } } diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java index 2d9e76ea3b5..802b1c32fba 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java @@ -455,4 +455,10 @@ public Container.ExecResult restoreJob(String confFile, String jobId) public String getServerLogs() { return server.getLogs(); } + + @Override + public void copyFileToContainer(String path, String targetPath) { + ContainerUtil.copyFileIntoContainers( + ContainerUtil.getResourcesFile(path).toPath(), targetPath, server); + } } diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/spark/AbstractTestSparkContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/spark/AbstractTestSparkContainer.java index fe07d082afd..9970ffb3aa7 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/spark/AbstractTestSparkContainer.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/spark/AbstractTestSparkContainer.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.e2e.common.container.AbstractTestContainer; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.util.ContainerUtil; import org.testcontainers.containers.Container; import org.testcontainers.containers.GenericContainer; @@ -118,4 +119,10 @@ public Container.ExecResult executeJob(String confFile, List variables) public String getServerLogs() { return master.getLogs(); } + + @Override + public void copyFileToContainer(String path, String targetPath) { + ContainerUtil.copyFileIntoContainers( + ContainerUtil.getResourcesFile(path).toPath(), targetPath, master); + } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestDynamicCompileIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestDynamicCompileIT.java index 5c5e69dad25..b57b332353a 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestDynamicCompileIT.java +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestDynamicCompileIT.java @@ -27,12 +27,13 @@ public class TestDynamicCompileIT extends TestSuiteBase { + private final String basePath = "/dynamic_compile/conf/"; + @TestTemplate public void testDynamicSingleCompileGroovy(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = - container.executeJob( - "/dynamic_compile/single_dynamic_groovy_compile_transform.conf"); + container.executeJob(basePath + "single_dynamic_groovy_compile_transform.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } @@ -40,7 +41,7 @@ public void testDynamicSingleCompileGroovy(TestContainer container) public void testDynamicSingleCompileJava(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = - container.executeJob("/dynamic_compile/single_dynamic_java_compile_transform.conf"); + container.executeJob(basePath + "single_dynamic_java_compile_transform.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } @@ -48,8 +49,7 @@ public void testDynamicSingleCompileJava(TestContainer container) public void testDynamicMultipleCompileGroovy(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = - container.executeJob( - "/dynamic_compile/multiple_dynamic_groovy_compile_transform.conf"); + container.executeJob(basePath + "multiple_dynamic_groovy_compile_transform.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } @@ -57,8 +57,7 @@ public void testDynamicMultipleCompileGroovy(TestContainer container) public void testDynamicMultipleCompileJava(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = - container.executeJob( - "/dynamic_compile/multiple_dynamic_java_compile_transform.conf"); + container.executeJob(basePath + "multiple_dynamic_java_compile_transform.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } @@ -66,8 +65,25 @@ public void testDynamicMultipleCompileJava(TestContainer container) public void testDynamicMixedCompileJavaAndGroovy(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = - container.executeJob( - "/dynamic_compile/mixed_dynamic_groovy_java_compile_transform.conf"); + container.executeJob(basePath + "mixed_dynamic_groovy_java_compile_transform.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } + + @TestTemplate + public void testDynamicSinglePathGroovy(TestContainer container) + throws IOException, InterruptedException { + container.copyFileToContainer("/dynamic_compile/source_file/GroovyFile", "/tmp/GroovyFile"); + Container.ExecResult execResult = + container.executeJob(basePath + "single_groovy_path_compile.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } + + @TestTemplate + public void testDynamicSinglePathJava(TestContainer container) + throws IOException, InterruptedException { + container.copyFileToContainer("/dynamic_compile/source_file/JavaFile", "/tmp/JavaFile"); + Container.ExecResult execResult = + container.executeJob(basePath + "single_java_path_compile.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/mixed_dynamic_groovy_java_compile_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/mixed_dynamic_groovy_java_compile_transform.conf similarity index 98% rename from seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/mixed_dynamic_groovy_java_compile_transform.conf rename to seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/mixed_dynamic_groovy_java_compile_transform.conf index 5c32e8d5a03..e91765fbf3c 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/mixed_dynamic_groovy_java_compile_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/mixed_dynamic_groovy_java_compile_transform.conf @@ -43,6 +43,7 @@ transform { source_table_name = "fake" result_table_name = "fake1" compile_language="JAVA" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; @@ -80,6 +81,7 @@ transform { source_table_name = "fake1" result_table_name = "fake2" compile_language="GROOVY" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/multiple_dynamic_groovy_compile_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/multiple_dynamic_groovy_compile_transform.conf similarity index 98% rename from seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/multiple_dynamic_groovy_compile_transform.conf rename to seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/multiple_dynamic_groovy_compile_transform.conf index 31756b99415..8689404a17e 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/multiple_dynamic_groovy_compile_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/multiple_dynamic_groovy_compile_transform.conf @@ -40,6 +40,7 @@ transform { source_table_name = "fake" result_table_name = "fake1" compile_language="GROOVY" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor @@ -73,6 +74,7 @@ transform { source_table_name = "fake1" result_table_name = "fake2" compile_language="GROOVY" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/multiple_dynamic_java_compile_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/multiple_dynamic_java_compile_transform.conf similarity index 98% rename from seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/multiple_dynamic_java_compile_transform.conf rename to seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/multiple_dynamic_java_compile_transform.conf index 94e3a41272c..9e59a5e5350 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/multiple_dynamic_java_compile_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/multiple_dynamic_java_compile_transform.conf @@ -43,6 +43,7 @@ transform { source_table_name = "fake" result_table_name = "fake1" compile_language="JAVA" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; @@ -80,6 +81,7 @@ transform { source_table_name = "fake1" result_table_name = "fake2" compile_language="JAVA" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/single_dynamic_groovy_compile_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_groovy_compile_transform.conf similarity index 98% rename from seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/single_dynamic_groovy_compile_transform.conf rename to seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_groovy_compile_transform.conf index c478d33ddc5..7958b880765 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/single_dynamic_groovy_compile_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_groovy_compile_transform.conf @@ -40,6 +40,7 @@ transform { source_table_name = "fake" result_table_name = "fake1" compile_language="GROOVY" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/single_dynamic_java_compile_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_java_compile_transform.conf similarity index 99% rename from seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/single_dynamic_java_compile_transform.conf rename to seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_java_compile_transform.conf index d3a735b6300..b65877d465c 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/single_dynamic_java_compile_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_java_compile_transform.conf @@ -40,6 +40,7 @@ DynamicCompile { source_table_name = "fake" result_table_name = "fake1" compile_language="JAVA" + compile_pattern="SOURCE_CODE" source_code=""" import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_groovy_path_compile.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_groovy_path_compile.conf new file mode 100644 index 00000000000..c9b00bdee80 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_groovy_path_compile.conf @@ -0,0 +1,85 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + job.mode = "BATCH" +} + +source { + FakeSource { + result_table_name = "fake" + row.num = 100 + schema = { + fields { + id = "int" + name = "string" + } + } + } +} + +transform { + DynamicCompile { + source_table_name = "fake" + result_table_name = "fake1" + compile_language="GROOVY" + compile_pattern="ABSOLUTE_PATH" + absolute_path="""/tmp/GroovyFile""" + + } +} + +sink { + Assert { + source_table_name = "fake1" + rules = + { + row_rules = [ + { + rule_type = MIN_ROW + rule_value = 100 + } + ], + field_rules = [ + { + field_name = id + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = aa + field_type = string + field_value = [ + { + rule_type = NOT_NULL + equals_to = "AA" + + } + + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_java_path_compile.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_java_path_compile.conf new file mode 100644 index 00000000000..3925dbe91e8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_java_path_compile.conf @@ -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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + job.mode = "BATCH" +} + +source { + FakeSource { + result_table_name = "fake" + row.num = 100 + schema = { + fields { + id = "int" + name = "string" + } + } + } +} + +transform { +DynamicCompile { + source_table_name = "fake" + result_table_name = "fake1" + compile_language="JAVA" + compile_pattern="ABSOLUTE_PATH" + absolute_path="""/tmp/JavaFile""" + + + } +} + +sink { + Assert { + source_table_name = "fake1" + rules = + { + row_rules = [ + { + rule_type = MIN_ROW + rule_value = 100 + } + ], + field_rules = [ + { + field_name = id + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = col1 + field_type = string + field_value = [ + { + rule_type = NOT_NULL + equals_to = "test1" + + } + + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/GroovyFile b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/GroovyFile new file mode 100644 index 00000000000..9bb6a8fcdfe --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/GroovyFile @@ -0,0 +1,42 @@ +/* + * 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. + */ +import org.apache.seatunnel.api.table.catalog.Column +import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor +import org.apache.seatunnel.api.table.catalog.CatalogTable +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.type.*; +import java.util.ArrayList; +class demo { + public Column[] getInlineOutputColumns(CatalogTable inputCatalogTable) { + List columns = new ArrayList<>(); + PhysicalColumn destColumn = + PhysicalColumn.of( + "aa", + BasicType.STRING_TYPE, + 10, + true, + "", + ""); + columns.add(destColumn); + return columns.toArray(new Column[0]); + } + public Object[] getInlineOutputFieldValues(SeaTunnelRowAccessor inputRow) { + Object[] fieldValues = new Object[1]; + fieldValues[0]="AA" + return fieldValues; + } +}; \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/JavaFile b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/JavaFile new file mode 100644 index 00000000000..7d1947c077e --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/source_file/JavaFile @@ -0,0 +1,39 @@ +/* + * 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. + */ +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; + +import java.util.ArrayList; + + + public Column[] getInlineOutputColumns(CatalogTable inputCatalogTable) { + + ArrayList columns = new ArrayList(); + PhysicalColumn destColumn = + PhysicalColumn.of("col1", BasicType.STRING_TYPE, 10, true, "", ""); + return new Column[] {destColumn}; + } + + public Object[] getInlineOutputFieldValues(SeaTunnelRowAccessor inputRow) { + + Object[] fieldValues = new Object[1]; + fieldValues[0] = "test1"; + return fieldValues; + } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/CompilePattern.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/CompilePattern.java new file mode 100644 index 00000000000..9b8c83a89df --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/CompilePattern.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.dynamiccompile; + +public enum CompilePattern { + SOURCE_CODE, + ABSOLUTE_PATH +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java index d798871401c..ea55569420d 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.common.utils.FileUtils; import org.apache.seatunnel.common.utils.ReflectionUtils; import org.apache.seatunnel.transform.common.MultipleFieldOutputTransform; import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; @@ -28,6 +29,8 @@ import org.apache.seatunnel.transform.dynamiccompile.parse.JavaClassParse; import org.apache.seatunnel.transform.exception.TransformException; +import java.nio.file.Paths; + import static org.apache.seatunnel.transform.dynamiccompile.CompileTransformErrorCode.COMPILE_TRANSFORM_ERROR_CODE; public class DynamicCompileTransform extends MultipleFieldOutputTransform { @@ -39,6 +42,8 @@ public class DynamicCompileTransform extends MultipleFieldOutputTransform { private final String sourceCode; + private final CompilePattern compilePattern; + private AbstractParse DynamicCompileParse; public DynamicCompileTransform(ReadonlyConfig readonlyConfig, CatalogTable catalogTable) { @@ -51,7 +56,18 @@ public DynamicCompileTransform(ReadonlyConfig readonlyConfig, CatalogTable catal } else if (CompileLanguage.JAVA.equals(compileLanguage)) { DynamicCompileParse = new JavaClassParse(); } - sourceCode = readonlyConfig.get(DynamicCompileTransformConfig.SOURCE_CODE); + compilePattern = readonlyConfig.get(DynamicCompileTransformConfig.COMPILE_PATTERN); + + if (CompilePattern.SOURCE_CODE.equals(compilePattern)) { + sourceCode = readonlyConfig.get(DynamicCompileTransformConfig.SOURCE_CODE); + } else { + // NPE will never happen because it is required in the ABSOLUTE_PATH mode + sourceCode = + FileUtils.readFileToStr( + Paths.get( + readonlyConfig.get( + DynamicCompileTransformConfig.ABSOLUTE_PATH))); + } } @Override @@ -65,7 +81,7 @@ protected Column[] getOutputColumns() { try { result = ReflectionUtils.invoke( - DynamicCompileParse.parseClass(sourceCode).newInstance(), + getCompileLanguageInstance(), getInlineOutputColumns, inputCatalogTable); @@ -82,13 +98,17 @@ protected Object[] getOutputFieldValues(SeaTunnelRowAccessor inputRow) { try { result = ReflectionUtils.invoke( - DynamicCompileParse.parseClass(sourceCode).newInstance(), - getInlineOutputFieldValues, - inputRow); + getCompileLanguageInstance(), getInlineOutputFieldValues, inputRow); } catch (Exception e) { throw new TransformException(COMPILE_TRANSFORM_ERROR_CODE, e.getMessage()); } return (Object[]) result; } + + private Object getCompileLanguageInstance() + throws InstantiationException, IllegalAccessException { + Class compileClass = DynamicCompileParse.parseClassSourceCode(sourceCode); + return compileClass.newInstance(); + } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformConfig.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformConfig.java index 48a47d03830..f975ba28444 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformConfig.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformConfig.java @@ -39,4 +39,16 @@ public class DynamicCompileTransformConfig implements Serializable { .enumType(CompileLanguage.class) .noDefaultValue() .withDescription("compile language"); + + public static final Option ABSOLUTE_PATH = + Options.key("absolute_path") + .stringType() + .noDefaultValue() + .withDescription("absolute_path"); + + public static final Option COMPILE_PATTERN = + Options.key("compile_pattern") + .enumType(CompilePattern.class) + .defaultValue(CompilePattern.SOURCE_CODE) + .withDescription("compile_pattern"); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformFactory.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformFactory.java index 422bb0ff146..195102c4d91 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformFactory.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransformFactory.java @@ -38,7 +38,15 @@ public OptionRule optionRule() { return OptionRule.builder() .required( DynamicCompileTransformConfig.COMPILE_LANGUAGE, + DynamicCompileTransformConfig.COMPILE_PATTERN) + .conditional( + DynamicCompileTransformConfig.COMPILE_PATTERN, + CompilePattern.SOURCE_CODE, DynamicCompileTransformConfig.SOURCE_CODE) + .conditional( + DynamicCompileTransformConfig.COMPILE_PATTERN, + CompilePattern.ABSOLUTE_PATH, + DynamicCompileTransformConfig.ABSOLUTE_PATH) .build(); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/AbstractParse.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/AbstractParse.java index 906e9c26347..51d94fa1662 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/AbstractParse.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/AbstractParse.java @@ -21,5 +21,5 @@ public abstract class AbstractParse implements Serializable { - public abstract Class parseClass(String sourceCode); + public abstract Class parseClassSourceCode(String sourceCode); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/ParseUtil.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/AbstractParser.java similarity index 97% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/ParseUtil.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/AbstractParser.java index c4afd47e25d..3d8d58fd82d 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/ParseUtil.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/AbstractParser.java @@ -20,7 +20,7 @@ import java.util.concurrent.ConcurrentHashMap; -public abstract class ParseUtil { +public abstract class AbstractParser { protected static ConcurrentHashMap> classCache = new ConcurrentHashMap<>(); // Abstraction layer: Do not want to serialize and pass the classloader protected static String getClassKey(String sourceCode) { diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassParse.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassParse.java index d94607eb1f5..7ae95da6288 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassParse.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassParse.java @@ -20,7 +20,7 @@ public class GroovyClassParse extends AbstractParse { @Override - public Class parseClass(String sourceCode) { - return GroovyClassUtil.parseWithCache(sourceCode); + public Class parseClassSourceCode(String sourceCode) { + return GroovyClassParser.parseSourceCodeWithCache(sourceCode); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassUtil.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassParser.java similarity index 89% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassUtil.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassParser.java index 5fab0e8761f..c951335e371 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassUtil.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/GroovyClassParser.java @@ -18,10 +18,10 @@ import groovy.lang.GroovyClassLoader; -public class GroovyClassUtil extends ParseUtil { +public class GroovyClassParser extends AbstractParser { private static final GroovyClassLoader groovyClassLoader = new GroovyClassLoader(); - public static Class parseWithCache(String sourceCode) { + public static Class parseSourceCodeWithCache(String sourceCode) { return classCache.computeIfAbsent( getClassKey(sourceCode), clazz -> groovyClassLoader.parseClass(sourceCode)); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassParse.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassParse.java index 3cd5bdd96e9..9b77963eea6 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassParse.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassParse.java @@ -19,7 +19,7 @@ public class JavaClassParse extends AbstractParse { @Override - public Class parseClass(String sourceCode) { - return JavaClassUtil.parseWithCache(sourceCode); + public Class parseClassSourceCode(String sourceCode) { + return JavaClassParser.parseSourceCodeWithCache(sourceCode); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassUtil.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassParser.java similarity index 72% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassUtil.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassParser.java index 344b2708d4a..d9bee066f72 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassUtil.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/parse/JavaClassParser.java @@ -21,24 +21,29 @@ import java.util.function.Function; -public class JavaClassUtil extends ParseUtil { - - public static Class parseWithCache(String sourceCode) { +public class JavaClassParser extends AbstractParser { + public static Class parseSourceCodeWithCache(String sourceCode) { return classCache.computeIfAbsent( getClassKey(sourceCode), new Function>() { @Override public Class apply(String classKey) { - try { - ClassBodyEvaluator cbe = new ClassBodyEvaluator(); - cbe.cook(sourceCode); - return cbe.getClazz(); - - } catch (CompileException e) { - throw new RuntimeException(e); - } + return getInnerClass(sourceCode); } }); } + + private static Class getInnerClass(String FilePathOrSourceCode) { + try { + ClassBodyEvaluator cbe = new ClassBodyEvaluator(); + + cbe.cook(FilePathOrSourceCode); + + return cbe.getClazz(); + + } catch (CompileException e) { + throw new RuntimeException(e); + } + } } From 34a6b8e9f62d9e04da554062b46949c7075176ed Mon Sep 17 00:00:00 2001 From: Jast Date: Wed, 31 Jul 2024 14:43:11 +0800 Subject: [PATCH 016/361] [hotfix][connector-v2-hbase]fix and optimize hbase source problem (#7148) * [hotfix][improve][doc]optimize connector hbase source * [doc]add dependent document * [doc]update dependent document * [improve]improve static use * [hotfix]add test case * [hotfix]add test case --------- Co-authored-by: Jia Fan --- docs/en/connector-v2/source/Hbase.md | 109 ++++++++------- docs/zh/connector-v2/source/Hbase.md | 96 +++++++++++++ docs/zh/connector-v2/source/common-options.md | 81 +++++++++++ .../seatunnel/hbase/config/HbaseConfig.java | 27 +++- .../hbase/config/HbaseParameters.java | 24 +++- .../seatunnel/hbase/sink/HbaseSink.java | 2 +- .../seatunnel/hbase/source/HbaseSource.java | 6 +- .../hbase/source/HbaseSourceFactory.java | 1 - .../hbase/source/HbaseSourceReader.java | 30 ++-- .../e2e/connector/hbase/HbaseIT.java | 37 +++-- ...ase-source-to-assert-with-batch-query.conf | 132 ++++++++++++++++++ 11 files changed, 455 insertions(+), 90 deletions(-) create mode 100644 docs/zh/connector-v2/source/Hbase.md create mode 100644 docs/zh/connector-v2/source/common-options.md create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-source-to-assert-with-batch-query.conf diff --git a/docs/en/connector-v2/source/Hbase.md b/docs/en/connector-v2/source/Hbase.md index 677b827fb29..753d68eb6e8 100644 --- a/docs/en/connector-v2/source/Hbase.md +++ b/docs/en/connector-v2/source/Hbase.md @@ -1,12 +1,12 @@ # Hbase -> Hbase source connector +> Hbase Source Connector ## Description -Read data from Apache Hbase. +Reads data from Apache Hbase. -## Key features +## Key Features - [x] [batch](../../concept/connector-v2-features.md) - [ ] [stream](../../concept/connector-v2-features.md) @@ -17,75 +17,80 @@ Read data from Apache Hbase. ## Options -| name | type | required | default value | -|--------------------|--------|----------|---------------| -| zookeeper_quorum | string | yes | - | -| table | string | yes | - | -| query_columns | list | yes | - | -| schema | config | yes | - | -| hbase_extra_config | string | no | - | -| common-options | | no | - | +| Name | Type | Required | Default | +|--------------------|---------|----------|---------| +| zookeeper_quorum | string | Yes | - | +| table | string | Yes | - | +| schema | config | Yes | - | +| hbase_extra_config | string | No | - | +| caching | int | No | -1 | +| batch | int | No | -1 | +| cache_blocks | boolean | No | false | +| common-options | | No | - | ### zookeeper_quorum [string] -The zookeeper cluster host of hbase, example: "hadoop001:2181,hadoop002:2181,hadoop003:2181" +The zookeeper quorum for Hbase cluster hosts, e.g., "hadoop001:2181,hadoop002:2181,hadoop003:2181". ### table [string] -The table name you want to write, example: "seatunnel" - -### query_columns [list] - -The column name which you want to query in the table. If you want to query the rowkey column, please set "rowkey" in query_columns. -Other column format should be: columnFamily:columnName, example: ["rowkey", "columnFamily1:column1", "columnFamily1:column1", "columnFamily2:column1"] +The name of the table to write to, e.g., "seatunnel". ### schema [config] -Hbase uses byte arrays for storage. Therefore, you need to configure data types for each column in a table. For more information, see: [guide](../../concept/schema-feature.md#how-to-declare-type-supported). +Hbase stores data in byte arrays. Therefore, you need to configure the data types for each column in the table. For more information, see: [guide](../../concept/schema-feature.md#how-to-declare-type-supported). ### hbase_extra_config [config] -The extra configuration of hbase +Additional configurations for Hbase. + +### caching + +The caching parameter sets the number of rows fetched per server trip during scans. This reduces round-trips between client and server, improving scan efficiency. Default: -1. + +### batch + +The batch parameter sets the maximum number of columns returned per scan. This is useful for rows with many columns to avoid fetching excessive data at once, thus saving memory and improving performance. -### common options +### cache_blocks -Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details +The cache_blocks parameter determines whether to cache data blocks during scans. By default, HBase caches data blocks during scans. Setting this to false reduces memory usage during scans. Default in SeaTunnel: false. -## Examples +### common-options + +Common parameters for Source plugins, refer to [Common Source Options](common-options.md). + +## Example ```bash source { Hbase { - zookeeper_quorum = "hadoop001:2181,hadoop002:2181,hadoop003:2181" - table = "seatunnel_test" - query_columns=["rowkey", "columnFamily1:column1", "columnFamily1:column1", "columnFamily2:column1"] - schema = { - columns = [ - { - name = rowkey - type = string - }, - { - name = "columnFamily1:column1" - type = boolean - }, - { - name = "columnFamily1:column1" - type = double - }, - { - name = "columnFamily2:column1" - type = bigint - } - ] - } + zookeeper_quorum = "hadoop001:2181,hadoop002:2181,hadoop003:2181" + table = "seatunnel_test" + caching = 1000 + batch = 100 + cache_blocks = false + schema = { + columns = [ + { + name = "rowkey" + type = string + }, + { + name = "columnFamily1:column1" + type = boolean + }, + { + name = "columnFamily1:column2" + type = double + }, + { + name = "columnFamily2:column1" + type = bigint + } + ] + } } } ``` -## Changelog - -### next version - -- Add Hbase Source Connector - diff --git a/docs/zh/connector-v2/source/Hbase.md b/docs/zh/connector-v2/source/Hbase.md new file mode 100644 index 00000000000..5f15a30b99a --- /dev/null +++ b/docs/zh/connector-v2/source/Hbase.md @@ -0,0 +1,96 @@ +# Hbase + +> Hbase 源连接器 + +## 描述 + +从 Apache Hbase 读取数据。 + +## 主要功能 + +- [x] [批处理](../../concept/connector-v2-features.md) +- [ ] [流处理](../../concept/connector-v2-features.md) +- [ ] [精确一次](../../concept/connector-v2-features.md) +- [x] [Schema](../../concept/connector-v2-features.md) +- [x] [并行度](../../concept/connector-v2-features.md) +- [ ] [支持用户定义的拆分](../../concept/connector-v2-features.md) + +## 选项 + +| 名称 | 类型 | 必填 | 默认值 | +|--------------------|---------|----|-------| +| zookeeper_quorum | string | 是 | - | +| table | string | 是 | - | +| schema | config | 是 | - | +| hbase_extra_config | string | 否 | - | +| caching | int | 否 | -1 | +| batch | int | 否 | -1 | +| cache_blocks | boolean | 否 | false | +| common-options | | 否 | - | + +### zookeeper_quorum [string] + +hbase的zookeeper集群主机,例如:“hadoop001:2181,hadoop002:2181,hadoop003:2181” + +### table [string] + +要写入的表名,例如:“seatunnel” + +### schema [config] + +Hbase 使用字节数组进行存储。因此,您需要为表中的每一列配置数据类型。有关更多信息,请参阅:[guide](../../concept/schema-feature.md#how-to-declare-type-supported)。 + +### hbase_extra_config [config] + +hbase 的额外配置 + +### caching + +caching 参数用于设置在扫描过程中一次从服务器端获取的行数。这可以减少客户端与服务器之间的往返次数,从而提高扫描效率。默认值:-1 + +### batch + +batch 参数用于设置在扫描过程中每次返回的最大列数。这对于处理有很多列的行特别有用,可以避免一次性返回过多数据,从而节省内存并提高性能。 + +### cache_blocks + +cache_blocks 参数用于设置在扫描过程中是否缓存数据块。默认情况下,HBase 会在扫描时将数据块缓存到块缓存中。如果设置为 false,则在扫描过程中不会缓存数据块,从而减少内存的使用。在SeaTunnel中默认值为: false + +### 常用选项 + +Source 插件常用参数,具体请参考 [Source 常用选项](common-options.md) + +## 示例 + +```bash +source { + Hbase { + zookeeper_quorum = "hadoop001:2181,hadoop002:2181,hadoop003:2181" + table = "seatunnel_test" + caching = 1000 + batch = 100 + cache_blocks = false + schema = { + columns = [ + { + name = "rowkey" + type = string + }, + { + name = "columnFamily1:column1" + type = boolean + }, + { + name = "columnFamily1:column2" + type = double + }, + { + name = "columnFamily2:column1" + type = bigint + } + ] + } + } +} +``` + diff --git a/docs/zh/connector-v2/source/common-options.md b/docs/zh/connector-v2/source/common-options.md new file mode 100644 index 00000000000..902dca2c195 --- /dev/null +++ b/docs/zh/connector-v2/source/common-options.md @@ -0,0 +1,81 @@ +# Source Common Options + +> Source connector 的常用参数 + +| 名称 | 类型 | 必填 | 默认值 | 描述 | +|-------------------|--------|----|-----|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| result_table_name | String | 否 | - | 当未指定 `result_table_name` 时,此插件处理的数据将不会被注册为可由其他插件直接访问的数据集 `(dataStream/dataset)`,或称为临时表 `(table)`。
当指定了 `result_table_name` 时,此插件处理的数据将被注册为可由其他插件直接访问的数据集 `(dataStream/dataset)`,或称为临时表 `(table)`。此处注册的数据集 `(dataStream/dataset)` 可通过指定 `source_table_name` 直接被其他插件访问。 | +| parallelism | Int | 否 | - | 当未指定 `parallelism` 时,默认使用环境中的 `parallelism`。
当指定了 `parallelism` 时,将覆盖环境中的 `parallelism` 设置。 | + +# 重要提示 + +在作业配置中使用 `result_table_name` 时,必须设置 `source_table_name` 参数。 + +## 任务示例 + +### 简单示例 + +> 注册一个流或批处理数据源,并在注册时返回表名 `fake_table` + +```bash +source { + FakeSourceStream { + result_table_name = "fake_table" + } +} +``` + +### 复杂示例 + +> 这是将Fake数据源转换并写入到两个不同的目标中 + +```bash +env { + job.mode = "BATCH" +} + +source { + FakeSource { + result_table_name = "fake" + row.num = 100 + schema = { + fields { + id = "int" + name = "string" + age = "int" + c_timestamp = "timestamp" + c_date = "date" + c_map = "map" + c_array = "array" + c_decimal = "decimal(30, 8)" + c_row = { + c_row = { + c_int = int + } + } + } + } + } +} + +transform { + Sql { + source_table_name = "fake" + result_table_name = "fake1" + # 查询表名必须与字段 'source_table_name' 相同 + query = "select id, regexp_replace(name, '.+', 'b') as name, age+1 as age, pi() as pi, c_timestamp, c_date, c_map, c_array, c_decimal, c_row from fake" + } + # SQL 转换支持基本函数和条件操作 + # 但不支持复杂的 SQL 操作,包括:多源表/行 JOIN 和聚合操作等 +} + +sink { + Console { + source_table_name = "fake1" + } + Console { + source_table_name = "fake" + } +} +``` + diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseConfig.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseConfig.java index 88c068bee11..44a5640ffed 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseConfig.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseConfig.java @@ -42,12 +42,6 @@ public class HbaseConfig { .noDefaultValue() .withDescription("Hbase rowkey column"); - public static final Option> QUERY_COLUMNS = - Options.key("query_columns") - .listType() - .noDefaultValue() - .withDescription("query Hbase columns"); - public static final Option ROWKEY_DELIMITER = Options.key("rowkey_delimiter") .stringType() @@ -104,6 +98,27 @@ public class HbaseConfig { .withDescription( "The expiration time configuration for writing hbase data. The default value is -1, indicating no expiration time."); + public static final Option HBASE_CACHE_BLOCKS_CONFIG = + Options.key("cache_blocks") + .booleanType() + .defaultValue(false) + .withDescription( + "When it is false, data blocks are not cached. When it is true, data blocks are cached. This value should be set to false when scanning a large amount of data to reduce memory consumption. The default value is false"); + + public static final Option HBASE_CACHING_CONFIG = + Options.key("caching") + .intType() + .defaultValue(-1) + .withDescription( + "Set the number of rows read from the server each time can reduce the number of round trips between the client and the server, thereby improving performance. The default value is -1."); + + public static final Option HBASE_BATCH_CONFIG = + Options.key("batch") + .intType() + .defaultValue(-1) + .withDescription( + "Set the batch size to control the maximum number of cells returned each time, thereby controlling the amount of data returned by a single RPC call. The default value is -1."); + public enum NullMode { SKIP, EMPTY; diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseParameters.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseParameters.java index 490e2481070..c25f04b3753 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseParameters.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseParameters.java @@ -30,10 +30,12 @@ import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ENCODING; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.FAMILY_NAME; +import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.HBASE_BATCH_CONFIG; +import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.HBASE_CACHE_BLOCKS_CONFIG; +import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.HBASE_CACHING_CONFIG; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.HBASE_EXTRA_CONFIG; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.HBASE_TTL_CONFIG; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.NULL_MODE; -import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.QUERY_COLUMNS; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ROWKEY_COLUMNS; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ROWKEY_DELIMITER; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.TABLE; @@ -60,8 +62,14 @@ public class HbaseParameters implements Serializable { private Map hbaseExtraConfig; + @Builder.Default private int caching = HBASE_CACHING_CONFIG.defaultValue(); + + @Builder.Default private int batch = HBASE_BATCH_CONFIG.defaultValue(); + @Builder.Default private Long ttl = HBASE_TTL_CONFIG.defaultValue(); + @Builder.Default private boolean cacheBlocks = HBASE_CACHE_BLOCKS_CONFIG.defaultValue(); + @Builder.Default private String rowkeyDelimiter = ROWKEY_DELIMITER.defaultValue(); @Builder.Default private HbaseConfig.NullMode nullMode = NULL_MODE.defaultValue(); @@ -72,7 +80,7 @@ public class HbaseParameters implements Serializable { @Builder.Default private HbaseConfig.EnCoding enCoding = ENCODING.defaultValue(); - public static HbaseParameters buildWithConfig(Config pluginConfig) { + public static HbaseParameters buildWithSinkConfig(Config pluginConfig) { HbaseParametersBuilder builder = HbaseParameters.builder(); // required parameters @@ -113,18 +121,26 @@ public static HbaseParameters buildWithConfig(Config pluginConfig) { return builder.build(); } - public static HbaseParameters buildWithSinkConfig(Config pluginConfig) { + public static HbaseParameters buildWithSourceConfig(Config pluginConfig) { HbaseParametersBuilder builder = HbaseParameters.builder(); // required parameters builder.zookeeperQuorum(pluginConfig.getString(ZOOKEEPER_QUORUM.key())); builder.table(pluginConfig.getString(TABLE.key())); - builder.columns(pluginConfig.getStringList(QUERY_COLUMNS.key())); if (pluginConfig.hasPath(HBASE_EXTRA_CONFIG.key())) { Config extraConfig = pluginConfig.getConfig(HBASE_EXTRA_CONFIG.key()); builder.hbaseExtraConfig(TypesafeConfigUtils.configToMap(extraConfig)); } + if (pluginConfig.hasPath(HBASE_CACHING_CONFIG.key())) { + builder.caching(pluginConfig.getInt(HBASE_CACHING_CONFIG.key())); + } + if (pluginConfig.hasPath(HBASE_BATCH_CONFIG.key())) { + builder.batch(pluginConfig.getInt(HBASE_BATCH_CONFIG.key())); + } + if (pluginConfig.hasPath(HBASE_CACHE_BLOCKS_CONFIG.key())) { + builder.cacheBlocks(pluginConfig.getBoolean(HBASE_CACHE_BLOCKS_CONFIG.key())); + } return builder.build(); } } diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSink.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSink.java index 848e1e82053..4f7b929223f 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSink.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSink.java @@ -79,7 +79,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { "PluginName: %s, PluginType: %s, Message: %s", getPluginName(), PluginType.SINK, result.getMsg())); } - this.hbaseParameters = HbaseParameters.buildWithConfig(pluginConfig); + this.hbaseParameters = HbaseParameters.buildWithSinkConfig(pluginConfig); if (hbaseParameters.getFamilyNames().size() == 0) { throw new HbaseConnectorException( SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSource.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSource.java index 869e33f6235..3aca3161516 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSource.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSource.java @@ -44,7 +44,6 @@ import java.util.List; -import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.QUERY_COLUMNS; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.TABLE; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ZOOKEEPER_QUORUM; @@ -68,8 +67,7 @@ public String getPluginName() { HbaseSource(Config pluginConfig) { this.pluginConfig = pluginConfig; CheckResult result = - CheckConfigUtil.checkAllExists( - pluginConfig, ZOOKEEPER_QUORUM.key(), TABLE.key(), QUERY_COLUMNS.key()); + CheckConfigUtil.checkAllExists(pluginConfig, ZOOKEEPER_QUORUM.key(), TABLE.key()); if (!result.isSuccess()) { throw new HbaseConnectorException( SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, @@ -77,7 +75,7 @@ public String getPluginName() { "PluginName: %s, PluginType: %s, Message: %s", getPluginName(), PluginType.SOURCE, result.getMsg())); } - this.hbaseParameters = HbaseParameters.buildWithSinkConfig(pluginConfig); + this.hbaseParameters = HbaseParameters.buildWithSourceConfig(pluginConfig); this.catalogTable = CatalogTableUtil.buildWithConfig(pluginConfig); this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); } diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceFactory.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceFactory.java index 4eec3e00482..2de385dbd18 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceFactory.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceFactory.java @@ -45,7 +45,6 @@ public OptionRule optionRule() { return OptionRule.builder() .required(HbaseConfig.ZOOKEEPER_QUORUM) .required(HbaseConfig.TABLE) - .required(HbaseConfig.QUERY_COLUMNS) .build(); } diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceReader.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceReader.java index 556374844e9..526ac826db1 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceReader.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceReader.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.util.Bytes; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; @@ -39,13 +40,13 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Arrays; import java.util.Deque; -import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.stream.Collectors; @Slf4j public class HbaseSourceReader implements SourceReader { @@ -54,7 +55,6 @@ public class HbaseSourceReader implements SourceReader namesMap; - private final Set columnFamilies = new LinkedHashSet<>(); private final SourceReader.Context context; private final SeaTunnelRowType seaTunnelRowType; private volatile boolean noMoreSplit = false; @@ -74,16 +74,17 @@ public HbaseSourceReader( this.seaTunnelRowType = seaTunnelRowType; this.namesMap = Maps.newConcurrentMap(); - this.columnNames = hbaseParameters.getColumns(); + this.columnNames = + Arrays.asList(seaTunnelRowType.getFieldNames()).stream() + .filter(name -> !ROW_KEY.equals(name)) + .collect(Collectors.toList()); // Check if input column names are in format: [ columnFamily:column ]. this.columnNames.stream() - .peek( + .forEach( column -> Preconditions.checkArgument( - (column.contains(":") && column.split(":").length == 2) - || this.ROW_KEY.equalsIgnoreCase(column), - "Invalid column names, it should be [ColumnFamily:Column] format")) - .forEach(column -> this.columnFamilies.add(column.split(":")[0])); + column.contains(":") && column.split(":").length == 2, + "Invalid column names, it should be [ColumnFamily:Column] format")); connection = HbaseConnectionUtil.getHbaseConnection(hbaseParameters); } @@ -122,6 +123,15 @@ public void pollNext(Collector output) throws Exception { Scan scan = new Scan(); scan.withStartRow(split.getStartRow(), true); scan.withStopRow(split.getEndRow(), true); + scan.setCacheBlocks(hbaseParameters.isCacheBlocks()); + scan.setCaching(hbaseParameters.getCaching()); + scan.setBatch(hbaseParameters.getBatch()); + for (String columnName : this.columnNames) { + String[] columnNameSplit = columnName.split(":"); + scan.addColumn( + Bytes.toBytes(columnNameSplit[0]), + Bytes.toBytes(columnNameSplit[1])); + } this.currentScanner = this.connection .getTable(TableName.valueOf(hbaseParameters.getTable())) @@ -152,7 +162,7 @@ private byte[][] convertRawRow(Result result) { byte[] bytes; try { // handle rowkey column - if (this.ROW_KEY.equals(columnName)) { + if (ROW_KEY.equals(columnName)) { bytes = result.getRow(); } else { byte[][] arr = this.namesMap.get(columnName); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java index 13a7a8805a6..85ceef92353 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java @@ -93,18 +93,7 @@ public void tearDown() throws Exception { @TestTemplate public void testHbaseSink(TestContainer container) throws IOException, InterruptedException { - deleteData(table); - Container.ExecResult sinkExecResult = container.executeJob("/fake-to-hbase.conf"); - Assertions.assertEquals(0, sinkExecResult.getExitCode()); - Table hbaseTable = hbaseConnection.getTable(table); - Scan scan = new Scan(); - ResultScanner scanner = hbaseTable.getScanner(scan); - ArrayList results = new ArrayList<>(); - for (Result result : scanner) { - results.add(result); - } - Assertions.assertEquals(results.size(), 5); - scanner.close(); + fakeToHbase(container); Container.ExecResult sourceExecResult = container.executeJob("/hbase-to-assert.conf"); Assertions.assertEquals(0, sourceExecResult.getExitCode()); } @@ -177,6 +166,30 @@ public void testHbaseSinkAssignCfSink(TestContainer container) Assertions.assertEquals(cf2Count, 5); } + @TestTemplate + public void testHbaseSourceWithBatchQuery(TestContainer container) + throws IOException, InterruptedException { + fakeToHbase(container); + Container.ExecResult sourceExecResult = + container.executeJob("/hbase-source-to-assert-with-batch-query.conf"); + Assertions.assertEquals(0, sourceExecResult.getExitCode()); + } + + private void fakeToHbase(TestContainer container) throws IOException, InterruptedException { + deleteData(table); + Container.ExecResult sinkExecResult = container.executeJob("/fake-to-hbase.conf"); + Assertions.assertEquals(0, sinkExecResult.getExitCode()); + Table hbaseTable = hbaseConnection.getTable(table); + Scan scan = new Scan(); + ResultScanner scanner = hbaseTable.getScanner(scan); + ArrayList results = new ArrayList<>(); + for (Result result : scanner) { + results.add(result); + } + Assertions.assertEquals(results.size(), 5); + scanner.close(); + } + private void deleteData(TableName table) throws IOException { Table hbaseTable = hbaseConnection.getTable(table); Scan scan = new Scan(); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-source-to-assert-with-batch-query.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-source-to-assert-with-batch-query.conf new file mode 100644 index 00000000000..c89cf28e25d --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-source-to-assert-with-batch-query.conf @@ -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. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + Hbase { + zookeeper_quorum = "hbase_e2e:2181" + table = "seatunnel_test" + query_columns=["rowkey", "info:age", "info:c_double", "info:c_boolean","info:c_bigint","info:c_smallint","info:c_tinyint","info:c_float"] + caching = 1000 + batch = 100 + cache_blocks = false + schema = { + columns = [ + { + name = rowkey + type = string + }, + { + name = "info:age" + type = int + }, + { + name = "info:c_double" + type = double + }, + { + name = "info:c_boolean" + type = boolean + }, + { + name = "info:c_bigint" + type = bigint + }, + { + name = "info:c_smallint" + type = smallint + }, + { + name = "info:c_tinyint" + type = tinyint + }, + { + name = "info:c_float" + type = float + } + ] + } + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + }, + { + rule_type = MIN_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = rowkey + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = "info:c_boolean" + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = "info:c_double" + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = "info:c_bigint" + field_type = bigint + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = "info:age" + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file From 82f5d8c71212b0ceb4a57d9cc7cd97cab5aa6d4e Mon Sep 17 00:00:00 2001 From: hailin0 Date: Sat, 3 Aug 2024 12:02:19 +0800 Subject: [PATCH 017/361] [Improve][SQL-Transform] Remove escape identifier from output fields (#7297) --- .../resources/sql_transform/func_system.conf | 9 +- .../transform/sql/zeta/ZetaSQLEngine.java | 10 +- .../transform/sql/zeta/ZetaSQLFunction.java | 26 +++- .../transform/sql/zeta/ZetaSQLType.java | 16 ++- .../transform/sql/SQLTransformTest.java | 136 ++++++++++++++++++ 5 files changed, 191 insertions(+), 6 deletions(-) diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf index 14f41665e34..a189c7c2ddc 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf @@ -49,7 +49,7 @@ transform { Sql { source_table_name = "fake" result_table_name = "fake1" - query = "select cast(id as STRING) as id, cast(id as INT) as id2, cast(id as DOUBLE) as id3 , cast(c1 as double) as c1_1, cast(c1 as DECIMAL(10,2)) as c1_2, cast(c2 as DATE) as c2_1, coalesce(c3,'Unknown') c3_1, ifnull(c3,'Unknown') c3_2, ifnull(nullif(name,'Joy Ding'),'NULL') name1, nullif(name,'Joy Ding_') name2, cast(c4 as timestamp) as c4_1, cast(c4 as decimal(17,4)) as c4_2, cast(c5 as date) as c5, cast(c6 as time) as c6, cast(name as bytes) as c7 from fake" + query = "select cast(id as STRING) as id, cast(id as INT) as id2, cast(id as DOUBLE) as id3 , cast(c1 as double) as c1_1, cast(c1 as DECIMAL(10,2)) as c1_2, cast(c2 as DATE) as c2_1, coalesce(c3,'Unknown') c3_1, ifnull(c3,'Unknown') c3_2, ifnull(nullif(name,'Joy Ding'),'NULL') name1, nullif(name,'Joy Ding_') name2, cast(c4 as timestamp) as c4_1, cast(c4 as decimal(17,4)) as c4_2, cast(c5 as date) as c5, cast(c6 as time) as c6, cast(name as bytes) as c7, name as `apply` from fake" } } @@ -164,6 +164,13 @@ sink { rule_type = NOT_NULL } ] + }, + { + field_name = "apply" + field_type = "string" + field_value = [ + {equals_to = "Joy Ding"} + ] } ] } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java index 42f5d8205dc..993b4e0a3c2 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java @@ -50,6 +50,8 @@ public class ZetaSQLEngine implements SQLEngine { private static final Logger log = LoggerFactory.getLogger(ZetaSQLEngine.class); + public static final String ESCAPE_IDENTIFIER = "`"; + private String inputTableName; @Nullable private String catalogTableName; private SeaTunnelRowType inputRowType; @@ -193,9 +195,13 @@ public SeaTunnelRowType typeMapping(List inputColumnsMapping) { } else if (selectItem instanceof SelectExpressionItem) { SelectExpressionItem expressionItem = (SelectExpressionItem) selectItem; Expression expression = expressionItem.getExpression(); - if (expressionItem.getAlias() != null) { - fieldNames[idx] = expressionItem.getAlias().getName(); + String aliasName = expressionItem.getAlias().getName(); + if (aliasName.startsWith(ESCAPE_IDENTIFIER) + && aliasName.endsWith(ESCAPE_IDENTIFIER)) { + aliasName = aliasName.substring(1, aliasName.length() - 1); + } + fieldNames[idx] = aliasName; } else { if (expression instanceof Column) { fieldNames[idx] = ((Column) expression).getColumnName(); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java index 44b9ca20b7c..a6221e4a277 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java @@ -227,6 +227,13 @@ public Object computeForValue(Expression expression, Object[] inputFields) { Column columnExp = (Column) expression; String columnName = columnExp.getColumnName(); int index = inputRowType.indexOf(columnName, false); + if (index == -1 + && columnName.startsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER) + && columnName.endsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER)) { + columnName = columnName.substring(1, columnName.length() - 1); + index = inputRowType.indexOf(columnName, false); + } + if (index != -1) { return inputFields[index]; } else { @@ -237,11 +244,26 @@ public Object computeForValue(Expression expression, Object[] inputFields) { SeaTunnelRow parRowValues = new SeaTunnelRow(inputFields); Object res = parRowValues; for (int i = 0; i < deep; i++) { + String key = columnNames[i]; if (parDataType instanceof MapType) { - return ((Map) res).get(columnNames[i]); + Map mapValue = ((Map) res); + if (mapValue.containsKey(key)) { + return mapValue.get(key); + } else if (key.startsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER) + && key.endsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER)) { + key = key.substring(1, key.length() - 1); + return mapValue.get(key); + } + return null; } parRowValues = (SeaTunnelRow) res; - int idx = ((SeaTunnelRowType) parDataType).indexOf(columnNames[i], false); + int idx = ((SeaTunnelRowType) parDataType).indexOf(key, false); + if (idx == -1 + && key.startsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER) + && key.endsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER)) { + key = key.substring(1, key.length() - 1); + idx = ((SeaTunnelRowType) parDataType).indexOf(key, false); + } if (idx == -1) { throw new IllegalArgumentException( String.format("can't find field [%s]", fullyQualifiedName)); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java index 45b269bae67..9b527ae8c2f 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java @@ -111,6 +111,13 @@ public SeaTunnelDataType getExpressionType(Expression expression) { Column columnExp = (Column) expression; String columnName = columnExp.getColumnName(); int index = inputRowType.indexOf(columnName, false); + if (index == -1 + && columnName.startsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER) + && columnName.endsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER)) { + columnName = columnName.substring(1, columnName.length() - 1); + index = inputRowType.indexOf(columnName, false); + } + if (index != -1) { return inputRowType.getFieldType(index); } else { @@ -121,7 +128,14 @@ public SeaTunnelDataType getExpressionType(Expression expression) { SeaTunnelRowType parRowType = inputRowType; SeaTunnelDataType filedTypeRes = null; for (int i = 0; i < deep; i++) { - int idx = parRowType.indexOf(columnNames[i], false); + String key = columnNames[i]; + int idx = parRowType.indexOf(key, false); + if (idx == -1 + && key.startsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER) + && key.endsWith(ZetaSQLEngine.ESCAPE_IDENTIFIER)) { + key = key.substring(1, key.length() - 1); + idx = parRowType.indexOf(key, false); + } if (idx == -1) { throw new IllegalArgumentException( String.format("can't find field [%s]", fullyQualifiedName)); diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java index 854fae5cb32..ff253eac210 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java @@ -19,18 +19,22 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.PhysicalColumn; import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TableSchema; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.MapType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Objects; @@ -144,4 +148,136 @@ private CatalogTable getCatalogTable() { new ArrayList<>(), "It has column information."); } + + @Test + public void testEscapeIdentifier() { + String tableName = "test"; + String[] fields = new String[] {"id", "apply"}; + CatalogTable table = + CatalogTableUtil.getCatalogTable( + tableName, + new SeaTunnelRowType( + fields, + new SeaTunnelDataType[] { + BasicType.INT_TYPE, BasicType.STRING_TYPE + })); + ReadonlyConfig config = + ReadonlyConfig.fromMap( + Collections.singletonMap( + "query", + "select id, trim(`apply`) as `apply` from test where `apply` = 'a'")); + SQLTransform sqlTransform = new SQLTransform(config, table); + TableSchema tableSchema = sqlTransform.transformTableSchema(); + SeaTunnelRow result = + sqlTransform.transformRow( + new SeaTunnelRow(new Object[] {Integer.valueOf(1), String.valueOf("a")})); + Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); + Assertions.assertEquals("a", result.getField(1)); + result = + sqlTransform.transformRow( + new SeaTunnelRow(new Object[] {Integer.valueOf(1), String.valueOf("b")})); + Assertions.assertNull(result); + + config = + ReadonlyConfig.fromMap( + Collections.singletonMap( + "query", + "select id, IFNULL(`apply`, '1') as `apply` from test where `apply` = 'a'")); + sqlTransform = new SQLTransform(config, table); + tableSchema = sqlTransform.transformTableSchema(); + result = + sqlTransform.transformRow( + new SeaTunnelRow(new Object[] {Integer.valueOf(1), String.valueOf("a")})); + Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); + Assertions.assertEquals( + BasicType.STRING_TYPE, tableSchema.getColumns().get(1).getDataType()); + Assertions.assertEquals("a", result.getField(1)); + + table = + CatalogTableUtil.getCatalogTable( + tableName, + new SeaTunnelRowType( + fields, + new SeaTunnelDataType[] {BasicType.INT_TYPE, BasicType.LONG_TYPE})); + config = + ReadonlyConfig.fromMap( + Collections.singletonMap( + "query", + "select id, `apply` + 1 as `apply` from test where `apply` > 0")); + sqlTransform = new SQLTransform(config, table); + tableSchema = sqlTransform.transformTableSchema(); + result = + sqlTransform.transformRow( + new SeaTunnelRow(new Object[] {Integer.valueOf(1), Long.valueOf(1)})); + Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); + Assertions.assertEquals(BasicType.LONG_TYPE, tableSchema.getColumns().get(1).getDataType()); + Assertions.assertEquals(Long.valueOf(2), result.getField(1)); + result = + sqlTransform.transformRow( + new SeaTunnelRow(new Object[] {Integer.valueOf(1), Long.valueOf(0)})); + Assertions.assertNull(result); + + table = + CatalogTableUtil.getCatalogTable( + tableName, + new SeaTunnelRowType( + fields, + new SeaTunnelDataType[] { + BasicType.INT_TYPE, + new MapType( + BasicType.STRING_TYPE, BasicType.STRING_TYPE) + })); + config = + ReadonlyConfig.fromMap( + Collections.singletonMap( + "query", + "select id, `apply`.k1 as `apply` from test where `apply`.k1 = 'a'")); + sqlTransform = new SQLTransform(config, table); + tableSchema = sqlTransform.transformTableSchema(); + result = + sqlTransform.transformRow( + new SeaTunnelRow( + new Object[] { + Integer.valueOf(1), Collections.singletonMap("k1", "a") + })); + Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); + Assertions.assertEquals( + BasicType.STRING_TYPE, tableSchema.getColumns().get(1).getDataType()); + Assertions.assertEquals("a", result.getField(1)); + result = + sqlTransform.transformRow( + new SeaTunnelRow( + new Object[] { + Integer.valueOf(1), Collections.singletonMap("k1", "b") + })); + Assertions.assertNull(result); + + table = + CatalogTableUtil.getCatalogTable( + tableName, + new SeaTunnelRowType( + new String[] {"id", "map"}, + new SeaTunnelDataType[] { + BasicType.INT_TYPE, + new MapType( + BasicType.STRING_TYPE, BasicType.STRING_TYPE) + })); + config = + ReadonlyConfig.fromMap( + Collections.singletonMap( + "query", + "select id, map.`apply` as `apply` from test where map.`apply` = 'a'")); + sqlTransform = new SQLTransform(config, table); + tableSchema = sqlTransform.transformTableSchema(); + result = + sqlTransform.transformRow( + new SeaTunnelRow( + new Object[] { + Integer.valueOf(1), Collections.singletonMap("apply", "a") + })); + Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); + Assertions.assertEquals( + BasicType.STRING_TYPE, tableSchema.getColumns().get(1).getDataType()); + Assertions.assertEquals("a", result.getField(1)); + } } From 2fd4eec22aeb3a35c558eeecbc35d35ad217db10 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Mon, 5 Aug 2024 11:35:02 +0800 Subject: [PATCH 018/361] [Fix][Doc] Fix hybrid cluster deployment document display error (#7306) --- docs/en/seatunnel-engine/hybrid-cluster-deployment.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docs/en/seatunnel-engine/hybrid-cluster-deployment.md b/docs/en/seatunnel-engine/hybrid-cluster-deployment.md index c969376f162..60260f91bb0 100644 --- a/docs/en/seatunnel-engine/hybrid-cluster-deployment.md +++ b/docs/en/seatunnel-engine/hybrid-cluster-deployment.md @@ -178,10 +178,6 @@ hazelcast: TCP is the recommended method for use in a standalone SeaTunnel Engine cluster. Alternatively, Hazelcast provides several other service discovery methods. For more details, please refer to [Hazelcast Network](https://docs.hazelcast.com/imdg/4.1/clusters/setting-up-clusters) --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - -sidebar_position: 5 -------------------- ### 5.3 IMap Persistence Configuration From 4f120ff34b1b5cf0536ff88cd9b571eff3b1fdcb Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Mon, 5 Aug 2024 14:34:01 +0800 Subject: [PATCH 019/361] [Improve] Update snapshot version to 2.3.7 (#7305) --- bin/install-plugin.cmd | 4 +-- bin/install-plugin.sh | 4 +-- .../en/seatunnel-engine/download-seatunnel.md | 6 ++-- docs/en/start-v2/kubernetes/kubernetes.mdx | 36 +++++++++---------- docs/en/start-v2/locally/deployment.md | 6 ++-- .../zh/seatunnel-engine/download-seatunnel.md | 8 ++--- docs/zh/start-v2/locally/deployment.md | 6 ++-- pom.xml | 2 +- tools/dependencies/known-dependencies.txt | 8 ++--- 9 files changed, 40 insertions(+), 40 deletions(-) diff --git a/bin/install-plugin.cmd b/bin/install-plugin.cmd index e4d7e27432b..799f1d2fd66 100644 --- a/bin/install-plugin.cmd +++ b/bin/install-plugin.cmd @@ -22,8 +22,8 @@ REM Get seatunnel home set "SEATUNNEL_HOME=%~dp0..\" echo Set SEATUNNEL_HOME to [%SEATUNNEL_HOME%] -REM Connector default version is 2.3.6, you can also choose a custom version. eg: 2.1.2: install-plugin.bat 2.1.2 -set "version=2.3.6" +REM Connector default version is 2.3.7, you can also choose a custom version. eg: 2.1.2: install-plugin.bat 2.1.2 +set "version=2.3.7" if not "%~1"=="" set "version=%~1" REM Create the lib directory diff --git a/bin/install-plugin.sh b/bin/install-plugin.sh index 2766112add6..43d0bcb837a 100755 --- a/bin/install-plugin.sh +++ b/bin/install-plugin.sh @@ -23,8 +23,8 @@ # get seatunnel home SEATUNNEL_HOME=$(cd $(dirname $0);cd ../;pwd) -# connector default version is 2.3.6, you can also choose a custom version. eg: 2.1.2: sh install-plugin.sh 2.1.2 -version=2.3.6 +# connector default version is 2.3.7, you can also choose a custom version. eg: 2.1.2: sh install-plugin.sh 2.1.2 +version=2.3.7 if [ -n "$1" ]; then version="$1" diff --git a/docs/en/seatunnel-engine/download-seatunnel.md b/docs/en/seatunnel-engine/download-seatunnel.md index ffbf833820a..e1ddd88b681 100644 --- a/docs/en/seatunnel-engine/download-seatunnel.md +++ b/docs/en/seatunnel-engine/download-seatunnel.md @@ -21,7 +21,7 @@ Go to the [Seatunnel Download Page](https://seatunnel.apache.org/download) to do Or you can also download it through the terminal. ```shell -export version="2.3.6" +export version="2.3.7" wget "https://archive.apache.org/dist/seatunnel/${version}/apache-seatunnel-${version}-bin.tar.gz" tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" ``` @@ -34,10 +34,10 @@ Starting from the 2.2.0-beta version, the binary package no longer provides the sh bin/install-plugin.sh ``` -If you need a specific connector version, taking 2.3.6 as an example, you need to execute the following command. +If you need a specific connector version, taking 2.3.7 as an example, you need to execute the following command. ```bash -sh bin/install-plugin.sh 2.3.6 +sh bin/install-plugin.sh 2.3.7 ``` Usually you don't need all the connector plugins, so you can specify the plugins you need through configuring `config/plugin_config`, for example, if you only need the `connector-console` plugin, then you can modify the plugin.properties configuration file as follows. diff --git a/docs/en/start-v2/kubernetes/kubernetes.mdx b/docs/en/start-v2/kubernetes/kubernetes.mdx index b40e561ec72..7c5a4ac2795 100644 --- a/docs/en/start-v2/kubernetes/kubernetes.mdx +++ b/docs/en/start-v2/kubernetes/kubernetes.mdx @@ -44,7 +44,7 @@ To run the image with SeaTunnel, first create a `Dockerfile`: ```Dockerfile FROM flink:1.13 -ENV SEATUNNEL_VERSION="2.3.6" +ENV SEATUNNEL_VERSION="2.3.7" ENV SEATUNNEL_HOME="/opt/seatunnel" RUN wget https://dlcdn.apache.org/seatunnel/${SEATUNNEL_VERSION}/apache-seatunnel-${SEATUNNEL_VERSION}-bin.tar.gz @@ -56,13 +56,13 @@ RUN cd ${SEATUNNEL_HOME} && sh bin/install-plugin.sh ${SEATUNNEL_VERSION} Then run the following commands to build the image: ```bash -docker build -t seatunnel:2.3.6-flink-1.13 -f Dockerfile . +docker build -t seatunnel:2.3.7-flink-1.13 -f Dockerfile . ``` -Image `seatunnel:2.3.6-flink-1.13` needs to be present in the host (minikube) so that the deployment can take place. +Image `seatunnel:2.3.7-flink-1.13` needs to be present in the host (minikube) so that the deployment can take place. Load image to minikube via: ```bash -minikube image load seatunnel:2.3.6-flink-1.13 +minikube image load seatunnel:2.3.7-flink-1.13 ``` @@ -72,7 +72,7 @@ minikube image load seatunnel:2.3.6-flink-1.13 ```Dockerfile FROM openjdk:8 -ENV SEATUNNEL_VERSION="2.3.6" +ENV SEATUNNEL_VERSION="2.3.7" ENV SEATUNNEL_HOME="/opt/seatunnel" RUN wget https://dlcdn.apache.org/seatunnel/${SEATUNNEL_VERSION}/apache-seatunnel-${SEATUNNEL_VERSION}-bin.tar.gz @@ -84,13 +84,13 @@ RUN cd ${SEATUNNEL_HOME} && sh bin/install-plugin.sh ${SEATUNNEL_VERSION} Then run the following commands to build the image: ```bash -docker build -t seatunnel:2.3.6 -f Dockerfile . +docker build -t seatunnel:2.3.7 -f Dockerfile . ``` -Image `seatunnel:2.3.6` need to be present in the host (minikube) so that the deployment can take place. +Image `seatunnel:2.3.7` need to be present in the host (minikube) so that the deployment can take place. Load image to minikube via: ```bash -minikube image load seatunnel:2.3.6 +minikube image load seatunnel:2.3.7 ``` @@ -100,7 +100,7 @@ minikube image load seatunnel:2.3.6 ```Dockerfile FROM openjdk:8 -ENV SEATUNNEL_VERSION="2.3.6" +ENV SEATUNNEL_VERSION="2.3.7" ENV SEATUNNEL_HOME="/opt/seatunnel" RUN wget https://dlcdn.apache.org/seatunnel/${SEATUNNEL_VERSION}/apache-seatunnel-${SEATUNNEL_VERSION}-bin.tar.gz @@ -112,13 +112,13 @@ RUN cd ${SEATUNNEL_HOME} && sh bin/install-plugin.sh ${SEATUNNEL_VERSION} Then run the following commands to build the image: ```bash -docker build -t seatunnel:2.3.6 -f Dockerfile . +docker build -t seatunnel:2.3.7 -f Dockerfile . ``` -Image `seatunnel:2.3.6` needs to be present in the host (minikube) so that the deployment can take place. +Image `seatunnel:2.3.7` needs to be present in the host (minikube) so that the deployment can take place. Load image to minikube via: ```bash -minikube image load seatunnel:2.3.6 +minikube image load seatunnel:2.3.7 ``` @@ -191,7 +191,7 @@ none ]}> -In this guide we will use [seatunnel.streaming.conf](https://github.com/apache/seatunnel/blob/2.3.6-release/config/v2.streaming.conf.template): +In this guide we will use [seatunnel.streaming.conf](https://github.com/apache/seatunnel/blob/2.3.7-release/config/v2.streaming.conf.template): ```conf env { @@ -245,7 +245,7 @@ kind: FlinkDeployment metadata: name: seatunnel-flink-streaming-example spec: - image: seatunnel:2.3.6-flink-1.13 + image: seatunnel:2.3.7-flink-1.13 flinkVersion: v1_13 flinkConfiguration: taskmanager.numberOfTaskSlots: "2" @@ -291,7 +291,7 @@ kubectl apply -f seatunnel-flink.yaml -In this guide we will use [seatunnel.streaming.conf](https://github.com/apache/seatunnel/blob/2.3.6-release/config/v2.streaming.conf.template): +In this guide we will use [seatunnel.streaming.conf](https://github.com/apache/seatunnel/blob/2.3.7-release/config/v2.streaming.conf.template): ```conf env { @@ -334,7 +334,7 @@ metadata: spec: containers: - name: seatunnel - image: seatunnel:2.3.6 + image: seatunnel:2.3.7 command: ["/bin/sh","-c","/opt/seatunnel/bin/seatunnel.sh --config /data/seatunnel.streaming.conf -e local"] resources: limits: @@ -366,7 +366,7 @@ kubectl apply -f seatunnel.yaml -In this guide we will use [seatunnel.streaming.conf](https://github.com/apache/seatunnel/blob/2.3.6-release/config/v2.streaming.conf.template): +In this guide we will use [seatunnel.streaming.conf](https://github.com/apache/seatunnel/blob/2.3.7-release/config/v2.streaming.conf.template): ```conf env { @@ -524,7 +524,7 @@ spec: spec: containers: - name: seatunnel - image: seatunnel:2.3.6 + image: seatunnel:2.3.7 imagePullPolicy: IfNotPresent ports: - containerPort: 5801 diff --git a/docs/en/start-v2/locally/deployment.md b/docs/en/start-v2/locally/deployment.md index 69cf5164e95..0d5f0e26d11 100644 --- a/docs/en/start-v2/locally/deployment.md +++ b/docs/en/start-v2/locally/deployment.md @@ -21,7 +21,7 @@ Visit the [SeaTunnel Download Page](https://seatunnel.apache.org/download) to do Or you can also download it through the terminal: ```shell -export version="2.3.6" +export version="2.3.7" wget "https://archive.apache.org/dist/seatunnel/${version}/apache-seatunnel-${version}-bin.tar.gz" tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" ``` @@ -34,10 +34,10 @@ Starting from the 2.2.0-beta version, the binary package no longer provides the sh bin/install-plugin.sh ``` -If you need a specific connector version, taking 2.3.6 as an example, you need to execute the following command: +If you need a specific connector version, taking 2.3.7 as an example, you need to execute the following command: ```bash -sh bin/install-plugin.sh 2.3.6 +sh bin/install-plugin.sh 2.3.7 ``` Usually you don't need all connector plugins, so you can specify the plugins you need through configuring `config/plugin_config`. For example, if you only need the `connector-console` plugin, you can modify the plugin.properties configuration file as follows: diff --git a/docs/zh/seatunnel-engine/download-seatunnel.md b/docs/zh/seatunnel-engine/download-seatunnel.md index c108f4812a3..74281d0648f 100644 --- a/docs/zh/seatunnel-engine/download-seatunnel.md +++ b/docs/zh/seatunnel-engine/download-seatunnel.md @@ -21,7 +21,7 @@ import TabItem from '@theme/TabItem'; 或者您也可以通过终端下载 ```shell -export version="2.3.6" +export version="2.3.7" wget "https://archive.apache.org/dist/seatunnel/${version}/apache-seatunnel-${version}-bin.tar.gz" tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" ``` @@ -31,13 +31,13 @@ tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" 从2.2.0-beta版本开始,二进制包不再默认提供连接器依赖,因此在第一次使用它时,您需要执行以下命令来安装连接器:(当然,您也可以从 [Apache Maven Repository](https://repo.maven.apache.org/maven2/org/apache/seatunnel/) 手动下载连接器,然后将其移动至`connectors/seatunnel`目录下)。 ```bash -sh bin/install-plugin.sh 2.3.6 +sh bin/install-plugin.sh 2.3.7 ``` -如果您需要指定的连接器版本,以2.3.6为例,您需要执行如下命令 +如果您需要指定的连接器版本,以2.3.7为例,您需要执行如下命令 ```bash -sh bin/install-plugin.sh 2.3.6 +sh bin/install-plugin.sh 2.3.7 ``` 通常您并不需要所有的连接器插件,所以您可以通过配置`config/plugin_config`来指定您所需要的插件,例如,您只需要`connector-console`插件,那么您可以修改plugin.properties配置文件如下 diff --git a/docs/zh/start-v2/locally/deployment.md b/docs/zh/start-v2/locally/deployment.md index 9fa70f16040..167abeaeaab 100644 --- a/docs/zh/start-v2/locally/deployment.md +++ b/docs/zh/start-v2/locally/deployment.md @@ -21,7 +21,7 @@ import TabItem from '@theme/TabItem'; 或者您也可以通过终端下载: ```shell -export version="2.3.6" +export version="2.3.7" wget "https://archive.apache.org/dist/seatunnel/${version}/apache-seatunnel-${version}-bin.tar.gz" tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" ``` @@ -34,10 +34,10 @@ tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" sh bin/install-plugin.sh ``` -如果您需要指定的连接器版本,以2.3.6为例,您需要执行如下命令: +如果您需要指定的连接器版本,以2.3.7为例,您需要执行如下命令: ```bash -sh bin/install-plugin.sh 2.3.6 +sh bin/install-plugin.sh 2.3.7 ``` 通常您并不需要所有的连接器插件,可以通过配置`config/plugin_config`来指定您所需要的插件,例如,您只需要`connector-console`插件,那么您可以修改plugin.properties配置文件如下: diff --git a/pom.xml b/pom.xml index 41854d78fce..7ca8e0652f6 100644 --- a/pom.xml +++ b/pom.xml @@ -56,7 +56,7 @@ - 2.3.6-SNAPSHOT + 2.3.7-SNAPSHOT 2.1.1 UTF-8 1.8 diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt index 8532f7cba43..161134511c8 100755 --- a/tools/dependencies/known-dependencies.txt +++ b/tools/dependencies/known-dependencies.txt @@ -24,9 +24,9 @@ protostuff-collectionschema-1.8.0.jar protostuff-core-1.8.0.jar protostuff-runtime-1.8.0.jar scala-library-2.12.15.jar -seatunnel-jackson-2.3.6-SNAPSHOT-optional.jar -seatunnel-guava-2.3.6-SNAPSHOT-optional.jar -seatunnel-hazelcast-shade-2.3.6-SNAPSHOT-optional.jar +seatunnel-jackson-2.3.7-SNAPSHOT-optional.jar +seatunnel-guava-2.3.7-SNAPSHOT-optional.jar +seatunnel-hazelcast-shade-2.3.7-SNAPSHOT-optional.jar slf4j-api-1.7.25.jar jsqlparser-4.5.jar animal-sniffer-annotations-1.17.jar @@ -44,4 +44,4 @@ accessors-smart-2.4.7.jar asm-9.1.jar avro-1.11.1.jar groovy-4.0.16.jar -seatunnel-janino-2.3.6-SNAPSHOT-optional.jar \ No newline at end of file +seatunnel-janino-2.3.7-SNAPSHOT-optional.jar \ No newline at end of file From 9d56cc33b4f62316ed694c63a088fd50c1f51628 Mon Sep 17 00:00:00 2001 From: virvle <55478661+virvle@users.noreply.github.com> Date: Mon, 5 Aug 2024 17:51:26 +0800 Subject: [PATCH 020/361] Update Oracle-CDC.md (#7285) --- .github/workflows/update_build_status.yml | 2 +- docs/en/connector-v2/source/Oracle-CDC.md | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/.github/workflows/update_build_status.yml b/.github/workflows/update_build_status.yml index 05cf4914a25..03718ba7472 100644 --- a/.github/workflows/update_build_status.yml +++ b/.github/workflows/update_build_status.yml @@ -105,4 +105,4 @@ jobs: } } } - } + } \ No newline at end of file diff --git a/docs/en/connector-v2/source/Oracle-CDC.md b/docs/en/connector-v2/source/Oracle-CDC.md index cedbda141f6..5d22aa1c4ae 100644 --- a/docs/en/connector-v2/source/Oracle-CDC.md +++ b/docs/en/connector-v2/source/Oracle-CDC.md @@ -91,6 +91,8 @@ GRANT SELECT ON V_$ARCHIVED_LOG TO logminer_user; GRANT SELECT ON V_$ARCHIVE_DEST_STATUS TO logminer_user; GRANT EXECUTE ON DBMS_LOGMNR TO logminer_user; GRANT EXECUTE ON DBMS_LOGMNR_D TO logminer_user; +GRANT SELECT ANY TRANSACTION TO logminer_user; +GRANT SELECT ON V_$TRANSACTION TO logminer_user; ``` ##### Oracle 11g is not supported From 73632bad2b93e6879c673e0f00bb83035aa51408 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Tue, 6 Aug 2024 10:08:45 +0800 Subject: [PATCH 021/361] [Hotfix][Zeta] Fix task cannot be stopped when system is busy (#7292) --- .../engine/server/TaskExecutionService.java | 22 ++++++++++++++----- .../engine/server/master/JobMaster.java | 17 +++++++++++++- 2 files changed, 32 insertions(+), 7 deletions(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java index 94f0fa324fc..00716f2c90a 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java @@ -55,6 +55,7 @@ import com.google.common.collect.Lists; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.hazelcast.core.OperationTimeoutException; import com.hazelcast.instance.impl.NodeState; import com.hazelcast.internal.metrics.DynamicMetricsProvider; import com.hazelcast.internal.metrics.MetricDescriptor; @@ -624,9 +625,12 @@ private void updateMetricsContextInImap() { }); }); if (localMap.size() > 0) { + boolean lockedIMap = false; try { - if (!metricsImap.tryLock( - Constant.IMAP_RUNNING_JOB_METRICS_KEY, 5, TimeUnit.SECONDS)) { + lockedIMap = + metricsImap.tryLock( + Constant.IMAP_RUNNING_JOB_METRICS_KEY, 5, TimeUnit.SECONDS); + if (!lockedIMap) { logger.warning("try lock failed in update metrics"); return; } @@ -640,10 +644,16 @@ private void updateMetricsContextInImap() { "The Imap acquisition failed due to the hazelcast node being offline or restarted, and will be retried next time", e); } finally { - try { - metricsImap.unlock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); - } catch (Throwable e) { - logger.warning("unlock imap failed in update metrics", e); + if (lockedIMap) { + boolean unLockedIMap = false; + while (!unLockedIMap) { + try { + metricsImap.unlock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); + unLockedIMap = true; + } catch (OperationTimeoutException e) { + logger.warning("unlock imap failed in update metrics", e); + } + } } } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java index aa74460b056..888114bec95 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java @@ -72,6 +72,7 @@ import com.hazelcast.cluster.Address; import com.hazelcast.core.HazelcastInstanceNotActiveException; +import com.hazelcast.core.OperationTimeoutException; import com.hazelcast.flakeidgen.FlakeIdGenerator; import com.hazelcast.internal.serialization.Data; import com.hazelcast.jet.datamodel.Tuple2; @@ -674,8 +675,12 @@ public void removeMetricsContext( if ((pipelineStatus.equals(PipelineStatus.FINISHED) && !checkpointManager.isPipelineSavePointEnd(pipelineLocation)) || pipelineStatus.equals(PipelineStatus.CANCELED)) { + + boolean lockedIMap = false; try { metricsImap.lock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); + lockedIMap = true; + HashMap centralMap = metricsImap.get(Constant.IMAP_RUNNING_JOB_METRICS_KEY); if (centralMap != null) { @@ -693,7 +698,17 @@ public void removeMetricsContext( metricsImap.put(Constant.IMAP_RUNNING_JOB_METRICS_KEY, centralMap); } } finally { - metricsImap.unlock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); + if (lockedIMap) { + boolean unLockedIMap = false; + while (!unLockedIMap) { + try { + metricsImap.unlock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); + unLockedIMap = true; + } catch (OperationTimeoutException e) { + LOGGER.warning("unlock imap failed in update metrics", e); + } + } + } } } } From c94ea325b7a70ebbfc3eabdaa5025c6d0292384c Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Tue, 6 Aug 2024 11:23:52 +0800 Subject: [PATCH 022/361] [Fix][Doc] Fix miss sink-options-placeholders.md in sidebars (#7310) --- docs/sidebars.js | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/sidebars.js b/docs/sidebars.js index 1a9a1cf6ec5..33655a48171 100644 --- a/docs/sidebars.js +++ b/docs/sidebars.js @@ -90,6 +90,7 @@ const sidebars = { "concept/connector-v2-features", 'concept/schema-feature', 'concept/JobEnvConfig', + 'concept/sink-options-placeholders', 'concept/sql-config', 'concept/speed-limit', 'concept/event-listener' From bb2c912404fac13e829044c60259f2ab27bff3a1 Mon Sep 17 00:00:00 2001 From: Guangdong Liu <804167098@qq.com> Date: Tue, 6 Aug 2024 21:46:12 +0800 Subject: [PATCH 023/361] [Fix][Doris] Fix the abnormality of deleting data in CDC scenario. (#7315) --- .../serialize/SeaTunnelRowSerializer.java | 77 ++++---- .../doris/sink/writer/DorisStreamLoad.java | 32 +--- .../connector-doris-e2e/pom.xml | 14 ++ .../e2e/connector/doris/DorisCDCSinkIT.java | 171 ++++++++++++++++-- .../src/test/resources/ddl/mysql_cdc.sql | 38 ++++ .../test/resources/docker/server-gtids/my.cnf | 65 +++++++ .../src/test/resources/docker/setup.sql | 28 +++ .../write-cdc-changelog-to-doris.conf | 18 +- 8 files changed, 356 insertions(+), 87 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/ddl/mysql_cdc.sql create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/server-gtids/my.cnf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/setup.sql diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java index 0c5b9c0c420..0e67257a32e 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonGenerator; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.seatunnel.api.serialization.SerializationSchema; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -29,6 +30,7 @@ import org.apache.seatunnel.format.text.TextSerializationSchema; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -42,6 +44,7 @@ public class SeaTunnelRowSerializer implements DorisSerializer { private final SeaTunnelRowType seaTunnelRowType; private final String fieldDelimiter; private final boolean enableDelete; + private final SerializationSchema serialize; public SeaTunnelRowSerializer( String type, @@ -49,32 +52,46 @@ public SeaTunnelRowSerializer( String fieldDelimiter, boolean enableDelete) { this.type = type; - this.seaTunnelRowType = seaTunnelRowType; this.fieldDelimiter = fieldDelimiter; this.enableDelete = enableDelete; - } + List fieldNames = new ArrayList<>(Arrays.asList(seaTunnelRowType.getFieldNames())); + List> fieldTypes = + new ArrayList<>(Arrays.asList(seaTunnelRowType.getFieldTypes())); + + if (enableDelete) { + fieldNames.add(LoadConstants.DORIS_DELETE_SIGN); + fieldTypes.add(STRING_TYPE); + } - public byte[] buildJsonString(SeaTunnelRow row, SeaTunnelRowType seaTunnelRowType) - throws IOException { + this.seaTunnelRowType = + new SeaTunnelRowType( + fieldNames.toArray(new String[0]), + fieldTypes.toArray(new SeaTunnelDataType[0])); - JsonSerializationSchema jsonSerializationSchema = - new JsonSerializationSchema(seaTunnelRowType, NULL_VALUE); - ObjectMapper mapper = jsonSerializationSchema.getMapper(); - mapper.configure(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, true); - return jsonSerializationSchema.serialize(row); + if (JSON.equals(type)) { + JsonSerializationSchema jsonSerializationSchema = + new JsonSerializationSchema(this.seaTunnelRowType, NULL_VALUE); + ObjectMapper mapper = jsonSerializationSchema.getMapper(); + mapper.configure(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, true); + this.serialize = jsonSerializationSchema; + } else { + this.serialize = + TextSerializationSchema.builder() + .seaTunnelRowType(this.seaTunnelRowType) + .delimiter(fieldDelimiter) + .nullValue(NULL_VALUE) + .build(); + } } - public byte[] buildCSVString(SeaTunnelRow row, SeaTunnelRowType seaTunnelRowType) - throws IOException { + public byte[] buildJsonString(SeaTunnelRow row) { + + return serialize.serialize(row); + } - TextSerializationSchema build = - TextSerializationSchema.builder() - .seaTunnelRowType(seaTunnelRowType) - .delimiter(fieldDelimiter) - .nullValue(NULL_VALUE) - .build(); + public byte[] buildCSVString(SeaTunnelRow row) { - return build.serialize(row); + return serialize.serialize(row); } public String parseDeleteSign(RowKind rowKind) { @@ -93,29 +110,17 @@ public void open() throws IOException {} @Override public byte[] serialize(SeaTunnelRow seaTunnelRow) throws IOException { - List fieldNames = Arrays.asList(seaTunnelRowType.getFieldNames()); - List> fieldTypes = Arrays.asList(seaTunnelRowType.getFieldTypes()); - if (enableDelete) { - SeaTunnelRow seaTunnelRowEnableDelete = seaTunnelRow.copy(); - seaTunnelRowEnableDelete.setField( - seaTunnelRow.getFields().length, parseDeleteSign(seaTunnelRow.getRowKind())); - fieldNames.add(LoadConstants.DORIS_DELETE_SIGN); - fieldTypes.add(STRING_TYPE); + + List newFields = new ArrayList<>(Arrays.asList(seaTunnelRow.getFields())); + newFields.add(parseDeleteSign(seaTunnelRow.getRowKind())); + seaTunnelRow = new SeaTunnelRow(newFields.toArray()); } if (JSON.equals(type)) { - return buildJsonString( - seaTunnelRow, - new SeaTunnelRowType( - fieldNames.toArray(new String[0]), - fieldTypes.toArray(new SeaTunnelDataType[0]))); + return buildJsonString(seaTunnelRow); } else if (CSV.equals(type)) { - return buildCSVString( - seaTunnelRow, - new SeaTunnelRowType( - fieldNames.toArray(new String[0]), - fieldTypes.toArray(new SeaTunnelDataType[0]))); + return buildCSVString(seaTunnelRow); } else { throw new IllegalArgumentException("The type " + type + " is not supported!"); } diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisStreamLoad.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisStreamLoad.java index eadcf94cd56..40b75aedc61 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisStreamLoad.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisStreamLoad.java @@ -17,7 +17,10 @@ package org.apache.seatunnel.connectors.doris.sink.writer; +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; + import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.common.utils.JsonUtils; import org.apache.seatunnel.connectors.doris.config.DorisConfig; import org.apache.seatunnel.connectors.doris.exception.DorisConnectorErrorCode; import org.apache.seatunnel.connectors.doris.exception.DorisConnectorException; @@ -31,9 +34,9 @@ import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.util.EntityUtils; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; import java.io.IOException; @@ -64,23 +67,23 @@ public class DorisStreamLoad implements Serializable { private static final String ABORT_URL_PATTERN = "http://%s/api/%s/_stream_load_2pc"; private static final String JOB_EXIST_FINISHED = "FINISHED"; private final String loadUrlStr; - private final String hostPort; + @Getter private final String hostPort; private final String abortUrlStr; private final String user; private final String passwd; - private final String db; + @Getter private final String db; private final String table; private final boolean enable2PC; private final boolean enableDelete; private final Properties streamLoadProp; private final RecordStream recordStream; - private Future pendingLoadFuture; + @Getter private Future pendingLoadFuture; private final CloseableHttpClient httpClient; private final ExecutorService executorService; private volatile boolean loadBatchFirstRecord; private volatile boolean loading = false; private String label; - private long recordCount = 0; + @Getter private long recordCount = 0; public DorisStreamLoad( String hostPort, @@ -115,18 +118,6 @@ public DorisStreamLoad( loadBatchFirstRecord = true; } - public String getDb() { - return db; - } - - public String getHostPort() { - return hostPort; - } - - public Future getPendingLoadFuture() { - return pendingLoadFuture; - } - public void abortPreCommit(String labelSuffix, long chkID) throws Exception { long startChkID = chkID; log.info("abort for labelSuffix {}. start chkId {}.", labelSuffix, chkID); @@ -196,10 +187,6 @@ public void writeRecord(byte[] record) throws IOException { recordCount++; } - public long getRecordCount() { - return recordCount; - } - public String getLoadFailedMsg() { if (!loading) { return null; @@ -300,10 +287,9 @@ public void abortTransaction(long txnID) throws Exception { "Fail to abort transaction " + txnID + " with url " + abortUrlStr); } - ObjectMapper mapper = new ObjectMapper(); String loadResult = EntityUtils.toString(response.getEntity()); Map res = - mapper.readValue(loadResult, new TypeReference>() {}); + JsonUtils.parseObject(loadResult, new TypeReference>() {}); if (!LoadStatus.SUCCESS.equals(res.get("status"))) { if (ResponseUtil.isCommitted(res.get("msg"))) { throw new DorisConnectorException( diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/pom.xml index af85d92acef..7a3008adb3a 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/pom.xml @@ -49,5 +49,19 @@ ${mysql.version} test + + org.apache.seatunnel + connector-cdc-mysql + ${project.version} + test-jar + test + + + + org.testcontainers + mysql + ${testcontainer.version} + test + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisCDCSinkIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisCDCSinkIT.java index 9afa91d4e81..33108b8b8eb 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisCDCSinkIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisCDCSinkIT.java @@ -17,16 +17,27 @@ package org.apache.seatunnel.e2e.connector.doris; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlContainer; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlVersion; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.UniqueDatabase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.TestTemplate; import org.testcontainers.containers.Container; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerLoggerFactory; import lombok.extern.slf4j.Slf4j; +import java.sql.Connection; +import java.sql.DriverManager; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; @@ -34,11 +45,18 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.awaitility.Awaitility.await; + @Slf4j -@Disabled("we need resolve the issue of network between containers") +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK}, + disabledReason = "Currently SPARK do not support cdc") public class DorisCDCSinkIT extends AbstractDorisIT { private static final String DATABASE = "test"; @@ -60,34 +78,121 @@ public class DorisCDCSinkIT extends AbstractDorisIT { + "\"replication_allocation\" = \"tag.location.default: 1\"" + ")"; + // mysql + private static final String MYSQL_HOST = "mysql_cdc_e2e"; + private static final String MYSQL_USER_NAME = "mysqluser"; + private static final String MYSQL_USER_PASSWORD = "mysqlpw"; + private static final String MYSQL_DATABASE = "mysql_cdc"; + private static final MySqlContainer MYSQL_CONTAINER = createMySqlContainer(MySqlVersion.V8_0); + private static final String SOURCE_TABLE = "mysql_cdc_e2e_source_table"; + + @TestContainerExtension + protected final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Doris-CDC/lib && cd /tmp/seatunnel/plugins/Doris-CDC/lib && wget " + + driverUrl()); + Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); + }; + + private final UniqueDatabase inventoryDatabase = + new UniqueDatabase( + MYSQL_CONTAINER, MYSQL_DATABASE, "mysqluser", "mysqlpw", MYSQL_DATABASE); + + private static MySqlContainer createMySqlContainer(MySqlVersion version) { + return new MySqlContainer(version) + .withConfigurationOverride("docker/server-gtids/my.cnf") + .withSetupSQL("docker/setup.sql") + .withNetwork(NETWORK) + .withNetworkAliases(MYSQL_HOST) + .withDatabaseName(MYSQL_DATABASE) + .withUsername(MYSQL_USER_NAME) + .withPassword(MYSQL_USER_PASSWORD) + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger("mysql-docker-image"))); + } + + private String driverUrl() { + return "https://repo1.maven.org/maven2/com/mysql/mysql-connector-j/8.0.32/mysql-connector-j-8.0.32.jar"; + } + @BeforeAll public void init() { + log.info("The second stage: Starting Mysql containers..."); + Startables.deepStart(Stream.of(MYSQL_CONTAINER)).join(); + log.info("Mysql Containers are started"); + inventoryDatabase.createAndInitialize(); + log.info("Mysql ddl execution is complete"); initializeJdbcTable(); } @TestTemplate public void testDorisCDCSink(TestContainer container) throws Exception { - Container.ExecResult execResult = - container.executeJob("/write-cdc-changelog-to-doris.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); + + clearTable(DATABASE, SINK_TABLE); + CompletableFuture.supplyAsync( + () -> { + try { + container.executeJob("/write-cdc-changelog-to-doris.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + return null; + }); String sinkSql = String.format("select * from %s.%s", DATABASE, SINK_TABLE); - Set> actual = new HashSet<>(); - try (Statement sinkStatement = jdbcConnection.createStatement()) { - ResultSet sinkResultSet = sinkStatement.executeQuery(sinkSql); - while (sinkResultSet.next()) { - List row = - Arrays.asList( - sinkResultSet.getLong("uuid"), - sinkResultSet.getString("name"), - sinkResultSet.getInt("score")); - actual.add(row); - } - } + Set> expected = - Stream.>of(Arrays.asList(1L, "A_1", 100), Arrays.asList(3L, "C", 100)) + Stream.>of( + Arrays.asList(1L, "Alice", 95), Arrays.asList(2L, "Bob", 88)) .collect(Collectors.toSet()); - Assertions.assertIterableEquals(expected, actual); + + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Set> actual = new HashSet<>(); + try (Statement sinkStatement = jdbcConnection.createStatement()) { + ResultSet sinkResultSet = sinkStatement.executeQuery(sinkSql); + while (sinkResultSet.next()) { + List row = + Arrays.asList( + sinkResultSet.getLong("uuid"), + sinkResultSet.getString("name"), + sinkResultSet.getInt("score")); + actual.add(row); + } + } + Assertions.assertIterableEquals(expected, actual); + }); + + executeSql("DELETE FROM " + MYSQL_DATABASE + "." + SOURCE_TABLE + " WHERE uuid = 1"); + + Set> expectedAfterDelete = + Stream.>of(Arrays.asList(2L, "Bob", 88)).collect(Collectors.toSet()); + + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Set> actual = new HashSet<>(); + try (Statement sinkStatement = jdbcConnection.createStatement()) { + ResultSet sinkResultSet = sinkStatement.executeQuery(sinkSql); + while (sinkResultSet.next()) { + List row = + Arrays.asList( + sinkResultSet.getLong("uuid"), + sinkResultSet.getString("name"), + sinkResultSet.getInt("score")); + actual.add(row); + } + } + Assertions.assertIterableEquals(expectedAfterDelete, actual); + }); + executeSql( + "INSERT INTO " + MYSQL_DATABASE + "." + SOURCE_TABLE + " VALUES (1, 'Alice', 95)"); } private void initializeJdbcTable() { @@ -100,4 +205,32 @@ private void initializeJdbcTable() { throw new RuntimeException("Initializing table failed!", e); } } + + private void executeDorisSql(String sql) { + try (Statement statement = jdbcConnection.createStatement()) { + statement.execute(sql); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private Connection getJdbcConnection() throws SQLException { + return DriverManager.getConnection( + MYSQL_CONTAINER.getJdbcUrl(), + MYSQL_CONTAINER.getUsername(), + MYSQL_CONTAINER.getPassword()); + } + + // Execute SQL + private void executeSql(String sql) { + try (Connection connection = getJdbcConnection()) { + connection.createStatement().execute(sql); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private void clearTable(String database, String tableName) { + executeDorisSql("truncate table " + database + "." + tableName); + } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/ddl/mysql_cdc.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/ddl/mysql_cdc.sql new file mode 100644 index 00000000000..638da2981b3 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/ddl/mysql_cdc.sql @@ -0,0 +1,38 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: inventory +-- ---------------------------------------------------------------------------------------------------------------- +CREATE DATABASE IF NOT EXISTS `mysql_cdc`; + +use mysql_cdc; +-- Create a mysql data source table +CREATE TABLE IF NOT EXISTS `mysql_cdc`.`mysql_cdc_e2e_source_table` ( + `uuid` BIGINT, + `name` VARCHAR(128), + `score` INT, + PRIMARY KEY (`uuid`) +) ENGINE=InnoDB; + + + +truncate table `mysql_cdc`.`mysql_cdc_e2e_source_table`; + +INSERT INTO `mysql_cdc`.`mysql_cdc_e2e_source_table` (uuid, name, score) VALUES +(1, 'Alice', 95), +(2, 'Bob', 88); \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/server-gtids/my.cnf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/server-gtids/my.cnf new file mode 100644 index 00000000000..a390897885d --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/server-gtids/my.cnf @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# For advice on how to change settings please see +# http://dev.mysql.com/doc/refman/5.7/en/server-configuration-defaults.html + +[mysqld] +# +# Remove leading # and set to the amount of RAM for the most important data +# cache in MySQL. Start at 70% of total RAM for dedicated server, else 10%. +# innodb_buffer_pool_size = 128M +# +# Remove leading # to turn on a very important data integrity option: logging +# changes to the binary log between backups. +# log_bin +# +# Remove leading # to set options mainly useful for reporting servers. +# The server defaults are faster for transactions and fast SELECTs. +# Adjust sizes as needed, experiment to find the optimal values. +# join_buffer_size = 128M +# sort_buffer_size = 2M +# read_rnd_buffer_size = 2M +skip-host-cache +skip-name-resolve +#datadir=/var/lib/mysql +#socket=/var/lib/mysql/mysql.sock +secure-file-priv=/var/lib/mysql +user=mysql + +# Disabling symbolic-links is recommended to prevent assorted security risks +symbolic-links=0 + +#log-error=/var/log/mysqld.log +#pid-file=/var/run/mysqld/mysqld.pid + +# ---------------------------------------------- +# Enable the binlog for replication & CDC +# ---------------------------------------------- + +# Enable binary replication log and set the prefix, expiration, and log format. +# The prefix is arbitrary, expiration can be short for integration tests but would +# be longer on a production system. Row-level info is required for ingest to work. +# Server ID is required, but this will vary on production systems +server-id = 223344 +log_bin = mysql-bin +expire_logs_days = 1 +binlog_format = row + +# enable gtid mode +gtid_mode = on +enforce_gtid_consistency = on \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/setup.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/setup.sql new file mode 100644 index 00000000000..429061558ba --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/docker/setup.sql @@ -0,0 +1,28 @@ +-- +-- 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. +-- + +-- In production you would almost certainly limit the replication user must be on the follower (slave) machine, +-- to prevent other clients accessing the log from other machines. For example, 'replicator'@'follower.acme.com'. +-- However, in this database we'll grant 2 users different privileges: +-- +-- 1) 'mysqluser' - all privileges +-- 2) 'st_user_source' - all privileges required by the snapshot reader AND binlog reader (used for testing) +-- +GRANT ALL PRIVILEGES ON *.* TO 'mysqluser'@'%'; + +CREATE USER 'st_user_source' IDENTIFIED BY 'mysqlpw'; +GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT, DROP, LOCK TABLES ON *.* TO 'st_user_source'@'%'; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/write-cdc-changelog-to-doris.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/write-cdc-changelog-to-doris.conf index d4d4e69f9d6..7e811c709b3 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/write-cdc-changelog-to-doris.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/write-cdc-changelog-to-doris.conf @@ -17,23 +17,24 @@ env { parallelism = 1 - job.mode = "BATCH" + job.mode = "STREAMING" + checkpoint.interval = 5000 } source { MySQL-CDC { parallelism = 1 - server-id = 5656 - username = "root" - password = "Bigdata2023@" - table-names = ["test.e2e_table_sink"] - base-url = "jdbc:mysql://119.3.230.145:56725/test" + server-id = 5652 + username = "st_user_source" + password = "mysqlpw" + table-names = ["mysql_cdc.mysql_cdc_e2e_source_table"] + base-url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" } } sink { Doris { - fenodes = "10.16.10.14:8234" + fenodes = "doris_e2e:8030" username = root password = "" database = "test" @@ -43,8 +44,7 @@ sink { sink.enable-delete = "true" doris.config { format = "csv" - "column_separator" = "\\x01" - "line_delimiter" = "\\x01" + "column_separator" = "," } } } \ No newline at end of file From f0cefbeb4ac856f1eca17f068b2549d750f321ee Mon Sep 17 00:00:00 2001 From: Arin <136636751+asapekia@users.noreply.github.com> Date: Tue, 6 Aug 2024 19:23:38 +0530 Subject: [PATCH 024/361] [Feature] [Activemq] Added activemq sink (#7251) --- .../workflows/labeler/label-scope-conf.yml | 5 + config/plugin_config | 1 + docs/en/connector-v2/sink/Activemq.md | 123 +++++++++ plugin-mapping.properties | 1 + .../connector-activemq/pom.xml | 65 +++++ .../activemq/client/ActivemqClient.java | 156 +++++++++++ .../activemq/config/ActivemqConfig.java | 242 ++++++++++++++++++ .../exception/ActivemqConnectorErrorCode.java | 51 ++++ .../exception/ActivemqConnectorException.java | 36 +++ .../seatunnel/activemq/sink/ActivemqSink.java | 48 ++++ .../activemq/sink/ActivemqSinkFactory.java | 83 ++++++ .../activemq/sink/ActivemqSinkWriter.java | 56 ++++ .../activemq/ActivemqFactoryTest.java | 31 +++ seatunnel-connectors-v2/pom.xml | 1 + seatunnel-dist/pom.xml | 7 + .../connector-activemq-e2e/pom.xml | 60 +++++ .../e2e/connector/activemq/ActivemqIT.java | 118 +++++++++ .../src/test/resources/e2e.json | 100 ++++++++ .../test/resources/fake_source_to_sink.conf | 117 +++++++++ .../resources/localfile_source_to_sink.conf | 117 +++++++++ .../seatunnel-connector-v2-e2e/pom.xml | 1 + 21 files changed, 1419 insertions(+) create mode 100644 docs/en/connector-v2/sink/Activemq.md create mode 100644 seatunnel-connectors-v2/connector-activemq/pom.xml create mode 100644 seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/client/ActivemqClient.java create mode 100644 seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/config/ActivemqConfig.java create mode 100644 seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorErrorCode.java create mode 100644 seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorException.java create mode 100644 seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSink.java create mode 100644 seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkFactory.java create mode 100644 seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkWriter.java create mode 100644 seatunnel-connectors-v2/connector-activemq/src/test/java/org/apache/seatunnel/connectors/seatunnel/activemq/ActivemqFactoryTest.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/pom.xml create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/activemq/ActivemqIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/e2e.json create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/fake_source_to_sink.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/localfile_source_to_sink.conf diff --git a/.github/workflows/labeler/label-scope-conf.yml b/.github/workflows/labeler/label-scope-conf.yml index 7db40f5ec5d..599ed649396 100644 --- a/.github/workflows/labeler/label-scope-conf.yml +++ b/.github/workflows/labeler/label-scope-conf.yml @@ -252,6 +252,11 @@ Milvus: - changed-files: - any-glob-to-any-file: seatunnel-connectors-v2/connector-milvus/** - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(milvus)/**' +activemq: + - all: + - changed-files: + - any-glob-to-any-file: seatunnel-connectors-v2/connector-activemq/** + - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(activemq)/**' Zeta Rest API: - changed-files: - any-glob-to-any-file: seatunnel-engine/**/server/rest/** diff --git a/config/plugin_config b/config/plugin_config index d80d2e6ab06..f6549168d6d 100644 --- a/config/plugin_config +++ b/config/plugin_config @@ -86,4 +86,5 @@ connector-rocketmq connector-tdengine connector-web3j connector-milvus +connector-activemq --end-- \ No newline at end of file diff --git a/docs/en/connector-v2/sink/Activemq.md b/docs/en/connector-v2/sink/Activemq.md new file mode 100644 index 00000000000..3151585d082 --- /dev/null +++ b/docs/en/connector-v2/sink/Activemq.md @@ -0,0 +1,123 @@ +# Activemq + +> Activemq sink connector + +## Description + +Used to write data to Activemq. + +## Key features + +- [ ] [exactly-once](../../concept/connector-v2-features.md) + +## Options + +| name | type | required | default value | +|-------------------------------------|---------|----------|---------------| +| host | string | no | - | +| port | int | no | - | +| virtual_host | string | no | - | +| username | string | no | - | +| password | string | no | - | +| queue_name | string | yes | - | +| uri | string | yes | - | +| check_for_duplicate | boolean | no | - | +| client_id | boolean | no | - | +| copy_message_on_send | boolean | no | - | +| disable_timeStamps_by_default | boolean | no | - | +| use_compression | boolean | no | - | +| always_session_async | boolean | no | - | +| dispatch_async | boolean | no | - | +| nested_map_and_list_enabled | boolean | no | - | +| warnAboutUnstartedConnectionTimeout | boolean | no | - | +| closeTimeout | int | no | - | + +### host [string] + +the default host to use for connections + +### port [int] + +the default port to use for connections + +### username [string] + +the AMQP user name to use when connecting to the broker + +### password [string] + +the password to use when connecting to the broker + +### uri [string] + +convenience method for setting the fields in an AMQP URI: host, port, username, password and virtual host + +### queue_name [string] + +the queue to write the message to + +### check_for_duplicate [boolean] + +will check for duplucate messages + +### client_id [string] + +client id + +### copy_message_on_send [boolean] + +if true, enables new JMS Message object as part of the send method + +### disable_timeStamps_by_default [boolean] + +disables timestamp for slight performance boost + +### use_compression [boolean] + +Enables the use of compression on the message’s body. + +### always_session_async [boolean] + +When true a separate thread is used for dispatching messages for each Session in the Connection. + +### always_sync_send [boolean] + +When true a MessageProducer will always use Sync sends when sending a Message + +### close_timeout [boolean] + +Sets the timeout, in milliseconds, before a close is considered complete. + +### dispatch_async [boolean] + +Should the broker dispatch messages asynchronously to the consumer + +### nested_map_and_list_enabled [boolean] + +Controls whether Structured Message Properties and MapMessages are supported + +### warn_about_unstarted_connection_timeout [int] + +The timeout, in milliseconds, from the time of connection creation to when a warning is generated + +## Example + +simple: + +```hocon +sink { + ActiveMQ { + uri="tcp://localhost:61616" + username = "admin" + password = "admin" + queue_name = "test1" + } +} +``` + +## Changelog + +### next version + +- Add Activemq Source Connector + diff --git a/plugin-mapping.properties b/plugin-mapping.properties index 9936afcbaaf..1942f875d7c 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -129,3 +129,4 @@ seatunnel.source.ObsFile = connector-file-obs seatunnel.sink.ObsFile = connector-file-obs seatunnel.source.Milvus = connector-milvus seatunnel.sink.Milvus = connector-milvus +seatunnel.sink.ActiveMQ = connector-activemq \ No newline at end of file diff --git a/seatunnel-connectors-v2/connector-activemq/pom.xml b/seatunnel-connectors-v2/connector-activemq/pom.xml new file mode 100644 index 00000000000..7a72a3b1c4c --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/pom.xml @@ -0,0 +1,65 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connectors-v2 + ${revision} + + + connector-activemq + SeaTunnel : Connectors V2 : Activemq + + + 5.14.5 + + + + org.apache.seatunnel + connector-common + ${project.version} + + + org.apache.activemq + activemq-client + ${activemq.version} + + + + org.apache.seatunnel + seatunnel-format-json + ${project.version} + + + + org.apache.seatunnel + seatunnel-format-json + ${project.version} + + + org.apache.seatunnel + seatunnel-format-text + ${project.version} + + + + diff --git a/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/client/ActivemqClient.java b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/client/ActivemqClient.java new file mode 100644 index 00000000000..f4983d35dbd --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/client/ActivemqClient.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.activemq.client; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.connectors.seatunnel.activemq.exception.ActivemqConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.activemq.exception.ActivemqConnectorException; + +import org.apache.activemq.ActiveMQConnectionFactory; + +import lombok.AllArgsConstructor; +import lombok.extern.slf4j.Slf4j; + +import javax.jms.Connection; +import javax.jms.Destination; +import javax.jms.JMSException; +import javax.jms.MessageProducer; +import javax.jms.Session; +import javax.jms.TextMessage; + +import java.nio.charset.StandardCharsets; + +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.ALWAYS_SESSION_ASYNC; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.ALWAYS_SYNC_SEND; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.CHECK_FOR_DUPLICATE; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.CLIENT_ID; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.CLOSE_TIMEOUT; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.CONSUMER_EXPIRY_CHECK_ENABLED; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.DISPATCH_ASYNC; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.NESTED_MAP_AND_LIST_ENABLED; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.PASSWORD; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.QUEUE_NAME; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.URI; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.USERNAME; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT; + +@Slf4j +@AllArgsConstructor +public class ActivemqClient { + private final ReadonlyConfig config; + private final ActiveMQConnectionFactory connectionFactory; + private final Connection connection; + + public ActivemqClient(ReadonlyConfig config) { + this.config = config; + try { + this.connectionFactory = getConnectionFactory(); + log.info("connection factory created"); + this.connection = createConnection(config); + log.info("connection created"); + + } catch (Exception e) { + e.printStackTrace(); + throw new ActivemqConnectorException( + ActivemqConnectorErrorCode.CREATE_ACTIVEMQ_CLIENT_FAILED, + "Error while create AMQ client "); + } + } + + public ActiveMQConnectionFactory getConnectionFactory() { + log.info("broker url : " + config.get(URI)); + ActiveMQConnectionFactory factory = new ActiveMQConnectionFactory(config.get(URI)); + + if (config.get(ALWAYS_SESSION_ASYNC) != null) { + factory.setAlwaysSessionAsync(config.get(ALWAYS_SESSION_ASYNC)); + } + + if (config.get(CLIENT_ID) != null) { + factory.setClientID(config.get(CLIENT_ID)); + } + + if (config.get(ALWAYS_SYNC_SEND) != null) { + factory.setAlwaysSyncSend(config.get(ALWAYS_SYNC_SEND)); + } + + if (config.get(CHECK_FOR_DUPLICATE) != null) { + factory.setCheckForDuplicates(config.get(CHECK_FOR_DUPLICATE)); + } + + if (config.get(CLOSE_TIMEOUT) != null) { + factory.setCloseTimeout(config.get(CLOSE_TIMEOUT)); + } + + if (config.get(CONSUMER_EXPIRY_CHECK_ENABLED) != null) { + factory.setConsumerExpiryCheckEnabled(config.get(CONSUMER_EXPIRY_CHECK_ENABLED)); + } + if (config.get(DISPATCH_ASYNC) != null) { + factory.setDispatchAsync(config.get(DISPATCH_ASYNC)); + } + + if (config.get(WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT) != null) { + factory.setWarnAboutUnstartedConnectionTimeout( + config.get(WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT)); + } + + if (config.get(NESTED_MAP_AND_LIST_ENABLED) != null) { + factory.setNestedMapAndListEnabled(config.get(NESTED_MAP_AND_LIST_ENABLED)); + } + return factory; + } + + public void write(byte[] msg) { + try { + this.connection.start(); + Session session = this.connection.createSession(false, Session.AUTO_ACKNOWLEDGE); + Destination destination = session.createQueue(config.get(QUEUE_NAME)); + MessageProducer producer = session.createProducer(destination); + String messageBody = new String(msg, StandardCharsets.UTF_8); + TextMessage objectMessage = session.createTextMessage(messageBody); + producer.send(objectMessage); + + } catch (JMSException e) { + throw new ActivemqConnectorException( + ActivemqConnectorErrorCode.SEND_MESSAGE_FAILED, + String.format( + "Cannot send AMQ message %s at %s", + config.get(QUEUE_NAME), config.get(CLIENT_ID)), + e); + } + } + + public void close() { + try { + if (connection != null) { + connection.close(); + } + } catch (JMSException e) { + throw new ActivemqConnectorException( + ActivemqConnectorErrorCode.CLOSE_CONNECTION_FAILED, + String.format( + "Error while closing AMQ connection with %s", config.get(QUEUE_NAME))); + } + } + + private Connection createConnection(ReadonlyConfig config) throws JMSException { + if (config.get(USERNAME) != null && config.get(PASSWORD) != null) { + return connectionFactory.createConnection(config.get(USERNAME), config.get(PASSWORD)); + } + return connectionFactory.createConnection(); + } +} diff --git a/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/config/ActivemqConfig.java b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/config/ActivemqConfig.java new file mode 100644 index 00000000000..868ac40a0c8 --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/config/ActivemqConfig.java @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.activemq.config; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; + +import com.google.common.annotations.VisibleForTesting; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.Setter; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +@Setter +@Getter +@AllArgsConstructor +public class ActivemqConfig implements Serializable { + private String host; + private Integer port; + private String username; + private String password; + private String uri; + private String queueName; + private Boolean checkForDuplicate; + private String clientID; + private Integer closeTimeout; + private Boolean consumerExpiryCheckEnabled; + private Boolean copyMessageOnSend; + private Boolean disableTimeStampsByDefault; + private Boolean dispatchAsync; + private Boolean nestedMapAndListEnabled; + private Boolean useCompression; + private Boolean alwaysSessionAsync; + private Boolean alwaysSyncSend; + private Integer warnAboutUnstartedConnectionTimeout; + + private final Map sinkOptionProps = new HashMap<>(); + + public static final Option HOST = + Options.key("host") + .stringType() + .noDefaultValue() + .withDescription("the default host to use for connections"); + + public static final Option PORT = + Options.key("port") + .intType() + .noDefaultValue() + .withDescription("the default port to use for connections"); + + public static final Option USERNAME = + Options.key("username") + .stringType() + .noDefaultValue() + .withDescription("the AMQP user name to use when connecting to the broker"); + + public static final Option PASSWORD = + Options.key("password") + .stringType() + .noDefaultValue() + .withDescription("the password to use when connecting to the broker"); + + public static final Option QUEUE_NAME = + Options.key("queue_name") + .stringType() + .noDefaultValue() + .withDescription("the queue to write the message to"); + + public static final Option URI = + Options.key("uri") + .stringType() + .noDefaultValue() + .withDescription( + "convenience method for setting the fields in an AMQP URI: host, port, username, password and virtual host"); + + public static final Option CHECK_FOR_DUPLICATE = + Options.key("check_for_duplicate") + .booleanType() + .noDefaultValue() + .withDescription( + "When true the consumer will check for duplicate messages and properly handle +" + + "the message to make sure that it is not processed twice inadvertently."); + public static final Option CLIENT_ID = + Options.key("client_id") + .stringType() + .noDefaultValue() + .withDescription("Sets the JMS clientID to use for the connection."); + + public static final Option COPY_MESSAGE_ON_SEND = + Options.key("copy_message_on_send") + .booleanType() + .noDefaultValue() + .withDescription( + "Should a JMS message be copied to a new JMS Message object as part of the send() method in JMS. " + + "This is enabled by default to be compliant with the JMS specification. " + + "For a performance boost set to false if you do not mutate JMS messages after they are sent."); + + public static final Option DISABLE_TIMESTAMP_BY_DEFAULT = + Options.key("disable_timeStamps_by_default") + .booleanType() + .noDefaultValue() + .withDescription( + "Sets whether or not timestamps on messages should be disabled or not. " + + "For a small performance boost set to false."); + + public static final Option USE_COMPRESSION = + Options.key("use_compression") + .booleanType() + .noDefaultValue() + .withDescription("Enables the use of compression on the message’s body."); + + public static final Option ALWAYS_SESSION_ASYNC = + Options.key("always_session_async") + .booleanType() + .noDefaultValue() + .withDescription( + "When true a separate thread is used for dispatching messages for each Session in the Connection. " + + "A separate thread is always used when there’s more than one session, " + + "or the session isn’t in Session.AUTO_ACKNOWLEDGE or Session.DUPS_OK_ACKNOWLEDGE mode."); + + public static final Option ALWAYS_SYNC_SEND = + Options.key("always_sync_send") + .booleanType() + .noDefaultValue() + .withDescription( + "When true a MessageProducer will always use Sync sends when sending a Message " + + "even if it is not required for the Delivery Mode."); + + public static final Option CLOSE_TIMEOUT = + Options.key("close_timeout") + .intType() + .noDefaultValue() + .withDescription( + "Sets the timeout, in milliseconds, before a close is considered complete. " + + "Normally a close() on a connection waits for confirmation from the broker. " + + "This allows the close operation to timeout preventing the client from hanging when no broker is available."); + + public static final Option DISPATCH_ASYNC = + Options.key("dispatch_async") + .booleanType() + .noDefaultValue() + .withDescription( + "Should the broker dispatch messages asynchronously to the consumer?"); + + public static final Option NESTED_MAP_AND_LIST_ENABLED = + Options.key("nested_map_and_list_enabled") + .booleanType() + .noDefaultValue() + .withDescription( + "Controls whether Structured Message Properties and MapMessages are supported " + + "so that Message properties and MapMessage entries can contain nested Map and List objects." + + " Available from version 4.1."); + + public static final Option WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT = + Options.key("warn_about_unstarted_connection_timeout") + .intType() + .noDefaultValue() + .withDescription( + "The timeout, in milliseconds, from the time of connection creation to when a warning is generated " + + "if the connection is not properly started via Connection.start() and a message is received by a consumer. " + + "It is a very common gotcha to forget to start the connection and then wonder why no messages are delivered " + + "so this option makes the default case to create a warning if the user forgets. " + + "To disable the warning just set the value to < 0."); + + public static final Option CONSUMER_EXPIRY_CHECK_ENABLED = + Options.key("consumer_expiry_check_enabled") + .booleanType() + .noDefaultValue() + .withDescription( + "Controls whether message expiration checking is done in each " + + "MessageConsumer prior to dispatching a message."); + + public ActivemqConfig(Config config) { + this.host = config.getString(HOST.key()); + this.port = config.getInt(PORT.key()); + this.queueName = config.getString(QUEUE_NAME.key()); + this.uri = config.getString(URI.key()); + if (config.hasPath(USERNAME.key())) { + this.username = config.getString(USERNAME.key()); + } + if (config.hasPath(PASSWORD.key())) { + this.password = config.getString(PASSWORD.key()); + } + if (config.hasPath(CHECK_FOR_DUPLICATE.key())) { + this.checkForDuplicate = config.getBoolean(CHECK_FOR_DUPLICATE.key()); + } + if (config.hasPath(CLIENT_ID.key())) { + this.clientID = config.getString(CLIENT_ID.key()); + } + if (config.hasPath(COPY_MESSAGE_ON_SEND.key())) { + this.copyMessageOnSend = config.getBoolean(COPY_MESSAGE_ON_SEND.key()); + } + if (config.hasPath(DISABLE_TIMESTAMP_BY_DEFAULT.key())) { + this.disableTimeStampsByDefault = config.getBoolean(DISABLE_TIMESTAMP_BY_DEFAULT.key()); + } + if (config.hasPath(USE_COMPRESSION.key())) { + this.useCompression = config.getBoolean(USE_COMPRESSION.key()); + } + if (config.hasPath(ALWAYS_SESSION_ASYNC.key())) { + this.alwaysSessionAsync = config.getBoolean(ALWAYS_SESSION_ASYNC.key()); + } + if (config.hasPath(ALWAYS_SYNC_SEND.key())) { + this.alwaysSyncSend = config.getBoolean(ALWAYS_SYNC_SEND.key()); + } + if (config.hasPath(CLOSE_TIMEOUT.key())) { + this.closeTimeout = config.getInt(CLOSE_TIMEOUT.key()); + } + if (config.hasPath(DISPATCH_ASYNC.key())) { + this.dispatchAsync = config.getBoolean(DISPATCH_ASYNC.key()); + } + if (config.hasPath(NESTED_MAP_AND_LIST_ENABLED.key())) { + this.nestedMapAndListEnabled = config.getBoolean(NESTED_MAP_AND_LIST_ENABLED.key()); + } + if (config.hasPath(WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT.key())) { + this.warnAboutUnstartedConnectionTimeout = + config.getInt(WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT.key()); + } + } + + @VisibleForTesting + public ActivemqConfig() {} +} diff --git a/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorErrorCode.java b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorErrorCode.java new file mode 100644 index 00000000000..138a49cdcb1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorErrorCode.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.activemq.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; + +public enum ActivemqConnectorErrorCode implements SeaTunnelErrorCode { + HANDLE_SHUTDOWN_SIGNAL_FAILED("ACTIVEMQ-01", "handle queue consumer shutdown signal failed"), + CREATE_ACTIVEMQ_CLIENT_FAILED("ACTIVEMQ-02", "create activemq client failed"), + CLOSE_CONNECTION_FAILED("ACTIVEMQ-03", "close connection failed"), + SEND_MESSAGE_FAILED("ACTIVEMQ-04", "send messages failed"), + MESSAGE_ACK_FAILED( + "ACTIVEMQ-05", "messages could not be acknowledged during checkpoint creation"), + MESSAGE_ACK_REJECTED("ACTIVEMQ-06", "messages could not be acknowledged with basicReject"), + PARSE_URI_FAILED("ACTIVEMQ-07", "parse uri failed"), + INIT_SSL_CONTEXT_FAILED("ACTIVEMQ-08", "initialize ssl context failed"), + SETUP_SSL_FACTORY_FAILED("ACTIVEMQ-09", "setup ssl factory failed"); + + private final String code; + private final String description; + + ActivemqConnectorErrorCode(String code, String description) { + this.code = code; + this.description = description; + } + + @Override + public String getCode() { + return code; + } + + @Override + public String getDescription() { + return description; + } +} diff --git a/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorException.java b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorException.java new file mode 100644 index 00000000000..7791c8e6184 --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/exception/ActivemqConnectorException.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.activemq.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; + +public class ActivemqConnectorException extends SeaTunnelRuntimeException { + public ActivemqConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage) { + super(seaTunnelErrorCode, errorMessage); + } + + public ActivemqConnectorException( + SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage, Throwable cause) { + super(seaTunnelErrorCode, errorMessage, cause); + } + + public ActivemqConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, Throwable cause) { + super(seaTunnelErrorCode, cause); + } +} diff --git a/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSink.java b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSink.java new file mode 100644 index 00000000000..d1d37017959 --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSink.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.activemq.sink; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; + +import java.io.IOException; + +public class ActivemqSink extends AbstractSimpleSink { + private final SeaTunnelRowType seaTunnelRowType; + private final ReadonlyConfig pluginConfig; + + @Override + public String getPluginName() { + return "ActiveMQ"; + } + + public ActivemqSink(ReadonlyConfig pluginConfig, SeaTunnelRowType rowType) { + this.pluginConfig = pluginConfig; + this.seaTunnelRowType = rowType; + } + + @Override + public AbstractSinkWriter createWriter(SinkWriter.Context context) + throws IOException { + return new ActivemqSinkWriter(pluginConfig, seaTunnelRowType); + } +} diff --git a/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkFactory.java b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkFactory.java new file mode 100644 index 00000000000..7f0dca38f6a --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkFactory.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.activemq.sink; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.connector.TableSink; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableSinkFactory; +import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; + +import com.google.auto.service.AutoService; + +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.ALWAYS_SESSION_ASYNC; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.ALWAYS_SYNC_SEND; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.CHECK_FOR_DUPLICATE; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.CLIENT_ID; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.CLOSE_TIMEOUT; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.COPY_MESSAGE_ON_SEND; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.DISABLE_TIMESTAMP_BY_DEFAULT; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.DISPATCH_ASYNC; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.HOST; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.NESTED_MAP_AND_LIST_ENABLED; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.PASSWORD; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.PORT; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.QUEUE_NAME; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.URI; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.USERNAME; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.USE_COMPRESSION; +import static org.apache.seatunnel.connectors.seatunnel.activemq.config.ActivemqConfig.WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT; + +@AutoService(Factory.class) +public class ActivemqSinkFactory implements TableSinkFactory { + + @Override + public String factoryIdentifier() { + return "ActiveMQ"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required(QUEUE_NAME, URI) + .bundled(USERNAME, PASSWORD) + .optional( + HOST, + PORT, + CLIENT_ID, + CHECK_FOR_DUPLICATE, + COPY_MESSAGE_ON_SEND, + DISABLE_TIMESTAMP_BY_DEFAULT, + USE_COMPRESSION, + ALWAYS_SESSION_ASYNC, + ALWAYS_SYNC_SEND, + CLOSE_TIMEOUT, + DISPATCH_ASYNC, + NESTED_MAP_AND_LIST_ENABLED, + WARN_ABOUT_UNSTARTED_CONNECTION_TIMEOUT) + .build(); + } + + @Override + public TableSink createSink(TableSinkFactoryContext context) { + return () -> + new ActivemqSink( + context.getOptions(), + context.getCatalogTable().getTableSchema().toPhysicalRowDataType()); + } +} diff --git a/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkWriter.java b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkWriter.java new file mode 100644 index 00000000000..f3395552c44 --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/main/java/org/apache/seatunnel/connectors/seatunnel/activemq/sink/ActivemqSinkWriter.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.activemq.sink; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.serialization.SerializationSchema; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.activemq.client.ActivemqClient; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; +import org.apache.seatunnel.format.json.JsonSerializationSchema; + +import java.util.Optional; + +public class ActivemqSinkWriter extends AbstractSinkWriter { + private ActivemqClient activeMQClient; + + private final SerializationSchema serializationSchema; + + public ActivemqSinkWriter(ReadonlyConfig config, SeaTunnelRowType seaTunnelRowType) { + this.activeMQClient = new ActivemqClient(config); + this.serializationSchema = new JsonSerializationSchema(seaTunnelRowType); + } + + @Override + public void write(SeaTunnelRow element) { + activeMQClient.write(serializationSchema.serialize(element)); + } + + @Override + public Optional prepareCommit() { + return Optional.empty(); + } + + @Override + public void close() { + if (activeMQClient != null) { + activeMQClient.close(); + } + } +} diff --git a/seatunnel-connectors-v2/connector-activemq/src/test/java/org/apache/seatunnel/connectors/seatunnel/activemq/ActivemqFactoryTest.java b/seatunnel-connectors-v2/connector-activemq/src/test/java/org/apache/seatunnel/connectors/seatunnel/activemq/ActivemqFactoryTest.java new file mode 100644 index 00000000000..90732d8a0ed --- /dev/null +++ b/seatunnel-connectors-v2/connector-activemq/src/test/java/org/apache/seatunnel/connectors/seatunnel/activemq/ActivemqFactoryTest.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.activemq; + +import org.apache.seatunnel.connectors.seatunnel.activemq.sink.ActivemqSinkFactory; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +class ActivemqFactoryTest { + + @Test + void optionRule() { + Assertions.assertNotNull((new ActivemqSinkFactory()).optionRule()); + } +} diff --git a/seatunnel-connectors-v2/pom.xml b/seatunnel-connectors-v2/pom.xml index 68274736f08..e0564a5572b 100644 --- a/seatunnel-connectors-v2/pom.xml +++ b/seatunnel-connectors-v2/pom.xml @@ -78,6 +78,7 @@ connector-easysearch connector-web3j connector-milvus + connector-activemq diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index c96bf0b612b..a16d86cad5a 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -583,6 +583,13 @@ provided + + org.apache.seatunnel + connector-activemq + ${project.version} + provided + + com.aliyun.phoenix diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/pom.xml new file mode 100644 index 00000000000..d94d24b29d4 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/pom.xml @@ -0,0 +1,60 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connector-v2-e2e + ${revision} + + connector-activemq-e2e + SeaTunnel : E2E : Connector V2 : ActiveMQ + + + + + org.testcontainers + activemq + 1.20.1 + test + + + + org.apache.seatunnel + connector-common + ${project.version} + + + + org.apache.seatunnel + connector-fake + ${project.version} + test + + + org.apache.seatunnel + connector-file-local + ${project.version} + + + + org.apache.seatunnel + connector-activemq + ${project.version} + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/activemq/ActivemqIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/activemq/ActivemqIT.java new file mode 100644 index 00000000000..56b1faeab86 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/activemq/ActivemqIT.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.e2e.connector.activemq; + +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.TestContainer; + +import org.apache.activemq.ActiveMQConnectionFactory; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.wait.strategy.HostPortWaitStrategy; +import org.testcontainers.utility.DockerImageName; + +import javax.jms.Connection; +import javax.jms.ConnectionFactory; +import javax.jms.JMSException; +import javax.jms.MessageConsumer; +import javax.jms.MessageProducer; +import javax.jms.Queue; +import javax.jms.Session; +import javax.jms.TextMessage; + +import java.io.IOException; +import java.time.Duration; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class ActivemqIT extends TestSuiteBase { + + private static final String ACTIVEMQ_CONTAINER_HOST = "activemq-host"; + public GenericContainer activeMQContainer = + new GenericContainer<>(DockerImageName.parse("rmohr/activemq")) + .withExposedPorts(61616) + .withNetworkAliases(ACTIVEMQ_CONTAINER_HOST) + .withNetwork(NETWORK); + + private Connection connection; + private Session session; + private MessageProducer producer; + private MessageConsumer consumer; + + @BeforeAll + public void setup() throws JMSException, InterruptedException { + activeMQContainer + .withNetwork(NETWORK) + .waitingFor(new HostPortWaitStrategy().withStartupTimeout(Duration.ofMinutes(2))); + activeMQContainer.start(); + String brokerUrl = "tcp://127.0.0.1:" + activeMQContainer.getMappedPort(61616); + ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(brokerUrl); + connection = connectionFactory.createConnection(); + connection.start(); + + // Creating session for sending messages + session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE); + + // Getting the queue + Queue queue = session.createQueue("testQueue"); + + // Creating the producer & consumer + producer = session.createProducer(queue); + consumer = session.createConsumer(queue); + } + + @AfterAll + public void tearDown() throws JMSException { + // Cleaning up resources + if (producer != null) producer.close(); + if (session != null) session.close(); + if (connection != null) connection.close(); + } + + @Test + public void testSendMessage() throws JMSException { + String dummyPayload = "Dummy payload"; + + // Sending a text message to the queue + TextMessage message = session.createTextMessage(dummyPayload); + producer.send(message); + + // Receiving the message from the queue + TextMessage receivedMessage = (TextMessage) consumer.receive(5000); + + assertEquals(dummyPayload, receivedMessage.getText()); + } + + @TestTemplate + public void testSinkApacheActivemq(TestContainer container) + throws IOException, InterruptedException, JMSException { + Container.ExecResult execResult = container.executeJob("/fake_source_to_sink.conf"); + TextMessage textMessage = (TextMessage) consumer.receive(); + Assertions.assertTrue(textMessage.getText().contains("map")); + Assertions.assertTrue(textMessage.getText().contains("c_boolean")); + Assertions.assertTrue(textMessage.getText().contains("c_tinyint")); + Assertions.assertTrue(textMessage.getText().contains("c_timestamp")); + Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/e2e.json b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/e2e.json new file mode 100644 index 00000000000..040ee633bd8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/e2e.json @@ -0,0 +1,100 @@ +{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}} +{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}}{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/fake_source_to_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/fake_source_to_sink.conf new file mode 100644 index 00000000000..a3c0859ed14 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/fake_source_to_sink.conf @@ -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. +# + +###### +###### This config file is a demonstration of batch processing in SeaTunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" + # You can set spark configuration here + # see available properties defined by spark: https://spark.apache.org/docs/latest/configuration.html#available-properties + #job.mode = BATCH + job.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + FakeSource { + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + + + + # You can also use other input plugins, such as hdfs + # hdfs { + # result_table_name = "accesslog" + # path = "hdfs://hadoop-cluster-01/nginx/accesslog" + # format = "json" + # } + + # If you would like to get more information about how to configure seatunnel and see full list of input plugins, + # please go to https://seatunnel.apache.org/docs/category/source-v2 + + +transform { + # split data by specific delimiter + + # you can also use other transform plugins, such as sql + + + # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, + # please go to https://seatunnel.apache.org/docs/category/transform-v2 +} + + + +sink { + ActiveMQ { + host = "activemq-e2e" + port = "5672" + queue_name = "testQueue" + uri="tcp://activemq-host:61616" + } +} + + # you can also you other output plugins, such as sql + # hdfs { + # path = "hdfs://hadoop-cluster-01/nginx/accesslog_processed" + # save_mode = "append" + # } + + # If you would like to get more information about how to configure seatunnel and see full list of output plugins, + # please go to https://seatunnel.apache.org/docs/category/sink-v2 + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/localfile_source_to_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/localfile_source_to_sink.conf new file mode 100644 index 00000000000..7c5757c51f7 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-activemq-e2e/src/test/resources/localfile_source_to_sink.conf @@ -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. +# + +###### +###### This config file is a demonstration of batch processing in SeaTunnel config +###### + +env { + # You can set spark configuration here + # see available properties defined by spark: https://spark.apache.org/docs/latest/configuration.html#available-properties + #job.mode = BATCH + job.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + LocalFile { + path = "/e2e.json" + file_format_type = "json" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + C_MAP = "map" + C_ARRAY = "array" + C_STRING = string + C_BOOLEAN = boolean + C_TINYINT = tinyint + C_SMALLINT = smallint + C_INT = int + C_BIGINT = bigint + C_FLOAT = float + C_DOUBLE = double + C_BYTES = bytes + C_DATE = date + C_DECIMAL = "decimal(38, 18)" + C_TIMESTAMP = timestamp + } + } + } + result_table_name = "fake" + } +} + + # You can also use other input plugins, such as hdfs + # hdfs { + # result_table_name = "accesslog" + # path = "hdfs://hadoop-cluster-01/nginx/accesslog" + # format = "json" + # } + + # If you would like to get more information about how to configure seatunnel and see full list of input plugins, + # please go to https://seatunnel.apache.org/docs/category/source-v2 + + +transform { + # split data by specific delimiter + + # you can also use other transform plugins, such as sql + + + # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, + # please go to https://seatunnel.apache.org/docs/category/transform-v2 +} + + + +sink { + ActiveMQ { + host = "active-e2e" + port = "5672" + username = "guest" + password = "guest" + queue_name = "test1" + uri="tcp://localhost:61616" + } +} + + # you can also you other output plugins, such as sql + # hdfs { + # path = "hdfs://hadoop-cluster-01/nginx/accesslog_processed" + # save_mode = "append" + # } + + # If you would like to get more information about how to configure seatunnel and see full list of output plugins, + # please go to https://seatunnel.apache.org/docs/category/sink-v2 + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index 0a0f909e199..2db67f88147 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -75,6 +75,7 @@ connector-hive-e2e connector-hudi-e2e connector-milvus-e2e + connector-activemq-e2e From 855254e737051edbaf4ca08b95ca010fe18fd214 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Wed, 7 Aug 2024 11:03:06 +0800 Subject: [PATCH 025/361] [Feature][Transform] Add LLM transform (#7303) * [Feature][Transform] Add LLM transform * update * update * retrigger --- docs/en/transform-v2/llm.md | 122 ++++++++++++++++++ docs/zh/transform-v2/llm.md | 120 +++++++++++++++++ .../seatunnel/e2e/transform/TestLLMIT.java | 90 +++++++++++++ .../test/resources/llm_openai_transform.conf | 75 +++++++++++ .../src/test/resources/mockserver-config.json | 40 ++++++ seatunnel-transforms-v2/pom.xml | 15 +++ .../common/SeaTunnelRowAccessor.java | 4 + .../seatunnel/transform/llm/LLMTransform.java | 119 +++++++++++++++++ .../transform/llm/LLMTransformConfig.java | 71 ++++++++++ .../transform/llm/LLMTransformFactory.java | 59 +++++++++ .../transform/llm/ModelProvider.java | 22 ++++ .../transform/llm/model/AbstractModel.java | 69 ++++++++++ .../seatunnel/transform/llm/model/Model.java | 29 +++++ .../llm/model/openai/OpenAIModel.java | 104 +++++++++++++++ .../transform/LLMTransformFactoryTest.java | 32 +++++ .../transform/llm/LLMRequestJsonTest.java | 61 +++++++++ tools/dependencies/known-dependencies.txt | 2 + 17 files changed, 1034 insertions(+) create mode 100644 docs/en/transform-v2/llm.md create mode 100644 docs/zh/transform-v2/llm.md create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_openai_transform.conf create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/mockserver-config.json create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransform.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformConfig.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformFactory.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/AbstractModel.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/Model.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/openai/OpenAIModel.java create mode 100644 seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/LLMTransformFactoryTest.java create mode 100644 seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/llm/LLMRequestJsonTest.java diff --git a/docs/en/transform-v2/llm.md b/docs/en/transform-v2/llm.md new file mode 100644 index 00000000000..d03b8226f06 --- /dev/null +++ b/docs/en/transform-v2/llm.md @@ -0,0 +1,122 @@ +# LLM + +> LLM transform plugin + +## Description + +Leverage the power of a large language model (LLM) to process data by sending it to the LLM and receiving the +generated results. Utilize the LLM's capabilities to label, clean, enrich data, perform data inference, and +more. + +## Options + +| name | type | required | default value | +|------------------|--------|----------|--------------------------------------------| +| model_provider | enum | yes | | +| output_data_type | enum | no | String | +| prompt | string | yes | | +| model | string | yes | | +| api_key | string | yes | | +| openai.api_path | string | no | https://api.openai.com/v1/chat/completions | + +### model_provider + +The model provider to use. The available options are: +OPENAI + +### output_data_type + +The data type of the output data. The available options are: +STRING,INT,BIGINT,DOUBLE,BOOLEAN. +Default value is STRING. + +### prompt + +The prompt to send to the LLM. This parameter defines how LLM will process and return data, eg: + +The data read from source is a table like this: + +| name | age | +|---------------|-----| +| Jia Fan | 20 | +| Hailin Wang | 20 | +| Eric | 20 | +| Guangdong Liu | 20 | + +The prompt can be: + +``` +Determine whether someone is Chinese or American by their name +``` + +The result will be: + +| name | age | llm_output | +|---------------|-----|------------| +| Jia Fan | 20 | Chinese | +| Hailin Wang | 20 | Chinese | +| Eric | 20 | American | +| Guangdong Liu | 20 | Chinese | + +### model + +The model to use. Different model providers have different models. For example, the OpenAI model can be `gpt-4o-mini`. +If you use OpenAI model, please refer https://platform.openai.com/docs/models/model-endpoint-compatibility of `/v1/chat/completions` endpoint. + +### api_key + +The API key to use for the model provider. +If you use OpenAI model, please refer https://platform.openai.com/docs/api-reference/api-keys of how to get the API key. + +### openai.api_path + +The API path to use for the OpenAI model provider. In most cases, you do not need to change this configuration. If you are using an API agent's service, you may need to configure it to the agent's API address. + +### common options [string] + +Transform plugin common parameters, please refer to [Transform Plugin](common-options.md) for details + +## Example + +Determine the user's country through a LLM. + +```hocon +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema = { + fields { + id = "int" + name = "string" + } + } + rows = [ + {fields = [1, "Jia Fan"], kind = INSERT} + {fields = [2, "Hailin Wang"], kind = INSERT} + {fields = [3, "Tomas"], kind = INSERT} + {fields = [4, "Eric"], kind = INSERT} + {fields = [5, "Guangdong Liu"], kind = INSERT} + ] + } +} + +transform { + LLM { + model_provider = OPENAI + model = gpt-4o-mini + api_key = sk-xxx + prompt = "Determine whether someone is Chinese or American by their name" + } +} + +sink { + console { + } +} +``` + diff --git a/docs/zh/transform-v2/llm.md b/docs/zh/transform-v2/llm.md new file mode 100644 index 00000000000..acd3245b8eb --- /dev/null +++ b/docs/zh/transform-v2/llm.md @@ -0,0 +1,120 @@ +# LLM + +> LLM 转换插件 + +## 描述 + +利用大型语言模型 (LLM) 的强大功能来处理数据,方法是将数据发送到 LLM 并接收生成的结果。利用 LLM 的功能来标记、清理、丰富数据、执行数据推理等。 + +## 属性 + +| 名称 | 类型 | 是否必须 | 默认值 | +|------------------|--------|------|--------------------------------------------| +| model_provider | enum | yes | | +| output_data_type | enum | no | String | +| prompt | string | yes | | +| model | string | yes | | +| api_key | string | yes | | +| openai.api_path | string | no | https://api.openai.com/v1/chat/completions | + +### model_provider + +要使用的模型提供者。可用选项为: +OPENAI + +### output_data_type + +输出数据的数据类型。可用选项为: +STRING,INT,BIGINT,DOUBLE,BOOLEAN. +默认值为 STRING。 + +### prompt + +发送到 LLM 的提示。此参数定义 LLM 将如何处理和返回数据,例如: + +从源读取的数据是这样的表格: + +| name | age | +|---------------|-----| +| Jia Fan | 20 | +| Hailin Wang | 20 | +| Eric | 20 | +| Guangdong Liu | 20 | + +我们可以使用以下提示: + +``` +Determine whether someone is Chinese or American by their name +``` + +这将返回: + +| name | age | llm_output | +|---------------|-----|------------| +| Jia Fan | 20 | Chinese | +| Hailin Wang | 20 | Chinese | +| Eric | 20 | American | +| Guangdong Liu | 20 | Chinese | + +### model + +要使用的模型。不同的模型提供者有不同的模型。例如,OpenAI 模型可以是 `gpt-4o-mini`。 +如果使用 OpenAI 模型,请参考 https://platform.openai.com/docs/models/model-endpoint-compatibility 文档的`/v1/chat/completions` 端点。 + +### api_key + +用于模型提供者的 API 密钥。 +如果使用 OpenAI 模型,请参考 https://platform.openai.com/docs/api-reference/api-keys 文档的如何获取 API 密钥。 + +### openai.api_path + +用于 OpenAI 模型提供者的 API 路径。在大多数情况下,您不需要更改此配置。如果使用 API 代理的服务,您可能需要将其配置为代理的 API 地址。 + +### common options [string] + +转换插件的常见参数, 请参考 [Transform Plugin](common-options.md) 了解详情 + +## 示例 + +通过 LLM 确定用户所在的国家。 + +```hocon +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema = { + fields { + id = "int" + name = "string" + } + } + rows = [ + {fields = [1, "Jia Fan"], kind = INSERT} + {fields = [2, "Hailin Wang"], kind = INSERT} + {fields = [3, "Tomas"], kind = INSERT} + {fields = [4, "Eric"], kind = INSERT} + {fields = [5, "Guangdong Liu"], kind = INSERT} + ] + } +} + +transform { + LLM { + model_provider = OPENAI + model = gpt-4o-mini + api_key = sk-xxx + prompt = "Determine whether someone is Chinese or American by their name" + } +} + +sink { + console { + } +} +``` + diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java new file mode 100644 index 00000000000..6f17c5a94f7 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.e2e.transform; + +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.container.TestContainer; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerImageName; +import org.testcontainers.utility.DockerLoggerFactory; +import org.testcontainers.utility.MountableFile; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.util.Optional; +import java.util.stream.Stream; + +public class TestLLMIT extends TestSuiteBase implements TestResource { + private static final String TMP_DIR = "/tmp"; + private GenericContainer mockserverContainer; + private static final String IMAGE = "mockserver/mockserver:5.14.0"; + + @BeforeAll + @Override + public void startUp() { + Optional resource = + Optional.ofNullable(TestLLMIT.class.getResource("/mockserver-config.json")); + this.mockserverContainer = + new GenericContainer<>(DockerImageName.parse(IMAGE)) + .withNetwork(NETWORK) + .withNetworkAliases("mockserver") + .withExposedPorts(1080) + .withCopyFileToContainer( + MountableFile.forHostPath( + new File( + resource.orElseThrow( + () -> + new IllegalArgumentException( + "Can not get config file of mockServer")) + .getPath()) + .getAbsolutePath()), + TMP_DIR + "/mockserver-config.json") + .withEnv( + "MOCKSERVER_INITIALIZATION_JSON_PATH", + TMP_DIR + "/mockserver-config.json") + .withEnv("MOCKSERVER_LOG_LEVEL", "WARN") + .withLogConsumer(new Slf4jLogConsumer(DockerLoggerFactory.getLogger(IMAGE))) + .waitingFor(new HttpWaitStrategy().forPath("/").forStatusCode(404)); + Startables.deepStart(Stream.of(mockserverContainer)).join(); + } + + @AfterAll + @Override + public void tearDown() throws Exception { + if (mockserverContainer != null) { + mockserverContainer.stop(); + } + } + + @TestTemplate + public void testLLMWithOpenAI(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult execResult = container.executeJob("/llm_openai_transform.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } +} diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_openai_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_openai_transform.conf new file mode 100644 index 00000000000..54495935893 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_openai_transform.conf @@ -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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema = { + fields { + id = "int" + name = "string" + } + } + rows = [ + {fields = [1, "Jia Fan"], kind = INSERT} + {fields = [2, "Hailin Wang"], kind = INSERT} + {fields = [3, "Tomas"], kind = INSERT} + {fields = [4, "Eric"], kind = INSERT} + {fields = [5, "Guangdong Liu"], kind = INSERT} + ] + result_table_name = "fake" + } +} + +transform { + LLM { + source_table_name = "fake" + model_provider = OPENAI + model = gpt-4o-mini + api_key = sk-xxx + prompt = "Determine whether someone is Chinese or American by their name" + openai.api_path = "http://mockserver:1080/v1/chat/completions" + result_table_name = "llm_output" + } +} + +sink { + Assert { + source_table_name = "llm_output" + rules = + { + field_rules = [ + { + field_name = llm_output + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/mockserver-config.json b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/mockserver-config.json new file mode 100644 index 00000000000..b4a2e53bea8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/mockserver-config.json @@ -0,0 +1,40 @@ +// https://www.mock-server.com/mock_server/getting_started.html#request_matchers + +[ + { + "httpRequest": { + "method": "POST", + "path": "/v1/chat/completions" + }, + "httpResponse": { + "body": { + "id": "chatcmpl-9s4hoBNGV0d9Mudkhvgzg64DAWPnx", + "object": "chat.completion", + "created": 1722674828, + "model": "gpt-4o-mini", + "choices": [ + { + "index": 0, + "message": { + "role": "assistant", + "content": "[\"Chinese\"]" + }, + "logprobs": null, + "finish_reason": "stop" + } + ], + "usage": { + "prompt_tokens": 107, + "completion_tokens": 3, + "total_tokens": 110 + }, + "system_fingerprint": "fp_0f03d4f0ee", + "code": 0, + "msg": "ok" + }, + "headers": { + "Content-Type": "application/json" + } + } + } +] diff --git a/seatunnel-transforms-v2/pom.xml b/seatunnel-transforms-v2/pom.xml index ae8909f463d..4cbef9a4b83 100644 --- a/seatunnel-transforms-v2/pom.xml +++ b/seatunnel-transforms-v2/pom.xml @@ -29,6 +29,11 @@ seatunnel-transforms-v2 SeaTunnel : Transforms : V2 + + 4.5.13 + 4.4.4 + + @@ -77,6 +82,16 @@ ${project.version} optional + + org.apache.httpcomponents + httpclient + ${httpclient.version} + + + org.apache.httpcomponents + httpcore + ${httpcore.version} + diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SeaTunnelRowAccessor.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SeaTunnelRowAccessor.java index 0224ef4b8f7..5b97f341686 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SeaTunnelRowAccessor.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SeaTunnelRowAccessor.java @@ -41,4 +41,8 @@ public RowKind getRowKind() { public Object getField(int pos) { return row.getField(pos); } + + public Object[] getFields() { + return row.getFields(); + } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransform.java new file mode 100644 index 00000000000..d19960044f1 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransform.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.llm; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.SeaTunnelDataTypeConvertorUtil; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; +import org.apache.seatunnel.transform.common.SingleFieldOutputTransform; +import org.apache.seatunnel.transform.llm.model.Model; +import org.apache.seatunnel.transform.llm.model.openai.OpenAIModel; + +import lombok.NonNull; +import lombok.SneakyThrows; + +import java.util.Collections; +import java.util.List; + +public class LLMTransform extends SingleFieldOutputTransform { + private final ReadonlyConfig config; + private final SeaTunnelDataType outputDataType; + private Model model; + + public LLMTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { + super(inputCatalogTable); + this.config = config; + this.outputDataType = + SeaTunnelDataTypeConvertorUtil.deserializeSeaTunnelDataType( + "output", config.get(LLMTransformConfig.OUTPUT_DATA_TYPE).toString()); + } + + private void tryOpen() { + if (model == null) { + open(); + } + } + + @Override + public String getPluginName() { + return "LLM"; + } + + @Override + public void open() { + ModelProvider provider = config.get(LLMTransformConfig.MODEL_PROVIDER); + if (provider.equals(ModelProvider.OPENAI)) { + model = + new OpenAIModel( + inputCatalogTable.getSeaTunnelRowType(), + outputDataType.getSqlType(), + config.get(LLMTransformConfig.PROMPT), + config.get(LLMTransformConfig.MODEL), + config.get(LLMTransformConfig.API_KEY), + config.get(LLMTransformConfig.OPENAI_API_PATH)); + } else { + throw new IllegalArgumentException("Unsupported model provider: " + provider); + } + } + + @Override + protected Object getOutputFieldValue(SeaTunnelRowAccessor inputRow) { + tryOpen(); + SeaTunnelRow seaTunnelRow = new SeaTunnelRow(inputRow.getFields()); + try { + List values = model.inference(Collections.singletonList(seaTunnelRow)); + switch (outputDataType.getSqlType()) { + case STRING: + return String.valueOf(values.get(0)); + case INT: + return Integer.parseInt(values.get(0)); + case BIGINT: + return Long.parseLong(values.get(0)); + case DOUBLE: + return Double.parseDouble(values.get(0)); + case BOOLEAN: + return Boolean.parseBoolean(values.get(0)); + default: + throw new IllegalArgumentException( + "Unsupported output data type: " + outputDataType); + } + } catch (Exception e) { + throw new RuntimeException( + String.format("Failed to inference model with row %s", seaTunnelRow), e); + } + } + + @Override + protected Column getOutputColumn() { + return PhysicalColumn.of( + "llm_output", outputDataType, (Long) null, true, null, "Output column of LLM"); + } + + @SneakyThrows + @Override + public void close() { + if (model != null) { + model.close(); + } + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformConfig.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformConfig.java new file mode 100644 index 00000000000..ca3da7e6706 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformConfig.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.llm; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.api.table.type.SqlType; + +import java.io.Serializable; + +public class LLMTransformConfig implements Serializable { + + public static final Option MODEL_PROVIDER = + Options.key("model_provider") + .enumType(ModelProvider.class) + .noDefaultValue() + .withDescription("The model provider of LLM"); + + public static final Option OUTPUT_DATA_TYPE = + Options.key("output_data_type") + .enumType(SqlType.class) + .defaultValue(SqlType.STRING) + .withDescription("The output data type of LLM"); + + public static final Option PROMPT = + Options.key("prompt") + .stringType() + .noDefaultValue() + .withDescription("The prompt of LLM"); + + public static final Option MODEL = + Options.key("model") + .stringType() + .noDefaultValue() + .withDescription( + "The model of LLM, eg: if the model provider is OpenAI, the model should be gpt-3.5-turbo/gpt-4o-mini, etc."); + + public static final Option API_KEY = + Options.key("api_key") + .stringType() + .noDefaultValue() + .withDescription("The API key of LLM"); + + public static final Option INFERENCE_BATCH_SIZE = + Options.key("inference_batch_size") + .intType() + .defaultValue(100) + .withDescription("The row batch size of each inference"); + + // OPENAI specific options + public static final Option OPENAI_API_PATH = + Options.key("openai.api_path") + .stringType() + .defaultValue("https://api.openai.com/v1/chat/completions") + .withDescription("The API path of OpenAI"); +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformFactory.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformFactory.java new file mode 100644 index 00000000000..6fe5d53fe5b --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformFactory.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.llm; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.connector.TableTransform; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableTransformFactory; +import org.apache.seatunnel.api.table.factory.TableTransformFactoryContext; + +import com.google.auto.service.AutoService; + +@AutoService(Factory.class) +public class LLMTransformFactory implements TableTransformFactory { + @Override + public String factoryIdentifier() { + return "LLM"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required( + LLMTransformConfig.MODEL_PROVIDER, + LLMTransformConfig.MODEL, + LLMTransformConfig.PROMPT, + LLMTransformConfig.API_KEY) + .optional( + LLMTransformConfig.OUTPUT_DATA_TYPE, + LLMTransformConfig.INFERENCE_BATCH_SIZE) + .conditional( + LLMTransformConfig.MODEL_PROVIDER, + ModelProvider.OPENAI, + LLMTransformConfig.OPENAI_API_PATH) + .build(); + } + + @Override + public TableTransform createTransform(TableTransformFactoryContext context) { + CatalogTable catalogTable = context.getCatalogTables().get(0); + return () -> new LLMTransform(context.getOptions(), catalogTable); + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.java new file mode 100644 index 00000000000..a55d706c099 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.llm; + +public enum ModelProvider { + OPENAI +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/AbstractModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/AbstractModel.java new file mode 100644 index 00000000000..51d674c0ad8 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/AbstractModel.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.llm.model; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.format.json.RowToJsonConverters; + +import java.io.IOException; +import java.util.List; + +public abstract class AbstractModel implements Model { + + protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private final RowToJsonConverters.RowToJsonConverter rowToJsonConverters; + private final String prompt; + private final SqlType outputType; + + public AbstractModel(SeaTunnelRowType rowType, SqlType outputType, String prompt) { + this.prompt = prompt; + this.outputType = outputType; + this.rowToJsonConverters = new RowToJsonConverters().createConverter(rowType, null); + } + + private String getPromptWithLimit() { + return prompt + + "\n The following rules need to be followed: " + + "\n 1. The received data is an array, and the result is returned in the form of an array." + + "\n 2. Only the result needs to be returned, and no other information can be returned." + + "\n 3. The element type of the array is " + + outputType.toString() + + "." + + "\n Eg: [\"value1\", \"value2\"]"; + } + + @Override + public List inference(List rows) throws IOException { + ArrayNode rowsNode = OBJECT_MAPPER.createArrayNode(); + for (SeaTunnelRow row : rows) { + ObjectNode rowNode = OBJECT_MAPPER.createObjectNode(); + rowToJsonConverters.convert(OBJECT_MAPPER, rowNode, row); + rowsNode.add(rowNode); + } + return chatWithModel(getPromptWithLimit(), OBJECT_MAPPER.writeValueAsString(rowsNode)); + } + + protected abstract List chatWithModel(String promptWithLimit, String rowsJson) + throws IOException; +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/Model.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/Model.java new file mode 100644 index 00000000000..77a8da63281 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/Model.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.llm.model; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; + +public interface Model extends Closeable { + + List inference(List rows) throws IOException; +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/openai/OpenAIModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/openai/OpenAIModel.java new file mode 100644 index 00000000000..9477b873202 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/openai/OpenAIModel.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.llm.model.openai; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.transform.llm.model.AbstractModel; + +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; + +import com.google.common.annotations.VisibleForTesting; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.List; + +/** + * OpenAI model. Refer chat api + */ +@Slf4j +public class OpenAIModel extends AbstractModel { + + private final CloseableHttpClient client; + private final String apiKey; + private final String model; + private final String apiPath; + + public OpenAIModel( + SeaTunnelRowType rowType, + SqlType outputType, + String prompt, + String model, + String apiKey, + String apiPath) { + super(rowType, outputType, prompt); + this.apiKey = apiKey; + this.apiPath = apiPath; + this.model = model; + this.client = HttpClients.createDefault(); + } + + @Override + protected List chatWithModel(String prompt, String data) throws IOException { + HttpPost post = new HttpPost(apiPath); + post.setHeader("Authorization", "Bearer " + apiKey); + post.setHeader("Content-Type", "application/json"); + ObjectNode objectNode = createJsonNodeFromData(prompt, data); + post.setEntity(new StringEntity(OBJECT_MAPPER.writeValueAsString(objectNode), "UTF-8")); + post.setConfig( + RequestConfig.custom().setConnectTimeout(20000).setSocketTimeout(20000).build()); + CloseableHttpResponse response = client.execute(post); + String responseStr = EntityUtils.toString(response.getEntity()); + if (response.getStatusLine().getStatusCode() != 200) { + throw new IOException("Failed to chat with model, response: " + responseStr); + } + + JsonNode result = OBJECT_MAPPER.readTree(responseStr); + String resultData = result.get("choices").get(0).get("message").get("content").asText(); + return OBJECT_MAPPER.readValue(resultData, new TypeReference>() {}); + } + + @VisibleForTesting + public ObjectNode createJsonNodeFromData(String prompt, String data) { + ObjectNode objectNode = OBJECT_MAPPER.createObjectNode(); + objectNode.put("model", model); + ArrayNode messages = objectNode.putArray("messages"); + messages.addObject().put("role", "system").put("content", prompt); + messages.addObject().put("role", "user").put("content", data); + return objectNode; + } + + @Override + public void close() throws IOException { + if (client != null) { + client.close(); + } + } +} diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/LLMTransformFactoryTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/LLMTransformFactoryTest.java new file mode 100644 index 00000000000..39b27694805 --- /dev/null +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/LLMTransformFactoryTest.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform; + +import org.apache.seatunnel.transform.llm.LLMTransformFactory; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class LLMTransformFactoryTest { + + @Test + public void testOptionRule() throws Exception { + LLMTransformFactory replaceTransformFactory = new LLMTransformFactory(); + Assertions.assertNotNull(replaceTransformFactory.optionRule()); + } +} diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/llm/LLMRequestJsonTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/llm/LLMRequestJsonTest.java new file mode 100644 index 00000000000..f32cc870559 --- /dev/null +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/llm/LLMRequestJsonTest.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.llm; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; + +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.transform.llm.model.openai.OpenAIModel; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +public class LLMRequestJsonTest { + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + @Test + void testOpenAIRequestJson() throws IOException { + SeaTunnelRowType rowType = + new SeaTunnelRowType( + new String[] {"id", "name"}, + new SeaTunnelDataType[] {BasicType.INT_TYPE, BasicType.STRING_TYPE}); + OpenAIModel model = + new OpenAIModel( + rowType, + SqlType.STRING, + "Determine whether someone is Chinese or American by their name", + "gpt-3.5-turbo", + "sk-xxx", + "https://api.openai.com/v1/chat/completions"); + ObjectNode node = + model.createJsonNodeFromData( + "Determine whether someone is Chinese or American by their name", + "{\"id\":1, \"name\":\"John\"}"); + Assertions.assertEquals( + "{\"model\":\"gpt-3.5-turbo\",\"messages\":[{\"role\":\"system\",\"content\":\"Determine whether someone is Chinese or American by their name\"},{\"role\":\"user\",\"content\":\"{\\\"id\\\":1, \\\"name\\\":\\\"John\\\"}\"}]}", + OBJECT_MAPPER.writeValueAsString(node)); + model.close(); + } +} diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt index 161134511c8..eda697369ed 100755 --- a/tools/dependencies/known-dependencies.txt +++ b/tools/dependencies/known-dependencies.txt @@ -8,6 +8,8 @@ config-1.3.3.jar disruptor-3.4.4.jar guava-27.0-jre.jar hazelcast-5.1.jar +httpclient-4.5.13.jar +httpcore-4.4.4.jar jackson-annotations-2.13.3.jar jackson-core-2.13.3.jar jackson-databind-2.13.3.jar From a4db64d7c76f07daaf8030011ee5e8552396acd2 Mon Sep 17 00:00:00 2001 From: zhangdonghao <39961809+hawk9821@users.noreply.github.com> Date: Wed, 7 Aug 2024 11:52:19 +0800 Subject: [PATCH 026/361] [Improve][E2E] Support windows for the e2e of paimon (#7329) --- .../e2e/connector/paimon/PaimonSinkCDCIT.java | 53 ++++++++++++++++--- 1 file changed, 46 insertions(+), 7 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkCDCIT.java index c899dd0e8bf..4b1d7dd86ce 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkCDCIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkCDCIT.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.e2e.connector.paimon; import org.apache.seatunnel.common.utils.FileUtils; +import org.apache.seatunnel.core.starter.utils.CompressionUtils; import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; @@ -25,6 +26,7 @@ import org.apache.seatunnel.e2e.common.container.TestContainer; import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.commons.compress.archivers.ArchiveException; import org.apache.commons.lang3.StringUtils; import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Catalog; @@ -52,6 +54,7 @@ import lombok.extern.slf4j.Slf4j; +import java.io.File; import java.io.IOException; import java.time.LocalDate; import java.util.ArrayList; @@ -68,7 +71,8 @@ "Spark and Flink engine can not auto create paimon table on worker node in local file(e.g flink tm) by savemode feature which can lead error") @Slf4j public class PaimonSinkCDCIT extends TestSuiteBase implements TestResource { - private static final String CATALOG_ROOT_DIR = "/tmp/"; + + private static String CATALOG_ROOT_DIR = "/tmp/"; private static final String NAMESPACE = "paimon"; private static final String NAMESPACE_TAR = "paimon.tar.gz"; private static final String CATALOG_DIR = CATALOG_ROOT_DIR + NAMESPACE + "/"; @@ -77,10 +81,18 @@ public class PaimonSinkCDCIT extends TestSuiteBase implements TestResource { private static final String FAKE_DATABASE1 = "FakeDatabase1"; private static final String FAKE_TABLE2 = "FakeTable1"; private static final String FAKE_DATABASE2 = "FakeDatabase2"; + private String CATALOG_ROOT_DIR_WIN = "C:/Users/"; + private String CATALOG_DIR_WIN = CATALOG_ROOT_DIR_WIN + NAMESPACE + "/"; + private boolean isWindows; @BeforeAll @Override - public void startUp() throws Exception {} + public void startUp() throws Exception { + this.isWindows = + System.getProperties().getProperty("os.name").toUpperCase().contains("WINDOWS"); + CATALOG_ROOT_DIR_WIN = CATALOG_ROOT_DIR_WIN + System.getProperty("user.name") + "/tmp/"; + CATALOG_DIR_WIN = CATALOG_ROOT_DIR_WIN + NAMESPACE + "/"; + } @AfterAll @Override @@ -498,8 +510,15 @@ public void testFakeSinkPaimonWithFullTypeAndReadWithFilter(TestContainer contai protected final ContainerExtendedFactory containerExtendedFactory = container -> { - FileUtils.deleteFile(CATALOG_ROOT_DIR + NAMESPACE_TAR); - FileUtils.createNewDir(CATALOG_DIR); + if (isWindows) { + FileUtils.deleteFile(CATALOG_ROOT_DIR_WIN + NAMESPACE_TAR); + FileUtils.deleteFile(CATALOG_ROOT_DIR_WIN + "paimon.tar"); + FileUtils.createNewDir(CATALOG_ROOT_DIR_WIN); + } else { + FileUtils.deleteFile(CATALOG_ROOT_DIR + NAMESPACE_TAR); + FileUtils.createNewDir(CATALOG_DIR); + } + container.execInContainer( "sh", "-c", @@ -510,8 +529,13 @@ public void testFakeSinkPaimonWithFullTypeAndReadWithFilter(TestContainer contai + " " + NAMESPACE); container.copyFileFromContainer( - CATALOG_ROOT_DIR + NAMESPACE_TAR, CATALOG_ROOT_DIR + NAMESPACE_TAR); - extractFiles(); + CATALOG_ROOT_DIR + NAMESPACE_TAR, + (isWindows ? CATALOG_ROOT_DIR_WIN : CATALOG_ROOT_DIR) + NAMESPACE_TAR); + if (isWindows) { + extractFilesWin(); + } else { + extractFiles(); + } }; private void extractFiles() { @@ -532,6 +556,17 @@ private void extractFiles() { } } + private void extractFilesWin() { + try { + CompressionUtils.unGzip( + new File(CATALOG_ROOT_DIR_WIN + NAMESPACE_TAR), new File(CATALOG_ROOT_DIR_WIN)); + CompressionUtils.unTar( + new File(CATALOG_ROOT_DIR_WIN + "paimon.tar"), new File(CATALOG_ROOT_DIR_WIN)); + } catch (IOException | ArchiveException e) { + throw new RuntimeException(e); + } + } + private List loadPaimonData(String dbName, String tbName) throws Exception { Table table = getTable(dbName, tbName); ReadBuilder readBuilder = table.newReadBuilder(); @@ -575,7 +610,11 @@ private Identifier getIdentifier(String dbName, String tbName) { private Catalog getCatalog() { Options options = new Options(); - options.set("warehouse", "file://" + CATALOG_DIR); + if (isWindows) { + options.set("warehouse", "file://" + CATALOG_DIR_WIN); + } else { + options.set("warehouse", "file://" + CATALOG_DIR); + } Catalog catalog = CatalogFactory.createCatalog(CatalogContext.create(options)); return catalog; } From a12786b82101ac37859deeee521b59dfd6a9cd5f Mon Sep 17 00:00:00 2001 From: Carl-Zhou-CN <1058249259@qq.com> Date: Wed, 7 Aug 2024 12:27:50 +0800 Subject: [PATCH 027/361] [DOC][Oss] fix document configuration is rectified when the oss is selected as the checkpoint base (#7332) --- docs/en/seatunnel-engine/hybrid-cluster-deployment.md | 1 - docs/en/seatunnel-engine/separated-cluster-deployment.md | 1 - docs/zh/seatunnel-engine/hybrid-cluster-deployment.md | 1 - docs/zh/seatunnel-engine/separated-cluster-deployment.md | 1 - 4 files changed, 4 deletions(-) diff --git a/docs/en/seatunnel-engine/hybrid-cluster-deployment.md b/docs/en/seatunnel-engine/hybrid-cluster-deployment.md index 60260f91bb0..534d5e69c5e 100644 --- a/docs/en/seatunnel-engine/hybrid-cluster-deployment.md +++ b/docs/en/seatunnel-engine/hybrid-cluster-deployment.md @@ -258,7 +258,6 @@ map: fs.oss.accessKeyId: OSS access key id fs.oss.accessKeySecret: OSS access key secret fs.oss.endpoint: OSS endpoint - fs.oss.credentials.provider: org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider ``` Notice: When using OSS, make sure that the following jars are in the lib directory. diff --git a/docs/en/seatunnel-engine/separated-cluster-deployment.md b/docs/en/seatunnel-engine/separated-cluster-deployment.md index 6d094aa8143..168cac8d0f0 100644 --- a/docs/en/seatunnel-engine/separated-cluster-deployment.md +++ b/docs/en/seatunnel-engine/separated-cluster-deployment.md @@ -268,7 +268,6 @@ map: fs.oss.accessKeyId: OSS access key id fs.oss.accessKeySecret: OSS access key secret fs.oss.endpoint: OSS endpoint - fs.oss.credentials.provider: org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider ``` Notice: When using OSS, make sure that the following jars are in the lib directory. diff --git a/docs/zh/seatunnel-engine/hybrid-cluster-deployment.md b/docs/zh/seatunnel-engine/hybrid-cluster-deployment.md index f1deba3dec1..4d101b41678 100644 --- a/docs/zh/seatunnel-engine/hybrid-cluster-deployment.md +++ b/docs/zh/seatunnel-engine/hybrid-cluster-deployment.md @@ -258,7 +258,6 @@ map: fs.oss.accessKeyId: OSS access key id fs.oss.accessKeySecret: OSS access key secret fs.oss.endpoint: OSS endpoint - fs.oss.credentials.provider: org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider ``` 注意:使用OSS 时,确保 lib目录下有这几个jar. diff --git a/docs/zh/seatunnel-engine/separated-cluster-deployment.md b/docs/zh/seatunnel-engine/separated-cluster-deployment.md index 807fb8d28c7..ce328d3bd57 100644 --- a/docs/zh/seatunnel-engine/separated-cluster-deployment.md +++ b/docs/zh/seatunnel-engine/separated-cluster-deployment.md @@ -272,7 +272,6 @@ map: fs.oss.accessKeyId: OSS access key id fs.oss.accessKeySecret: OSS access key secret fs.oss.endpoint: OSS endpoint - fs.oss.credentials.provider: org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider ``` 注意:使用OSS 时,确保 lib目录下有这几个jar. From f6a1e51b890582f1cf785a40b4bb38564c5652f8 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Wed, 7 Aug 2024 12:44:13 +0800 Subject: [PATCH 028/361] [E2E] Enable JdbcPostgresIdentifierIT (#7326) --- .../seatunnel/jdbc/JdbcPostgresIdentifierIT.java | 6 ------ .../connectors/seatunnel/jdbc/JdbcMySqlCreateTableIT.java | 3 ++- .../seatunnel/jdbc/JdbcSqlServerCreateTableIT.java | 3 ++- .../seatunnel/connectors/seatunnel/jdbc/JdbcIrisIT.java | 8 ++++---- 4 files changed, 8 insertions(+), 12 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcPostgresIdentifierIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcPostgresIdentifierIT.java index 13adec70084..a7094044aa1 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcPostgresIdentifierIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcPostgresIdentifierIT.java @@ -20,9 +20,7 @@ import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; -import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; -import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; import org.junit.jupiter.api.AfterAll; @@ -53,10 +51,6 @@ import static org.awaitility.Awaitility.given; @Slf4j -@DisabledOnContainer( - value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support cdc") public class JdbcPostgresIdentifierIT extends TestSuiteBase implements TestResource { private static final String PG_IMAGE = "postgis/postgis"; private static final String PG_DRIVER_JAR = diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMySqlCreateTableIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMySqlCreateTableIT.java index 7397362f354..30c67838975 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMySqlCreateTableIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMySqlCreateTableIT.java @@ -59,7 +59,8 @@ @DisabledOnContainer( value = {}, type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support cdc") + disabledReason = + "Currently testcase does not depend on a specific engine, but needs to be started with the engine") public class JdbcMySqlCreateTableIT extends TestSuiteBase implements TestResource { private static final String SQLSERVER_IMAGE = "mcr.microsoft.com/mssql/server:2022-latest"; private static final String SQLSERVER_CONTAINER_HOST = "sqlserver"; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerCreateTableIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerCreateTableIT.java index ac8d7e31c8e..ae2e625b157 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerCreateTableIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerCreateTableIT.java @@ -59,7 +59,8 @@ @DisabledOnContainer( value = {}, type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support cdc") + disabledReason = + "Currently testcase does not depend on a specific engine, but needs to be started with the engine") public class JdbcSqlServerCreateTableIT extends TestSuiteBase implements TestResource { private static final String SQLSERVER_IMAGE = "mcr.microsoft.com/mssql/server:2022-latest"; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcIrisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcIrisIT.java index 8fff364c3f8..b99c823de88 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcIrisIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcIrisIT.java @@ -66,10 +66,6 @@ import java.util.Map; import java.util.stream.Collectors; -@DisabledOnContainer( - value = {}, - type = {EngineType.SPARK}, - disabledReason = "Currently SPARK do not support cdc, temporarily disable") @Slf4j public class JdbcIrisIT extends AbstractJdbcIT { private static final String IRIS_IMAGE = "intersystems/iris-community:2023.1"; @@ -322,6 +318,10 @@ public void testCatalog() { Assertions.assertFalse(catalog.tableExists(targetTablePath)); } + @DisabledOnContainer( + value = {}, + type = {EngineType.SPARK}, + disabledReason = "Currently SPARK do not support cdc") @TestTemplate public void testUpsert(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = container.executeJob("/jdbc_iris_upsert.conf"); From 460e73ec36f1afac42c36320fb79a47a45bfc823 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Wed, 7 Aug 2024 12:44:24 +0800 Subject: [PATCH 029/361] [E2E] Enable fakesource e2e of spark/flink (#7325) --- .../seatunnel/e2e/connector/fake/FakeWithSchemaTT.java | 6 ------ .../seatunnel/e2e/connector/fake/FakeWithTableNamesTT.java | 6 ------ 2 files changed, 12 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithSchemaTT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithSchemaTT.java index b81bb7b620a..e7628d7b5da 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithSchemaTT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithSchemaTT.java @@ -18,9 +18,7 @@ package org.apache.seatunnel.e2e.connector.fake; import org.apache.seatunnel.e2e.common.TestSuiteBase; -import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; -import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.TestTemplate; @@ -28,10 +26,6 @@ import java.io.IOException; -@DisabledOnContainer( - value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support CatalogTable") public class FakeWithSchemaTT extends TestSuiteBase { @TestTemplate public void testFakeConnector(TestContainer container) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithTableNamesTT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithTableNamesTT.java index 50e58b1ae39..50b82b346e5 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithTableNamesTT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithTableNamesTT.java @@ -18,9 +18,7 @@ package org.apache.seatunnel.e2e.connector.fake; import org.apache.seatunnel.e2e.common.TestSuiteBase; -import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; -import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.TestTemplate; @@ -28,10 +26,6 @@ import java.io.IOException; -@DisabledOnContainer( - value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support CatalogTable") public class FakeWithTableNamesTT extends TestSuiteBase { @TestTemplate public void testFakeConnector(TestContainer container) From 7c3cd99e007e2010fb88791cfd424d3642066a34 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Wed, 7 Aug 2024 12:44:41 +0800 Subject: [PATCH 030/361] [Improve] Remove unused code (#7324) --- .../java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java | 3 --- .../java/org/apache/seatunnel/e2e/connector/kudu/KuduIT.java | 3 --- 2 files changed, 6 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java index d4629851e79..36b25928d9b 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java @@ -84,9 +84,6 @@ import java.util.stream.Stream; @Slf4j -@DisabledOnContainer( - value = {}, - disabledReason = "Override TestSuiteBase @DisabledOnContainer") public class KafkaIT extends TestSuiteBase implements TestResource { private static final String KAFKA_IMAGE_NAME = "confluentinc/cp-kafka:7.0.9"; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kudu-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kudu/KuduIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kudu-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kudu/KuduIT.java index 015ab0d3e35..dc7ab433868 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kudu-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kudu/KuduIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kudu-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kudu/KuduIT.java @@ -74,9 +74,6 @@ import static org.awaitility.Awaitility.await; @Slf4j -@DisabledOnContainer( - value = {}, - disabledReason = "Override TestSuiteBase @DisabledOnContainer") public class KuduIT extends TestSuiteBase implements TestResource { private static final String IMAGE = "apache/kudu:1.15.0"; From 819c6856513c9171b60d63c324e793a6eecbb9f6 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Wed, 7 Aug 2024 12:45:54 +0800 Subject: [PATCH 031/361] [Improve][Jdbc] Merge user config primary key when create table (#7313) --- .../seatunnel/jdbc/sink/JdbcSinkFactory.java | 20 +++++++++++++++++++ .../jdbc/JdbcMysqlSaveModeHandlerIT.java | 7 +++++++ .../resources/jdbc_mysql_source_and_sink.conf | 5 ++++- 3 files changed, 31 insertions(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java index eff6bb67c67..35e9a986ab8 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java @@ -26,6 +26,7 @@ import org.apache.seatunnel.api.table.catalog.ConstraintKey; import org.apache.seatunnel.api.table.catalog.PrimaryKey; import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TableSchema; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; @@ -200,6 +201,25 @@ public TableSink createSink(TableSinkFactoryContext context) { .collect(Collectors.joining(","))); } } + } else { + // replace primary key to config + PrimaryKey configPk = + PrimaryKey.of( + catalogTable.getTablePath().getTableName() + "_config_pk", + config.get(PRIMARY_KEYS)); + TableSchema tableSchema = catalogTable.getTableSchema(); + catalogTable = + CatalogTable.of( + catalogTable.getTableId(), + TableSchema.builder() + .primaryKey(configPk) + .constraintKey(tableSchema.getConstraintKeys()) + .columns(tableSchema.getColumns()) + .build(), + catalogTable.getOptions(), + catalogTable.getPartitionKeys(), + catalogTable.getComment(), + catalogTable.getCatalogName()); } config = ReadonlyConfig.fromMap(new HashMap<>(map)); // always execute diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlSaveModeHandlerIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlSaveModeHandlerIT.java index c8acc950105..bc1361aa267 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlSaveModeHandlerIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlSaveModeHandlerIT.java @@ -47,6 +47,7 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -74,6 +75,7 @@ public class JdbcMysqlSaveModeHandlerIT extends AbstractJdbcIT { private static final String CREATE_SQL = "CREATE TABLE IF NOT EXISTS %s\n" + "(\n" + + " `id` bigint(20) NOT NULL,\n" + " `c_bit_1` bit(1) DEFAULT NULL,\n" + " `c_bit_8` bit(8) DEFAULT NULL,\n" + " `c_bit_16` bit(16) DEFAULT NULL,\n" @@ -164,6 +166,9 @@ void compareResult(String executeKey) { final List columns = table.getTableSchema().getColumns(); Assertions.assertEquals(columns.size(), columnsSource.size()); + Assertions.assertIterableEquals( + Collections.singletonList("id"), + table.getTableSchema().getPrimaryKey().getColumnNames()); } @Override @@ -175,6 +180,7 @@ String driverUrl() { Pair> initTestData() { String[] fieldNames = new String[] { + "id", "c_bit_1", "c_bit_8", "c_bit_16", @@ -229,6 +235,7 @@ Pair> initTestData() { SeaTunnelRow row = new SeaTunnelRow( new Object[] { + (long) i, i % 2 == 0 ? (byte) 1 : (byte) 0, new byte[] {byteArr}, new byte[] {byteArr, byteArr}, diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_mysql_source_and_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_mysql_source_and_sink.conf index bc379f8ba8a..6305f55c46b 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_mysql_source_and_sink.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_mysql_source_and_sink.conf @@ -40,9 +40,12 @@ sink { driver = "com.mysql.cj.jdbc.Driver" user = "root" password = "Abc!@#135_seatunnel" + generate_sink_sql = true - table = "test_laowang" database = "seatunnel" + table = "test_laowang" + primary_keys = ["id"] + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" data_save_mode="APPEND_DATA" } From 891652399e8b97fb5cf1c7e6bdee87e7ec48469e Mon Sep 17 00:00:00 2001 From: hailin0 Date: Wed, 7 Aug 2024 12:46:12 +0800 Subject: [PATCH 032/361] [Hotfix][Connector] Fix kafka consumer log next startup offset (#7312) --- .../kafka/source/KafkaSourceReader.java | 10 ++- .../kafka/source/KafkaSourceSplit.java | 4 + .../e2e/connector/kafka/KafkaIT.java | 62 +++++++++++++++ ..._offset_to_console_with_commit_offset.conf | 77 +++++++++++++++++++ 4 files changed, 151 insertions(+), 2 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka/kafkasource_group_offset_to_console_with_commit_offset.conf diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java index 02c2a9007e1..6f4753110bd 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java @@ -218,8 +218,14 @@ public void pollNext(Collector output) throws Exception { } }); if (Boundedness.BOUNDED.equals(context.getBoundedness())) { - finishedSplits.forEach(sourceSplits::remove); - if (sourceSplits.isEmpty()) { + for (KafkaSourceSplit split : finishedSplits) { + split.setFinish(true); + if (split.getStartOffset() == -1) { + // log next running read start offset + split.setStartOffset(split.getEndOffset()); + } + } + if (sourceSplits.stream().allMatch(KafkaSourceSplit::isFinish)) { context.signalNoMoreElement(); } } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java index 1c7cb17678f..8f5bc5f2d31 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java @@ -22,6 +22,9 @@ import org.apache.kafka.common.TopicPartition; +import lombok.Getter; +import lombok.Setter; + import java.util.Objects; public class KafkaSourceSplit implements SourceSplit { @@ -30,6 +33,7 @@ public class KafkaSourceSplit implements SourceSplit { private TopicPartition topicPartition; private long startOffset = -1L; private long endOffset = -1L; + @Setter @Getter private transient volatile boolean finish = false; public KafkaSourceSplit(TablePath tablePath, TopicPartition topicPartition) { this.tablePath = tablePath; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java index 36b25928d9b..0d9f5d5ef8a 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java @@ -35,12 +35,15 @@ import org.apache.seatunnel.connectors.seatunnel.kafka.serialize.DefaultSeaTunnelRowSerializer; import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; import org.apache.seatunnel.e2e.common.container.TestContainerId; import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.apache.seatunnel.format.avro.AvroDeserializationSchema; import org.apache.seatunnel.format.text.TextSerializationSchema; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsOptions; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -80,6 +83,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.stream.Stream; @@ -315,6 +319,23 @@ public void testSourceKafkaStartConfig(TestContainer container) testKafkaGroupOffsetsToConsole(container); } + @DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}, + disabledReason = "flink and spark won't commit offset when batch job finished") + @TestTemplate + public void testSourceKafkaStartConfigWithCommitOffset(TestContainer container) + throws Exception { + DefaultSeaTunnelRowSerializer serializer = + DefaultSeaTunnelRowSerializer.create( + "test_topic_group_with_commit_offset", + SEATUNNEL_ROW_TYPE, + DEFAULT_FORMAT, + DEFAULT_FIELD_DELIMITER); + generateTestData(row -> serializer.serializeRow(row), 0, 100); + testKafkaGroupOffsetsToConsoleWithCommitOffset(container); + } + @TestTemplate @DisabledOnContainer(value = {TestContainerId.SPARK_2_4}) public void testFakeSourceToKafkaAvroFormat(TestContainer container) @@ -511,6 +532,40 @@ public void testKafkaGroupOffsetsToConsole(TestContainer container) Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); } + public void testKafkaGroupOffsetsToConsoleWithCommitOffset(TestContainer container) + throws IOException, InterruptedException, ExecutionException { + Container.ExecResult execResult = + container.executeJob( + "/kafka/kafkasource_group_offset_to_console_with_commit_offset.conf"); + Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + + String consumerGroup = "SeaTunnel-Consumer-Group"; + TopicPartition topicPartition = + new TopicPartition("test_topic_group_with_commit_offset", 0); + try (AdminClient adminClient = createKafkaAdmin()) { + ListConsumerGroupOffsetsOptions options = + new ListConsumerGroupOffsetsOptions() + .topicPartitions(Arrays.asList(topicPartition)); + Map topicOffset = + adminClient + .listConsumerGroupOffsets(consumerGroup, options) + .partitionsToOffsetAndMetadata() + .thenApply( + result -> { + Map offsets = new HashMap<>(); + result.forEach( + (tp, oam) -> { + if (oam != null) { + offsets.put(tp, oam.offset()); + } + }); + return offsets; + }) + .get(); + Assertions.assertEquals(100L, topicOffset.get(topicPartition)); + } + } + public void testKafkaTimestampToConsole(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = @@ -518,6 +573,13 @@ public void testKafkaTimestampToConsole(TestContainer container) Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); } + private AdminClient createKafkaAdmin() { + Properties props = new Properties(); + String bootstrapServers = kafkaContainer.getBootstrapServers(); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + return AdminClient.create(props); + } + private void initKafkaProducer() { Properties props = new Properties(); String bootstrapServers = kafkaContainer.getBootstrapServers(); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka/kafkasource_group_offset_to_console_with_commit_offset.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka/kafkasource_group_offset_to_console_with_commit_offset.conf new file mode 100644 index 00000000000..e054c03964c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka/kafkasource_group_offset_to_console_with_commit_offset.conf @@ -0,0 +1,77 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" + read_limit.bytes_per_second=7000000 + read_limit.rows_per_second=400 +} + +source { + Kafka { + commit_on_checkpoint = true + consumer.group = "SeaTunnel-Consumer-Group" + + bootstrap.servers = "kafkaCluster:9092" + topic = "test_topic_group_with_commit_offset" + result_table_name = "kafka_table" + # The default format is json, which is optional + format = json + start_mode = group_offsets + schema = { + fields { + id = bigint + } + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of source plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/source/KafkaSource +} + +transform { +} + +sink { + Assert { + source_table_name = "kafka_table" + rules = + { + field_rules = [ + { + field_name = id + field_type = bigint + field_value = [ + + { + rule_type = MIN + rule_value = 100 + }, + { + rule_type = MAX + rule_value = 149 + } + ] + } + ] + } + } +} \ No newline at end of file From 764d8b0bc8a12100bf2ef07c9b1488ba2bda70ab Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Wed, 7 Aug 2024 12:46:30 +0800 Subject: [PATCH 033/361] [Improve][API] Make sure the table name in TablePath not be null (#7252) --- .../api/sink/DefaultSaveModeHandler.java | 7 +-- .../api/table/catalog/TableIdentifier.java | 18 +++++-- .../api/table/catalog/TablePath.java | 16 ++++-- .../api/sink/TablePlaceholderTest.java | 10 ++-- .../GoogleSheetsDeserializerTest.java | 2 +- .../seatunnel/http/source/HttpSource.java | 3 +- .../source/PulsarCanalDecoratorTest.java | 2 +- .../e2e/connector/kafka/KafkaIT.java | 4 +- .../server/task/SeaTunnelSourceCollector.java | 16 ++---- .../server/task/flow/SinkFlowLifeCycle.java | 14 ++--- .../avro/AvroSerializationSchemaTest.java | 2 +- .../json/JsonRowDataSerDeSchemaTest.java | 18 +++---- .../json/canal/CanalJsonSerDeSchemaTest.java | 54 +++++++++---------- .../debezium/DebeziumJsonSerDeSchemaTest.java | 42 +++++++-------- .../maxwell/MaxWellJsonSerDeSchemaTest.java | 2 +- .../json/ogg/OggJsonSerDeSchemaTest.java | 42 +++++++-------- 16 files changed, 129 insertions(+), 123 deletions(-) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DefaultSaveModeHandler.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DefaultSaveModeHandler.java index bbbe99281b2..e22dd7c99a5 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DefaultSaveModeHandler.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DefaultSaveModeHandler.java @@ -153,19 +153,16 @@ protected void dropTable() { protected void createTable() { if (!catalog.databaseExists(tablePath.getDatabaseName())) { - TablePath databasePath = TablePath.of(tablePath.getDatabaseName(), ""); try { log.info( "Creating database {} with action {}", tablePath.getDatabaseName(), catalog.previewAction( - Catalog.ActionType.CREATE_DATABASE, - databasePath, - Optional.empty())); + Catalog.ActionType.CREATE_DATABASE, tablePath, Optional.empty())); } catch (UnsupportedOperationException ignore) { log.info("Creating database {}", tablePath.getDatabaseName()); } - catalog.createDatabase(databasePath, true); + catalog.createDatabase(tablePath, true); } try { log.info( diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TableIdentifier.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TableIdentifier.java index 2d39f9b9842..101081255cc 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TableIdentifier.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TableIdentifier.java @@ -17,15 +17,16 @@ package org.apache.seatunnel.api.table.catalog; +import org.apache.commons.lang3.StringUtils; + import lombok.EqualsAndHashCode; import lombok.Getter; -import lombok.RequiredArgsConstructor; +import lombok.NonNull; import java.io.Serializable; @Getter @EqualsAndHashCode -@RequiredArgsConstructor public final class TableIdentifier implements Serializable { private static final long serialVersionUID = 1L; @@ -35,7 +36,18 @@ public final class TableIdentifier implements Serializable { private final String schemaName; - private final String tableName; + @NonNull private final String tableName; + + public TableIdentifier( + String catalogName, String databaseName, String schemaName, @NonNull String tableName) { + this.catalogName = catalogName; + this.databaseName = databaseName; + this.schemaName = schemaName; + this.tableName = tableName; + if (StringUtils.isEmpty(tableName)) { + throw new IllegalArgumentException("tableName cannot be empty"); + } + } public static TableIdentifier of(String catalogName, String databaseName, String tableName) { return new TableIdentifier(catalogName, databaseName, null, tableName); diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TablePath.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TablePath.java index 12572621874..30edc7ac80e 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TablePath.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/TablePath.java @@ -17,9 +17,11 @@ package org.apache.seatunnel.api.table.catalog; +import org.apache.commons.lang3.StringUtils; + import lombok.EqualsAndHashCode; import lombok.Getter; -import lombok.RequiredArgsConstructor; +import lombok.NonNull; import java.io.Serializable; import java.util.ArrayList; @@ -27,12 +29,20 @@ @Getter @EqualsAndHashCode -@RequiredArgsConstructor public final class TablePath implements Serializable { private static final long serialVersionUID = 1L; private final String databaseName; private final String schemaName; - private final String tableName; + @NonNull private final String tableName; + + public TablePath(String databaseName, String schemaName, @NonNull String tableName) { + this.databaseName = databaseName; + this.schemaName = schemaName; + this.tableName = tableName; + if (StringUtils.isEmpty(tableName)) { + throw new IllegalArgumentException("tableName cannot be empty"); + } + } public static final TablePath DEFAULT = TablePath.of("default", "default", "default"); diff --git a/seatunnel-api/src/test/java/org/apache/seatunnel/api/sink/TablePlaceholderTest.java b/seatunnel-api/src/test/java/org/apache/seatunnel/api/sink/TablePlaceholderTest.java index 1a87a53f97f..16a69d5db3d 100644 --- a/seatunnel-api/src/test/java/org/apache/seatunnel/api/sink/TablePlaceholderTest.java +++ b/seatunnel-api/src/test/java/org/apache/seatunnel/api/sink/TablePlaceholderTest.java @@ -77,7 +77,7 @@ public void testSinkOptions() { @Test public void testSinkOptionsWithNoTablePath() { ReadonlyConfig config = createConfig(); - CatalogTable table = createTestTableWithNoTablePath(); + CatalogTable table = createTestTableWithNoDatabaseAndSchemaName(); ReadonlyConfig newConfig = TablePlaceholder.replaceTablePlaceholder(config, table); Assertions.assertEquals("xyz_default_db_test", newConfig.get(DATABASE)); @@ -95,7 +95,7 @@ public void testSinkOptionsWithNoTablePath() { @Test public void testSinkOptionsWithExcludeKeys() { ReadonlyConfig config = createConfig(); - CatalogTable table = createTestTableWithNoTablePath(); + CatalogTable table = createTestTableWithNoDatabaseAndSchemaName(); ReadonlyConfig newConfig = TablePlaceholder.replaceTablePlaceholder( config, table, Arrays.asList(DATABASE.key())); @@ -116,7 +116,7 @@ public void testSinkOptionsWithExcludeKeys() { public void testSinkOptionsWithMultiTable() { ReadonlyConfig config = createConfig(); CatalogTable table1 = createTestTable(); - CatalogTable table2 = createTestTableWithNoTablePath(); + CatalogTable table2 = createTestTableWithNoDatabaseAndSchemaName(); ReadonlyConfig newConfig1 = TablePlaceholder.replaceTablePlaceholder(config, table1, Arrays.asList()); ReadonlyConfig newConfig2 = @@ -159,8 +159,8 @@ private static ReadonlyConfig createConfig() { return ReadonlyConfig.fromMap(configMap); } - private static CatalogTable createTestTableWithNoTablePath() { - TableIdentifier tableId = TableIdentifier.of("my-catalog", null, null, null); + private static CatalogTable createTestTableWithNoDatabaseAndSchemaName() { + TableIdentifier tableId = TableIdentifier.of("my-catalog", null, null, "default_table"); TableSchema tableSchema = TableSchema.builder() .primaryKey(PrimaryKey.of("my-pk", Arrays.asList("f1", "f2"))) diff --git a/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java b/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java index c55228471c2..e2e3139d887 100644 --- a/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java +++ b/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java @@ -42,7 +42,7 @@ public void testJsonParseError() { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); final DeserializationSchema deser = new JsonDeserializationSchema(catalogTables, false, false); diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java index 754a7b93664..c41e8a9a84a 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java @@ -28,6 +28,7 @@ import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.PhysicalColumn; import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.TableSchema; import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.BasicType; @@ -146,7 +147,7 @@ protected void buildSchemaWithConfig(Config pluginConfig) { } } else { TableIdentifier tableIdentifier = - TableIdentifier.of(HttpConfig.CONNECTOR_IDENTITY, null, null); + TableIdentifier.of(HttpConfig.CONNECTOR_IDENTITY, TablePath.DEFAULT); TableSchema tableSchema = TableSchema.builder() .column( diff --git a/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java b/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java index 7b1ee39fd48..ee5e1513fb1 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java @@ -58,7 +58,7 @@ void decoder() throws IOException { SeaTunnelRowType seaTunnelRowType = new SeaTunnelRowType(fieldNames, dataTypes); CatalogTable catalogTables = - CatalogTableUtil.getCatalogTable("", "", "", "", seaTunnelRowType); + CatalogTableUtil.getCatalogTable("", "", "", "test", seaTunnelRowType); CanalJsonDeserializationSchema canalJsonDeserializationSchema = CanalJsonDeserializationSchema.builder(catalogTables).build(); PulsarCanalDecorator pulsarCanalDecorator = diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java index 0d9f5d5ef8a..6e67aa021d1 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java @@ -412,7 +412,7 @@ public void testFakeSourceToKafkaAvroFormat(TestContainer container) }; SeaTunnelRowType fake_source_row_type = new SeaTunnelRowType(fieldNames, fieldTypes); CatalogTable catalogTable = - CatalogTableUtil.getCatalogTable("", "", "", "", fake_source_row_type); + CatalogTableUtil.getCatalogTable("", "", "", "test", fake_source_row_type); AvroDeserializationSchema avroDeserializationSchema = new AvroDeserializationSchema(catalogTable); List kafkaSTRow = @@ -464,7 +464,7 @@ public void testKafkaAvroToAssert(TestContainer container) Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); CatalogTable catalogTable = - CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); + CatalogTableUtil.getCatalogTable("", "", "", "test", SEATUNNEL_ROW_TYPE); AvroDeserializationSchema avroDeserializationSchema = new AvroDeserializationSchema(catalogTable); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java index 62612d0617a..e1b24947893 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java @@ -102,9 +102,11 @@ public SeaTunnelSourceCollector( tablePaths.forEach( tablePath -> sourceReceivedCountPerTable.put( - getFullName(tablePath), + tablePath.getFullName(), metricsContext.counter( - SOURCE_RECEIVED_COUNT + "#" + getFullName(tablePath)))); + SOURCE_RECEIVED_COUNT + + "#" + + tablePath.getFullName()))); } sourceReceivedCount = metricsContext.counter(SOURCE_RECEIVED_COUNT); sourceReceivedQPS = metricsContext.meter(SOURCE_RECEIVED_QPS); @@ -131,7 +133,7 @@ public void collect(T row) { sourceReceivedBytesPerSeconds.markEvent(size); flowControlGate.audit((SeaTunnelRow) row); if (StringUtils.isNotEmpty(tableId)) { - String tableName = getFullName(TablePath.of(tableId)); + String tableName = TablePath.of(tableId).getFullName(); Counter sourceTableCounter = sourceReceivedCountPerTable.get(tableName); if (Objects.nonNull(sourceTableCounter)) { sourceTableCounter.inc(); @@ -232,12 +234,4 @@ public void sendRecordToNext(Record record) throws IOException { } } } - - private String getFullName(TablePath tablePath) { - if (StringUtils.isBlank(tablePath.getTableName())) { - tablePath = - TablePath.of(tablePath.getDatabaseName(), tablePath.getSchemaName(), "default"); - } - return tablePath.getFullName(); - } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java index 516e1c97c41..de8257f1e94 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java @@ -138,9 +138,9 @@ public SinkFlowLifeCycle( sinkTables.forEach( tablePath -> sinkWriteCountPerTable.put( - getFullName(tablePath), + tablePath.getFullName(), metricsContext.counter( - SINK_WRITE_COUNT + "#" + getFullName(tablePath)))); + SINK_WRITE_COUNT + "#" + tablePath.getFullName()))); } } @@ -275,7 +275,7 @@ public void received(Record record) { sinkWriteBytesPerSeconds.markEvent(size); String tableId = ((SeaTunnelRow) record.getData()).getTableId(); if (StringUtils.isNotBlank(tableId)) { - String tableName = getFullName(TablePath.of(tableId)); + String tableName = TablePath.of(tableId).getFullName(); Counter sinkTableCounter = sinkWriteCountPerTable.get(tableName); if (Objects.nonNull(sinkTableCounter)) { sinkTableCounter.inc(); @@ -345,12 +345,4 @@ public void restoreState(List actionStateList) throws Except ((SupportResourceShare) this.writer).setMultiTableResourceManager(resourceManager, 0); } } - - private String getFullName(TablePath tablePath) { - if (StringUtils.isBlank(tablePath.getTableName())) { - tablePath = - TablePath.of(tablePath.getDatabaseName(), tablePath.getSchemaName(), "default"); - } - return tablePath.getFullName(); - } } diff --git a/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java b/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java index 1e1554be712..42b8029f16c 100644 --- a/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java @@ -160,7 +160,7 @@ private SeaTunnelRowType buildSeaTunnelRowType() { @Test public void testSerialization() throws IOException { SeaTunnelRowType rowType = buildSeaTunnelRowType(); - CatalogTable catalogTable = CatalogTableUtil.getCatalogTable("", "", "", "", rowType); + CatalogTable catalogTable = CatalogTableUtil.getCatalogTable("", "", "", "test", rowType); SeaTunnelRow seaTunnelRow = buildSeaTunnelRow(); AvroSerializationSchema serializationSchema = new AvroSerializationSchema(rowType); byte[] bytes = serializationSchema.serialize(seaTunnelRow); diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java index fb6fd9da767..beda96ff6e7 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java @@ -170,7 +170,7 @@ public void testSerDe() throws Exception { new MapType(STRING_TYPE, new MapType(STRING_TYPE, INT_TYPE)) }) }); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); JsonDeserializationSchema deserializationSchema = new JsonDeserializationSchema(catalogTables, false, false); @@ -230,7 +230,7 @@ public void testSerDeMultiRows() throws Exception { new SeaTunnelDataType[] {STRING_TYPE, INT_TYPE}) }); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); JsonDeserializationSchema deserializationSchema = new JsonDeserializationSchema(catalogTables, false, false); @@ -308,7 +308,7 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { new MapType(STRING_TYPE, DOUBLE_TYPE) }); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", rowType); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", rowType); JsonDeserializationSchema deserializationSchema = new JsonDeserializationSchema(catalogTables, false, true); @@ -327,7 +327,7 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { public void testDeserializationNullRow() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); JsonDeserializationSchema deserializationSchema = new JsonDeserializationSchema(catalogTables, true, false); @@ -339,7 +339,7 @@ public void testDeserializationNullRow() throws Exception { public void testDeserializationMissingNode() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); JsonDeserializationSchema deserializationSchema = new JsonDeserializationSchema(catalogTables, true, false); @@ -359,7 +359,7 @@ public void testDeserializationPassMissingField() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); // pass on missing field final JsonDeserializationSchema deser = @@ -382,7 +382,7 @@ public void testDeserializationMissingField() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); // fail on missing field final JsonDeserializationSchema deser = @@ -418,7 +418,7 @@ public void testDeserializationIgnoreParseError() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); SeaTunnelRow expected = new SeaTunnelRow(1); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); // ignore on parse error final JsonDeserializationSchema deser = @@ -446,7 +446,7 @@ public void testDeserializationNoJson() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); - CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "test", schema); String noJson = "{]"; final JsonDeserializationSchema deser = diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java index d35849e8bd4..efd639cd7b1 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java @@ -54,7 +54,7 @@ public class CanalJsonSerDeSchemaTest { new String[] {"id", "name", "description", "weight"}, new SeaTunnelDataType[] {INT_TYPE, STRING_TYPE, STRING_TYPE, FLOAT_TYPE}); private static final CatalogTable catalogTables = - CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); + CatalogTableUtil.getCatalogTable("", "", "", "test", SEATUNNEL_ROW_TYPE); @Test public void testFilteringTables() throws Exception { @@ -167,32 +167,32 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali List expected = Arrays.asList( - "SeaTunnelRow{tableId=.., kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[106, hammer, null, 1.0]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[106, hammer, null, 1.0]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=.., kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[101, scooter, Small 2-wheel scooter, 5.17]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[102, car battery, 12V car battery, 5.17]}", - "SeaTunnelRow{tableId=.., kind=-D, fields=[102, car battery, 12V car battery, 5.17]}", - "SeaTunnelRow{tableId=.., kind=-D, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}"); + "SeaTunnelRow{tableId=..test, kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[106, hammer, null, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[106, hammer, null, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=..test, kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[101, scooter, Small 2-wheel scooter, 5.17]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[102, car battery, 12V car battery, 5.17]}", + "SeaTunnelRow{tableId=..test, kind=-D, fields=[102, car battery, 12V car battery, 5.17]}", + "SeaTunnelRow{tableId=..test, kind=-D, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}"); List actual = collector.list.stream().map(Object::toString).collect(Collectors.toList()); assertEquals(expected, actual); diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java index 67d499efd91..a970aea55ae 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -55,7 +55,7 @@ public class DebeziumJsonSerDeSchemaTest { new String[] {"id", "name", "description", "weight"}, new SeaTunnelDataType[] {INT_TYPE, STRING_TYPE, STRING_TYPE, FLOAT_TYPE}); private static final CatalogTable catalogTables = - CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); + CatalogTableUtil.getCatalogTable("", "", "", "test", SEATUNNEL_ROW_TYPE); @Test void testNullRowMessages() throws Exception { @@ -175,26 +175,26 @@ private void testSerializationDeserialization(String resourceFile, boolean schem List expected = Arrays.asList( - "SeaTunnelRow{tableId=.., kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=.., kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); + "SeaTunnelRow{tableId=..test, kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=..test, kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); List actual = collector.list.stream().map(Object::toString).collect(Collectors.toList()); assertEquals(expected, actual); diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java index a4e06ac2b18..f82b272cf71 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java @@ -50,7 +50,7 @@ public class MaxWellJsonSerDeSchemaTest { new String[] {"id", "name", "description", "weight"}, new SeaTunnelDataType[] {INT_TYPE, STRING_TYPE, STRING_TYPE, FLOAT_TYPE}); private static final CatalogTable catalogTables = - CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); + CatalogTableUtil.getCatalogTable("", "", "", "test", SEATUNNEL_ROW_TYPE); @Test public void testFilteringTables() throws Exception { diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java index 04fea16ecad..20df0d945ab 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java @@ -55,7 +55,7 @@ public class OggJsonSerDeSchemaTest { new String[] {"id", "name", "description", "weight"}, new SeaTunnelDataType[] {INT_TYPE, STRING_TYPE, STRING_TYPE, FLOAT_TYPE}); private static final CatalogTable catalogTables = - CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); + CatalogTableUtil.getCatalogTable("", "", "", "test", SEATUNNEL_ROW_TYPE); @Test public void testFilteringTables() throws Exception { @@ -172,26 +172,26 @@ public void runTest(List lines, OggJsonDeserializationSchema deserializa List expected = Arrays.asList( - "SeaTunnelRow{tableId=.., kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=.., kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", - "SeaTunnelRow{tableId=.., kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=.., kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=.., kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); + "SeaTunnelRow{tableId=..test, kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=..test, kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", + "SeaTunnelRow{tableId=..test, kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=..test, kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=..test, kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); List actual = collector.list.stream().map(Object::toString).collect(Collectors.toList()); assertEquals(expected, actual); From 191d9e18b911a6b17e804ea26da618ed6f3d2df5 Mon Sep 17 00:00:00 2001 From: ChunFuWu <319355703@qq.com> Date: Wed, 7 Aug 2024 14:53:51 +0800 Subject: [PATCH 034/361] [FIX][SFTP] Fix username parameter error in sftp sink document (#7334) --- docs/en/connector-v2/sink/SftpFile.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/connector-v2/sink/SftpFile.md b/docs/en/connector-v2/sink/SftpFile.md index d06292bafda..7fdb542a2a6 100644 --- a/docs/en/connector-v2/sink/SftpFile.md +++ b/docs/en/connector-v2/sink/SftpFile.md @@ -36,7 +36,7 @@ By default, we use 2PC commit to ensure `exactly-once` |---------------------------------------|---------|----------|--------------------------------------------|-------------------------------------------------------------------------------------------------------------------| | host | string | yes | - | | | port | int | yes | - | | -| username | string | yes | - | | +| user | string | yes | - | | | password | string | yes | - | | | path | string | yes | - | | | tmp_path | string | yes | /tmp/seatunnel | The result file will write to a tmp path first and then use `mv` to submit tmp dir to target dir. Need a FTP dir. | @@ -72,9 +72,9 @@ The target sftp host is required The target sftp port is required -### username [string] +### user [string] -The target sftp username is required +The target sftp user is required ### password [string] @@ -229,7 +229,7 @@ For text file format with `have_partition` and `custom_filename` and `sink_colum SftpFile { host = "xxx.xxx.xxx.xxx" port = 22 - username = "username" + user = "username" password = "password" path = "/data/sftp/seatunnel/job1" tmp_path = "/data/sftp/seatunnel/tmp" From 16eeb1c12323b462474ad23103de96c0060a4537 Mon Sep 17 00:00:00 2001 From: He Wang Date: Wed, 7 Aug 2024 14:54:46 +0800 Subject: [PATCH 035/361] [Improve][Jdbc] Remove MysqlType references in JdbcDialect (#7333) --- .../jdbc/internal/dialect/JdbcDialect.java | 14 ++++---------- .../jdbc/internal/dialect/mysql/MysqlDialect.java | 7 +++---- 2 files changed, 7 insertions(+), 14 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java index e59776b6f95..f98f2cb3129 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java @@ -40,8 +40,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.mysql.cj.MysqlType; - import java.io.Serializable; import java.sql.Connection; import java.sql.PreparedStatement; @@ -532,8 +530,7 @@ default String buildAlterTableSql( "ALTER TABLE %s drop column %s", tableName, quoteIdentifier(oldColumnName)); } TypeConverter typeConverter = ConverterLoader.loadTypeConverter(dialectName()); - BasicTypeDefine typeBasicTypeDefine = - (BasicTypeDefine) typeConverter.reconvert(newColumn); + BasicTypeDefine typeBasicTypeDefine = (BasicTypeDefine) typeConverter.reconvert(newColumn); String basicSql = buildAlterTableBasicSql(alterOperation, tableName); basicSql = @@ -616,8 +613,7 @@ default String decorateWithColumnNameAndType( * @param typeBasicTypeDefine type basic type define of new column * @return alter table sql with nullable for sink table */ - default String decorateWithNullable( - String basicSql, BasicTypeDefine typeBasicTypeDefine) { + default String decorateWithNullable(String basicSql, BasicTypeDefine typeBasicTypeDefine) { StringBuilder sql = new StringBuilder(basicSql); if (typeBasicTypeDefine.isNullable()) { sql.append("NULL "); @@ -634,8 +630,7 @@ default String decorateWithNullable( * @param typeBasicTypeDefine type basic type define of new column * @return alter table sql with default value for sink table */ - default String decorateWithDefaultValue( - String basicSql, BasicTypeDefine typeBasicTypeDefine) { + default String decorateWithDefaultValue(String basicSql, BasicTypeDefine typeBasicTypeDefine) { Object defaultValue = typeBasicTypeDefine.getDefaultValue(); if (Objects.nonNull(defaultValue) && needsQuotesWithDefaultValue(typeBasicTypeDefine.getColumnType()) @@ -656,8 +651,7 @@ && needsQuotesWithDefaultValue(typeBasicTypeDefine.getColumnType()) * @param typeBasicTypeDefine type basic type define of new column * @return alter table sql with comment for sink table */ - default String decorateWithComment( - String basicSql, BasicTypeDefine typeBasicTypeDefine) { + default String decorateWithComment(String basicSql, BasicTypeDefine typeBasicTypeDefine) { String comment = typeBasicTypeDefine.getComment(); StringBuilder sql = new StringBuilder(basicSql); if (StringUtils.isNotBlank(comment)) { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java index 73ef12bc47b..fd0af3d9ffd 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java @@ -245,13 +245,12 @@ public void refreshTableSchemaBySchemaChangeEvent( } @Override - public String decorateWithComment( - String basicSql, BasicTypeDefine mysqlTypeBasicTypeDefine) { - MysqlType nativeType = mysqlTypeBasicTypeDefine.getNativeType(); + public String decorateWithComment(String basicSql, BasicTypeDefine typeBasicTypeDefine) { + MysqlType nativeType = (MysqlType) typeBasicTypeDefine.getNativeType(); if (NOT_SUPPORTED_DEFAULT_VALUES.contains(nativeType)) { return basicSql; } - return JdbcDialect.super.decorateWithComment(basicSql, mysqlTypeBasicTypeDefine); + return JdbcDialect.super.decorateWithComment(basicSql, typeBasicTypeDefine); } @Override From 064fcad36d3f0d152b2ebc2be926ec0ed40972ab Mon Sep 17 00:00:00 2001 From: lizhenglei <127465317+jackyyyyyssss@users.noreply.github.com> Date: Wed, 7 Aug 2024 21:57:57 +0800 Subject: [PATCH 036/361] [Improve][Transform] Improve DynamicCompile transform (#7319) --- docs/en/transform-v2/dynamic-compile.md | 47 +++-- docs/zh/transform-v2/dynamic-compile.md | 171 ++++++++++++++++++ .../e2e/transform/TestDynamicCompileIT.java | 80 +++++++- .../conf/mockserver-config.json | 18 ++ ...single_dynamic_http_compile_transform.conf | 115 ++++++++++++ 5 files changed, 416 insertions(+), 15 deletions(-) create mode 100644 docs/zh/transform-v2/dynamic-compile.md create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/mockserver-config.json create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_http_compile_transform.conf diff --git a/docs/en/transform-v2/dynamic-compile.md b/docs/en/transform-v2/dynamic-compile.md index 4a772e8cbf0..17e3b0047ee 100644 --- a/docs/en/transform-v2/dynamic-compile.md +++ b/docs/en/transform-v2/dynamic-compile.md @@ -4,6 +4,13 @@ ## Description +:::tip + +important clause +You need to ensure the security of your service and prevent attackers from uploading destructive code + +::: + Provide a programmable way to process rows, allowing users to customize any business behavior, even RPC requests based on existing row fields as parameters, or to expand fields by retrieving associated data from other data sources. To distinguish businesses, you can also define multiple transforms to combine, If the conversion is too complex, it may affect performance @@ -55,7 +62,7 @@ The data read from source is a table like this: transform { DynamicCompile { source_table_name = "fake" - result_table_name = "fake1" + result_table_name = "groovy_out" compile_language="GROOVY" compile_pattern="SOURCE_CODE" source_code=""" @@ -70,7 +77,7 @@ transform { List columns = new ArrayList<>(); PhysicalColumn destColumn = PhysicalColumn.of( - "aa", + "compile_language", BasicType.STRING_TYPE, 10, true, @@ -81,7 +88,7 @@ transform { } public Object[] getInlineOutputFieldValues(SeaTunnelRowAccessor inputRow) { Object[] fieldValues = new Object[1]; - fieldValues[0]="AA" + fieldValues[0]="GROOVY" return fieldValues; } };""" @@ -92,7 +99,7 @@ transform { transform { DynamicCompile { source_table_name = "fake" - result_table_name = "fake1" + result_table_name = "java_out" compile_language="JAVA" compile_pattern="SOURCE_CODE" source_code=""" @@ -106,7 +113,7 @@ transform { ArrayList columns = new ArrayList(); PhysicalColumn destColumn = PhysicalColumn.of( - "aa", + "compile_language", BasicType.STRING_TYPE, 10, true, @@ -119,7 +126,7 @@ transform { } public Object[] getInlineOutputFieldValues(SeaTunnelRowAccessor inputRow) { Object[] fieldValues = new Object[1]; - fieldValues[0]="AA"; + fieldValues[0]="JAVA"; return fieldValues; } """ @@ -130,7 +137,7 @@ transform { transform { DynamicCompile { source_table_name = "fake" - result_table_name = "fake1" + result_table_name = "groovy_out" compile_language="GROOVY" compile_pattern="ABSOLUTE_PATH" absolute_path="""/tmp/GroovyFile""" @@ -139,14 +146,26 @@ transform { } ``` -Then the data in result table `fake1` will like this +Then the data in result table `groovy_out` will like this + +| name | age | card | compile_language | +|----------|-----|------|------------------| +| Joy Ding | 20 | 123 | GROOVY | +| May Ding | 20 | 123 | GROOVY | +| Kin Dom | 20 | 123 | GROOVY | +| Joy Dom | 20 | 123 | GROOVY | + +Then the data in result table `java_out` will like this + +| name | age | card | compile_language | +|----------|-----|------|------------------| +| Joy Ding | 20 | 123 | JAVA | +| May Ding | 20 | 123 | JAVA | +| Kin Dom | 20 | 123 | JAVA | +| Joy Dom | 20 | 123 | JAVA | -| name | age | card | aa | -|----------|-----|------|----| -| Joy Ding | 20 | 123 | AA | -| May Ding | 20 | 123 | AA | -| Kin Dom | 20 | 123 | AA | -| Joy Dom | 20 | 123 | AA | +More complex examples can be referred to +https://github.com/apache/seatunnel/tree/dev/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf ## Changelog diff --git a/docs/zh/transform-v2/dynamic-compile.md b/docs/zh/transform-v2/dynamic-compile.md new file mode 100644 index 00000000000..0fef5c253e3 --- /dev/null +++ b/docs/zh/transform-v2/dynamic-compile.md @@ -0,0 +1,171 @@ +# DynamicCompile + +> 动态编译插件 + +## 描述 + +:::tip + +特别申明 +您需要确保服务的安全性,并防止攻击者上传破坏性代码 + +::: + +提供一种可编程的方式来处理行,允许用户自定义任何业务行为,甚至基于现有行字段作为参数的RPC请求,或者通过从其他数据源检索相关数据来扩展字段。为了区分业务,您还可以定义多个转换进行组合, +如果转换过于复杂,可能会影响性能 + +## 属性 + +| name | type | required | default value | +|------------------|--------|----------|---------------| +| source_code | string | no | | +| compile_language | Enum | yes | | +| compile_pattern | Enum | no | SOURCE_CODE | +| absolute_path | string | no | | + +### source_code [string] + +代码必须实现两个方法:getInlineOutputColumns和getInlineOutputFieldValues。getInlineOutputColumns确定要添加或转换的列,原始列结构可以从CatalogTable中获得 +GetInlineOutputFieldValues决定您的列值。您可以满足任何要求,甚至可以完成RPC请求以基于原始列获取新值 +如果有第三方依赖包,请将它们放在${SEATUNNEL_HOME}/lib中,如果您使用spark或flink,则需要将其放在相应服务的libs下。 + +### common options [string] + +转换插件的常见参数, 请参考 [Transform Plugin](common-options.md) 了解详情。 + +### compile_language [Enum] + +Java中的某些语法可能不受支持,请参阅https://github.com/janino-compiler/janino +GROOVY,JAVA + +### compile_pattern [Enum] + +SOURCE_CODE,ABSOLUTE_PATH +选择 SOURCE_CODE,SOURCE_CODE 属性必填;选择ABSOLUTE_PATH,ABSOLUTE_PATH属性必填。 + +### absolute_path [string] + +服务器上Java或Groovy文件的绝对路径 + +## Example + +源端数据读取的表格如下: + +| name | age | card | +|----------|-----|------| +| Joy Ding | 20 | 123 | +| May Ding | 20 | 123 | +| Kin Dom | 20 | 123 | +| Joy Dom | 20 | 123 | + +``` +transform { + DynamicCompile { + source_table_name = "fake" + result_table_name = "groovy_out" + compile_language="GROOVY" + compile_pattern="SOURCE_CODE" + source_code=""" + import org.apache.seatunnel.api.table.catalog.Column + import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor + import org.apache.seatunnel.api.table.catalog.CatalogTable + import org.apache.seatunnel.api.table.catalog.PhysicalColumn; + import org.apache.seatunnel.api.table.type.*; + import java.util.ArrayList; + class demo { + public Column[] getInlineOutputColumns(CatalogTable inputCatalogTable) { + List columns = new ArrayList<>(); + PhysicalColumn destColumn = + PhysicalColumn.of( + "compile_language", + BasicType.STRING_TYPE, + 10, + true, + "", + ""); + columns.add(destColumn); + return columns.toArray(new Column[0]); + } + public Object[] getInlineOutputFieldValues(SeaTunnelRowAccessor inputRow) { + Object[] fieldValues = new Object[1]; + fieldValues[0]="GROOVY" + return fieldValues; + } + };""" + + } +} + +transform { + DynamicCompile { + source_table_name = "fake" + result_table_name = "java_out" + compile_language="JAVA" + compile_pattern="SOURCE_CODE" + source_code=""" + import org.apache.seatunnel.api.table.catalog.Column; + import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; + import org.apache.seatunnel.api.table.catalog.*; + import org.apache.seatunnel.api.table.type.*; + import java.util.ArrayList; + public Column[] getInlineOutputColumns(CatalogTable inputCatalogTable) { + + ArrayList columns = new ArrayList(); + PhysicalColumn destColumn = + PhysicalColumn.of( + "compile_language", + BasicType.STRING_TYPE, + 10, + true, + "", + ""); + return new Column[]{ + destColumn + }; + + } + public Object[] getInlineOutputFieldValues(SeaTunnelRowAccessor inputRow) { + Object[] fieldValues = new Object[1]; + fieldValues[0]="JAVA"; + return fieldValues; + } + """ + + } + } + + transform { + DynamicCompile { + source_table_name = "fake" + result_table_name = "groovy_out" + compile_language="GROOVY" + compile_pattern="ABSOLUTE_PATH" + absolute_path="""/tmp/GroovyFile""" + + } +} +``` + +那么结果表 `groovy_out` 中的数据将会更新为: + +| name | age | card | compile_language | +|----------|-----|------|------------------| +| Joy Ding | 20 | 123 | GROOVY | +| May Ding | 20 | 123 | GROOVY | +| Kin Dom | 20 | 123 | GROOVY | +| Joy Dom | 20 | 123 | GROOVY | + +那么结果表 `java_out` 中的数据将会更新为: + +| name | age | card | compile_language | +|----------|-----|------|------------------| +| Joy Ding | 20 | 123 | JAVA | +| May Ding | 20 | 123 | JAVA | +| Kin Dom | 20 | 123 | JAVA | +| Joy Dom | 20 | 123 | JAVA | + +更多复杂例子可以参考 +https://github.com/apache/seatunnel/tree/dev/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf + +## Changelog + diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestDynamicCompileIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestDynamicCompileIT.java index b57b332353a..2528499fc1b 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestDynamicCompileIT.java +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestDynamicCompileIT.java @@ -17,18 +17,89 @@ package org.apache.seatunnel.e2e.transform; +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; +import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerImageName; +import org.testcontainers.utility.DockerLoggerFactory; +import org.testcontainers.utility.MountableFile; +import java.io.File; import java.io.IOException; +import java.net.URL; +import java.util.Optional; +import java.util.stream.Stream; -public class TestDynamicCompileIT extends TestSuiteBase { +public class TestDynamicCompileIT extends TestSuiteBase implements TestResource { private final String basePath = "/dynamic_compile/conf/"; + private static final String TMP_DIR = "/tmp"; + private GenericContainer mockserverContainer; + private static final String IMAGE = "mockserver/mockserver:5.14.0"; + + @BeforeAll + @Override + public void startUp() { + Optional resource = + Optional.ofNullable( + TestDynamicCompileIT.class.getResource( + "/dynamic_compile/conf/mockserver-config.json")); + this.mockserverContainer = + new GenericContainer<>(DockerImageName.parse(IMAGE)) + .withNetwork(NETWORK) + .withNetworkAliases("mockserver") + .withExposedPorts(1080) + .withCopyFileToContainer( + MountableFile.forHostPath( + new File( + resource.orElseThrow( + () -> + new IllegalArgumentException( + "Can not get config file of mockServer")) + .getPath()) + .getAbsolutePath()), + TMP_DIR + "/mockserver-config.json") + .withEnv( + "MOCKSERVER_INITIALIZATION_JSON_PATH", + TMP_DIR + "/mockserver-config.json") + .withEnv("MOCKSERVER_LOG_LEVEL", "WARN") + .withLogConsumer(new Slf4jLogConsumer(DockerLoggerFactory.getLogger(IMAGE))) + .waitingFor(new HttpWaitStrategy().forPath("/").forStatusCode(404)); + Startables.deepStart(Stream.of(mockserverContainer)).join(); + } + + @TestContainerExtension + protected final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Fake/lib && cd /tmp/seatunnel/plugins/Fake/lib && wget " + + "https://repo1.maven.org/maven2/cn/hutool/hutool-all/5.3.6/hutool-all-5.3.6.jar"); + Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); + }; + + @AfterAll + @Override + public void tearDown() { + if (mockserverContainer != null) { + mockserverContainer.stop(); + } + } + @TestTemplate public void testDynamicSingleCompileGroovy(TestContainer container) throws IOException, InterruptedException { @@ -86,4 +157,11 @@ public void testDynamicSinglePathJava(TestContainer container) container.executeJob(basePath + "single_java_path_compile.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } + + @TestTemplate + public void testHttpDynamic(TestContainer container) throws IOException, InterruptedException { + Container.ExecResult execResult = + container.executeJob(basePath + "single_dynamic_http_compile_transform.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/mockserver-config.json b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/mockserver-config.json new file mode 100644 index 00000000000..4890409f64b --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/mockserver-config.json @@ -0,0 +1,18 @@ + +// https://www.mock-server.com/mock_server/getting_started.html#request_matchers +[ + { + "httpRequest": { + "method": "GET", + "path": "/v1/compile" + }, + "httpResponse": { + "body": { + "compile": "seatunnel-compile" + }, + "headers": { + "Content-Type": "application/json" + } + } + } +] \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_http_compile_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_http_compile_transform.conf new file mode 100644 index 00000000000..904066d69bc --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/dynamic_compile/conf/single_dynamic_http_compile_transform.conf @@ -0,0 +1,115 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + job.mode = "BATCH" +} + +source { + FakeSource { + result_table_name = "fake" + row.num = 100 + schema = { + fields { + id = "int" + name = "string" + } + } + } +} + +transform { + DynamicCompile { + source_table_name = "fake" + result_table_name = "fake1" + compile_language="GROOVY" + compile_pattern="SOURCE_CODE" + source_code=""" + import cn.hutool.http.HttpUtil; + import org.apache.seatunnel.api.table.catalog.Column + import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor + import org.apache.seatunnel.api.table.catalog.CatalogTable + import org.apache.seatunnel.api.table.catalog.PhysicalColumn; + import org.apache.seatunnel.api.table.type.*; + class HttpDemo { + + public Column[] getInlineOutputColumns(CatalogTable inputCatalogTable) { + List columns = new ArrayList<>(); + PhysicalColumn destColumn = + PhysicalColumn.of( + "DynamicCompile", + BasicType.STRING_TYPE, + 10, + true, + "", + ""); + columns.add(destColumn); + return columns.toArray(new Column[0]); + } + public Object[] getInlineOutputFieldValues(SeaTunnelRowAccessor inputRow) { + + String body= HttpUtil.get("http://mockserver:1080/v1/compile"); + Object[] fieldValues = new Object[1]; + fieldValues[0]=body + return fieldValues; + } + };""" + + } +} + +sink { + Console { +Assert { + source_table_name = "fake1" + rules = + { + row_rules = [ + { + rule_type = MIN_ROW + rule_value = 100 + } + ], + field_rules = [ + { + field_name = id + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = DynamicCompile + field_type = string + field_value = [ + { + rule_type = NOT_NULL + + } + + ] + } + ] + } + } + } +} \ No newline at end of file From a18fca80061925ba3a2c9bd152b450dcf3d9308b Mon Sep 17 00:00:00 2001 From: Alex Ting Date: Thu, 8 Aug 2024 11:22:41 +0800 Subject: [PATCH 037/361] [Fix][Connector-tdengine] Fix sql exception and concurrentmodifyexception when connect to taos and read data --- .../tdengine/config/TDengineSourceConfig.java | 7 +- .../tdengine/source/TDengineSource.java | 56 ++++---- .../tdengine/source/TDengineSourceReader.java | 95 ++++++-------- .../source/TDengineSourceSplitEnumerator.java | 122 ++++++++++------- .../tdengine/state/TDengineSourceState.java | 20 ++- .../source/TDengineSourceReaderTest.java | 124 ++++++++++++++++++ 6 files changed, 279 insertions(+), 145 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-tdengine/src/test/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSourceReaderTest.java diff --git a/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/config/TDengineSourceConfig.java b/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/config/TDengineSourceConfig.java index 0908c733876..4eabb754cf0 100644 --- a/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/config/TDengineSourceConfig.java +++ b/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/config/TDengineSourceConfig.java @@ -30,7 +30,6 @@ import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.STABLE; import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.TIMEZONE; import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.UPPER_BOUND; -import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.URL; import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.USERNAME; @Data @@ -54,7 +53,10 @@ public class TDengineSourceConfig implements Serializable { public static TDengineSourceConfig buildSourceConfig(Config pluginConfig) { TDengineSourceConfig tdengineSourceConfig = new TDengineSourceConfig(); - tdengineSourceConfig.setUrl(pluginConfig.hasPath(URL) ? pluginConfig.getString(URL) : null); + tdengineSourceConfig.setUrl( + pluginConfig.hasPath(ConfigNames.URL) + ? pluginConfig.getString(ConfigNames.URL) + : null); tdengineSourceConfig.setDatabase( pluginConfig.hasPath(DATABASE) ? pluginConfig.getString(DATABASE) : null); tdengineSourceConfig.setStable( @@ -69,6 +71,7 @@ public static TDengineSourceConfig buildSourceConfig(Config pluginConfig) { pluginConfig.hasPath(LOWER_BOUND) ? pluginConfig.getString(LOWER_BOUND) : null); tdengineSourceConfig.setTimezone( pluginConfig.hasPath(TIMEZONE) ? pluginConfig.getString(TIMEZONE) : "UTC"); + return tdengineSourceConfig; } diff --git a/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSource.java b/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSource.java index 2f2e6a3f98f..e72773781ab 100644 --- a/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSource.java +++ b/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSource.java @@ -40,6 +40,7 @@ import org.apache.commons.lang3.ArrayUtils; import com.google.auto.service.AutoService; +import com.taosdata.jdbc.TSDBDriver; import lombok.SneakyThrows; import java.sql.Connection; @@ -49,6 +50,7 @@ import java.sql.Statement; import java.util.ArrayList; import java.util.List; +import java.util.Properties; import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.DATABASE; import static org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig.ConfigNames.PASSWORD; @@ -127,42 +129,36 @@ private StableMetadata getStableMetadata(TDengineSourceConfig config) throws SQL List fieldNames = new ArrayList<>(); List> fieldTypes = new ArrayList<>(); - String jdbcUrl = - String.join( - "", - config.getUrl(), - config.getDatabase(), - "?user=", - config.getUsername(), - "&password=", - config.getPassword()); + String jdbcUrl = String.join("", config.getUrl(), config.getDatabase()); + // check td driver whether exist and if not, try to register checkDriverExist(jdbcUrl); - try (Connection conn = DriverManager.getConnection(jdbcUrl)) { - try (Statement statement = conn.createStatement()) { + + Properties properties = new Properties(); + properties.put(TSDBDriver.PROPERTY_KEY_USER, config.getUsername()); + properties.put(TSDBDriver.PROPERTY_KEY_PASSWORD, config.getPassword()); + String metaSQL = + String.format( + "select table_name from information_schema.ins_tables where db_name = '%s' and stable_name='%s'", + config.getDatabase(), config.getStable()); + try (Connection conn = DriverManager.getConnection(jdbcUrl, properties); + Statement statement = conn.createStatement(); ResultSet metaResultSet = statement.executeQuery( - "desc " + config.getDatabase() + "." + config.getStable()); - while (metaResultSet.next()) { - if (timestampFieldName == null) { - timestampFieldName = metaResultSet.getString(1); - } - fieldNames.add(metaResultSet.getString(1)); - fieldTypes.add(TDengineTypeMapper.mapping(metaResultSet.getString(2))); + String.format( + "desc %s.%s", config.getDatabase(), config.getStable())); + ResultSet subTableNameResultSet = statement.executeQuery(metaSQL)) { + while (metaResultSet.next()) { + if (timestampFieldName == null) { + timestampFieldName = metaResultSet.getString(1); } + fieldNames.add(metaResultSet.getString(1)); + fieldTypes.add(TDengineTypeMapper.mapping(metaResultSet.getString(2))); } - try (Statement statement = conn.createStatement()) { - String metaSQL = - "select table_name from information_schema.ins_tables where db_name = '" - + config.getDatabase() - + "' and stable_name='" - + config.getStable() - + "';"; - ResultSet subTableNameResultSet = statement.executeQuery(metaSQL); - while (subTableNameResultSet.next()) { - String subTableName = subTableNameResultSet.getString(1); - subTableNames.add(subTableName); - } + + while (subTableNameResultSet.next()) { + String subTableName = subTableNameResultSet.getString(1); + subTableNames.add(subTableName); } } diff --git a/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSourceReader.java b/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSourceReader.java index 6782f085bd3..bb4184702d6 100644 --- a/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSourceReader.java +++ b/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSourceReader.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.connectors.seatunnel.tdengine.source; -import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -25,9 +24,6 @@ import org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig; import org.apache.seatunnel.connectors.seatunnel.tdengine.exception.TDengineConnectorException; -import org.apache.commons.lang3.StringUtils; - -import com.google.common.collect.Sets; import com.taosdata.jdbc.TSDBDriver; import lombok.extern.slf4j.Slf4j; @@ -39,84 +35,76 @@ import java.sql.Statement; import java.sql.Timestamp; import java.util.ArrayList; +import java.util.Deque; import java.util.List; import java.util.Objects; import java.util.Properties; -import java.util.Set; +import java.util.concurrent.ConcurrentLinkedDeque; import static org.apache.seatunnel.connectors.seatunnel.tdengine.utils.TDengineUtil.checkDriverExist; @Slf4j public class TDengineSourceReader implements SourceReader { - - private static final long THREAD_WAIT_TIME = 500L; - private final TDengineSourceConfig config; - private final Set sourceSplits; + private final Deque sourceSplits; private final Context context; private Connection conn; + private volatile boolean noMoreSplit; + public TDengineSourceReader(TDengineSourceConfig config, SourceReader.Context readerContext) { this.config = config; - this.sourceSplits = Sets.newHashSet(); + this.sourceSplits = new ConcurrentLinkedDeque<>(); this.context = readerContext; } @Override public void pollNext(Collector collector) throws InterruptedException { - if (sourceSplits.isEmpty()) { - Thread.sleep(THREAD_WAIT_TIME); - return; - } synchronized (collector.getCheckpointLock()) { - sourceSplits.forEach( - split -> { - try { - read(split, collector); - } catch (Exception e) { - throw new TDengineConnectorException( - CommonErrorCodeDeprecated.READER_OPERATION_FAILED, - "TDengine split read error", - e); - } - }); - } - - if (Boundedness.BOUNDED.equals(context.getBoundedness())) { - // signal to the source that we have reached the end of the data. - log.info("Closed the bounded TDengine source"); - context.signalNoMoreElement(); + log.info("polling new split from queue!"); + TDengineSourceSplit split = sourceSplits.poll(); + if (Objects.nonNull(split)) { + log.info( + "starting run new split {}, query sql: {}!", + split.splitId(), + split.getQuery()); + try { + read(split, collector); + } catch (Exception e) { + throw new TDengineConnectorException( + CommonErrorCodeDeprecated.READER_OPERATION_FAILED, + "TDengine split read error", + e); + } + } else if (noMoreSplit && sourceSplits.isEmpty()) { + // signal to the source that we have reached the end of the data. + log.info("Closed the bounded TDengine source"); + context.signalNoMoreElement(); + } else { + Thread.sleep(1000L); + } } } @Override public void open() { - String jdbcUrl = - StringUtils.join( - config.getUrl(), - config.getDatabase(), - "?user=", - config.getUsername(), - "&password=", - config.getPassword()); - Properties connProps = new Properties(); - // todo: when TSDBDriver.PROPERTY_KEY_BATCH_LOAD set to "true", - // there is a exception : Caused by: java.sql.SQLException: can't create connection with - // server - // under docker network env - // @bobo (tdengine) - connProps.setProperty(TSDBDriver.PROPERTY_KEY_BATCH_LOAD, "false"); + String jdbcUrl = config.getUrl(); + + Properties properties = new Properties(); + properties.put(TSDBDriver.PROPERTY_KEY_USER, config.getUsername()); + properties.put(TSDBDriver.PROPERTY_KEY_PASSWORD, config.getPassword()); + try { - // check td driver whether exist and if not, try to register checkDriverExist(jdbcUrl); - conn = DriverManager.getConnection(jdbcUrl, connProps); + conn = DriverManager.getConnection(jdbcUrl, properties); } catch (SQLException e) { throw new TDengineConnectorException( CommonErrorCodeDeprecated.READER_OPERATION_FAILED, - "get TDengine connection failed:" + jdbcUrl); + "get TDengine connection failed:" + jdbcUrl, + e); } } @@ -135,8 +123,8 @@ public void close() { } private void read(TDengineSourceSplit split, Collector output) throws Exception { - try (Statement statement = conn.createStatement()) { - final ResultSet resultSet = statement.executeQuery(split.getQuery()); + try (Statement statement = conn.createStatement(); + ResultSet resultSet = statement.executeQuery(split.getQuery())) { ResultSetMetaData meta = resultSet.getMetaData(); while (resultSet.next()) { @@ -151,6 +139,8 @@ private void read(TDengineSourceSplit split, Collector output) thr } private Object convertDataType(Object object) { + if (Objects.isNull(object)) return null; + if (Timestamp.class.equals(object.getClass())) { return ((Timestamp) object).toLocalDateTime(); } else if (byte[].class.equals(object.getClass())) { @@ -171,7 +161,8 @@ public void addSplits(List splits) { @Override public void handleNoMoreSplits() { - // do nothing + log.info("no more split accepted!"); + noMoreSplit = true; } @Override diff --git a/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSourceSplitEnumerator.java index d5787ba5573..911a9a6ec10 100644 --- a/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSourceSplitEnumerator.java @@ -17,28 +17,34 @@ package org.apache.seatunnel.connectors.seatunnel.tdengine.source; -import org.apache.seatunnel.api.source.SourceEvent; import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; import org.apache.seatunnel.connectors.seatunnel.tdengine.config.TDengineSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.tdengine.exception.TDengineConnectorException; import org.apache.seatunnel.connectors.seatunnel.tdengine.state.TDengineSourceState; +import lombok.extern.slf4j.Slf4j; + +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; import java.util.List; -import java.util.Objects; +import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; +@Slf4j public class TDengineSourceSplitEnumerator implements SourceSplitEnumerator { private final SourceSplitEnumerator.Context context; private final TDengineSourceConfig config; private final StableMetadata stableMetadata; - private Set pendingSplit = new HashSet<>(); - private Set assignedSplit = new HashSet<>(); + private volatile boolean shouldEnumerate; + private final Object stateLock = new Object(); + private final Map> pendingSplits = new ConcurrentHashMap<>(); public TDengineSourceSplitEnumerator( StableMetadata stableMetadata, @@ -55,8 +61,10 @@ public TDengineSourceSplitEnumerator( this.config = config; this.context = context; this.stableMetadata = stableMetadata; + this.shouldEnumerate = sourceState == null; if (sourceState != null) { - this.assignedSplit = sourceState.getAssignedSplit(); + this.shouldEnumerate = sourceState.isShouldEnumerate(); + this.pendingSplits.putAll(sourceState.getPendingSplits()); } } @@ -69,16 +77,33 @@ public void open() {} @Override public void run() { - pendingSplit = getAllSplits(); - assignSplit(context.registeredReaders()); + Set readers = context.registeredReaders(); + if (shouldEnumerate) { + List newSplits = discoverySplits(); + + synchronized (stateLock) { + addPendingSplit(newSplits); + shouldEnumerate = false; + } + + assignSplit(readers); + } + + log.info("No more splits to assign." + " Sending NoMoreSplitsEvent to reader {}.", readers); + readers.forEach(context::signalNoMoreSplits); } - /* - * each split has one sub table - */ - private Set getAllSplits() { + private void addPendingSplit(List newSplits) { + int readerCount = context.currentParallelism(); + for (TDengineSourceSplit split : newSplits) { + int ownerReader = getSplitOwner(split.splitId(), readerCount); + pendingSplits.computeIfAbsent(ownerReader, r -> new ArrayList<>()).add(split); + } + } + + private List discoverySplits() { final String timestampFieldName = stableMetadata.getTimestampFieldName(); - final Set splits = new HashSet<>(); + final List splits = new ArrayList<>(); for (String subTableName : stableMetadata.getSubTableNames()) { TDengineSourceSplit splitBySubTable = createSplitBySubTable(subTableName, timestampFieldName); @@ -92,9 +117,11 @@ private TDengineSourceSplit createSplitBySubTable( String selectFields = Arrays.stream(stableMetadata.getRowType().getFieldNames()) .skip(1) + .map(name -> String.format("`%s`", name)) .collect(Collectors.joining(",")); String subTableSQL = - "select " + selectFields + " from " + config.getDatabase() + "." + subTableName; + String.format( + "select %s from %s.`%s`", selectFields, config.getDatabase(), subTableName); String start = config.getLowerBound(); String end = config.getUpperBound(); if (start != null || end != null) { @@ -116,69 +143,64 @@ private TDengineSourceSplit createSplitBySubTable( @Override public void addSplitsBack(List splits, int subtaskId) { + log.info("Add back splits {} to TDengineSourceSplitEnumerator.", splits); if (!splits.isEmpty()) { - pendingSplit.addAll(splits); + addPendingSplit(splits); assignSplit(Collections.singletonList(subtaskId)); } } @Override public int currentUnassignedSplitSize() { - return pendingSplit.size(); + return pendingSplits.size(); } @Override public void registerReader(int subtaskId) { - if (!pendingSplit.isEmpty()) { + log.info("Register reader {} to TDengineSourceSplitEnumerator.", subtaskId); + if (!pendingSplits.isEmpty()) { assignSplit(Collections.singletonList(subtaskId)); } } - private void assignSplit(Collection taskIDList) { - assignedSplit = - pendingSplit.stream() - .map( - split -> { - int splitOwner = - getSplitOwner( - split.splitId(), context.currentParallelism()); - if (taskIDList.contains(splitOwner)) { - context.assignSplit(splitOwner, split); - return split; - } else { - return null; - } - }) - .filter(Objects::nonNull) - .collect(Collectors.toSet()); - pendingSplit.clear(); + private void assignSplit(Collection readers) { + log.info("Assign pendingSplits to readers {}", readers); + + for (int reader : readers) { + List assignmentForReader = pendingSplits.remove(reader); + if (assignmentForReader != null && !assignmentForReader.isEmpty()) { + log.info("Assign splits {} to reader {}", assignmentForReader, reader); + try { + context.assignSplit(reader, assignmentForReader); + } catch (Exception e) { + log.error( + "Failed to assign splits {} to reader {}", + assignmentForReader, + reader, + e); + pendingSplits.put(reader, assignmentForReader); + } + } + } } @Override public TDengineSourceState snapshotState(long checkpointId) { - return new TDengineSourceState(assignedSplit); - } - - @Override - public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { - SourceSplitEnumerator.super.handleSourceEvent(subtaskId, sourceEvent); - } - - @Override - public void notifyCheckpointComplete(long checkpointId) { - // nothing to do + synchronized (stateLock) { + return new TDengineSourceState(shouldEnumerate, pendingSplits); + } } @Override - public void notifyCheckpointAborted(long checkpointId) throws Exception { - SourceSplitEnumerator.super.notifyCheckpointAborted(checkpointId); - } + public void notifyCheckpointComplete(long checkpointId) {} @Override public void close() {} @Override public void handleSplitRequest(int subtaskId) { - // nothing to do + throw new TDengineConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + String.format("Unsupported handleSplitRequest: %d", subtaskId)); } } diff --git a/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/state/TDengineSourceState.java b/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/state/TDengineSourceState.java index fc839682a92..4832cd398ff 100644 --- a/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/state/TDengineSourceState.java +++ b/seatunnel-connectors-v2/connector-tdengine/src/main/java/org/apache/seatunnel/connectors/seatunnel/tdengine/state/TDengineSourceState.java @@ -19,18 +19,16 @@ import org.apache.seatunnel.connectors.seatunnel.tdengine.source.TDengineSourceSplit; +import lombok.AllArgsConstructor; +import lombok.Getter; + import java.io.Serializable; -import java.util.Set; +import java.util.List; +import java.util.Map; +@AllArgsConstructor +@Getter public class TDengineSourceState implements Serializable { - - private final Set assignedSplit; - - public TDengineSourceState(Set assignedSplit) { - this.assignedSplit = assignedSplit; - } - - public Set getAssignedSplit() { - return assignedSplit; - } + private boolean shouldEnumerate; + private final Map> pendingSplits; } diff --git a/seatunnel-connectors-v2/connector-tdengine/src/test/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSourceReaderTest.java b/seatunnel-connectors-v2/connector-tdengine/src/test/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSourceReaderTest.java new file mode 100644 index 00000000000..abd42fefe17 --- /dev/null +++ b/seatunnel-connectors-v2/connector-tdengine/src/test/java/org/apache/seatunnel/connectors/seatunnel/tdengine/source/TDengineSourceReaderTest.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.tdengine.source; + +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.tdengine.exception.TDengineConnectorException; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Random; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.logging.Logger; + +class TDengineSourceReaderTest { + Logger logger; + TDengineSourceReader tDengineSourceReader; + + @BeforeEach + void setup() { + tDengineSourceReader = new TDengineSourceReader(null, null); + + List sourceSplits = new ArrayList<>(); + int splitCnt = 100; + for (int i = 0; i < splitCnt; i++) { + sourceSplits.add(new TDengineSourceSplit(Integer.toString(i), "select sever_status()")); + } + + tDengineSourceReader.addSplits(sourceSplits); + + logger = Logger.getLogger("TDengineSourceReaderTest"); + } + + @Test + void testPoll() throws InterruptedException { + TestCollector testCollector = new TestCollector(); + + int totalSplitCnt = 150; + ThreadPoolExecutor pool = + new ThreadPoolExecutor(8, 8, 60, TimeUnit.SECONDS, new LinkedBlockingQueue<>()); + pool.execute( + () -> { + for (int i = 0; i < totalSplitCnt; i++) { + try { + tDengineSourceReader.pollNext(testCollector); + Thread.sleep(new Random().nextInt(5)); + } catch (TDengineConnectorException e) { + logger.info("skip create connection!"); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + }); + + int newSplitCnt = 50; + int threadCnt = 3; + for (int i = 0; i < threadCnt; i++) { + pool.execute( + () -> { + for (int idx = 0; idx < newSplitCnt; idx++) { + logger.info( + String.format( + "%s receive new split", + Thread.currentThread().getName())); + tDengineSourceReader.addSplits( + Collections.singletonList( + new TDengineSourceSplit( + String.format( + "new_%s", + Thread.currentThread().getName() + idx), + "select server_status()"))); + try { + Thread.sleep(new Random().nextInt(5)); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + }); + } + + pool.awaitTermination(3, TimeUnit.SECONDS); + } + + private static class TestCollector implements Collector { + + private final List rows = new ArrayList<>(); + + public List getRows() { + return rows; + } + + @Override + public void collect(SeaTunnelRow record) { + rows.add(record); + } + + @Override + public Object getCheckpointLock() { + return new Object(); + } + } +} From 16950a67cd5990ed3510fd91564390e2ab2cd44c Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Thu, 8 Aug 2024 17:27:47 +0800 Subject: [PATCH 038/361] [Improve][Connector-V2] Improve doris error msg (#7343) --- .../connectors/doris/sink/writer/DorisStreamLoad.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisStreamLoad.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisStreamLoad.java index 40b75aedc61..8ec59e81ece 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisStreamLoad.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisStreamLoad.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.common.utils.ExceptionUtils; import org.apache.seatunnel.common.utils.JsonUtils; import org.apache.seatunnel.connectors.doris.config.DorisConfig; import org.apache.seatunnel.connectors.doris.exception.DorisConnectorErrorCode; @@ -196,7 +197,7 @@ public String getLoadFailedMsg() { try { errorMessage = handlePreCommitResponse(pendingLoadFuture.get()).getMessage(); } catch (Exception e) { - errorMessage = e.getMessage(); + errorMessage = ExceptionUtils.getMessage(e); } recordStream.setErrorMessageByStreamLoad(errorMessage); return errorMessage; From dc3c23981b3a78a3f1af69bcde57fbd5bf57d7b4 Mon Sep 17 00:00:00 2001 From: dailai Date: Fri, 9 Aug 2024 13:11:30 +0800 Subject: [PATCH 039/361] [Improve][Jdbc] Skip all index when auto create table to improve performance of write (#7288) --- docs/en/connector-v2/sink/Jdbc.md | 7 + .../api/sink/DefaultSaveModeHandler.java | 6 +- .../seatunnel/api/table/catalog/Catalog.java | 19 ++ .../jdbc/catalog/AbstractJdbcCatalog.java | 24 ++- .../jdbc/catalog/JdbcCatalogOptions.java | 6 + .../jdbc/catalog/dm/DamengCatalog.java | 3 +- .../jdbc/catalog/iris/IrisCatalog.java | 10 +- .../iris/IrisCreateTableSqlBuilder.java | 9 +- .../iris/savemode/IrisSaveModeHandler.java | 8 +- .../jdbc/catalog/mysql/MySqlCatalog.java | 5 +- .../mysql/MysqlCreateTableSqlBuilder.java | 16 +- .../oceanbase/OceanBaseOracleCatalog.java | 5 +- .../jdbc/catalog/oracle/OracleCatalog.java | 10 +- .../oracle/OracleCreateTableSqlBuilder.java | 7 +- .../jdbc/catalog/psql/PostgresCatalog.java | 9 +- .../psql/PostgresCreateTableSqlBuilder.java | 10 +- .../catalog/redshift/RedshiftCatalog.java | 5 +- .../RedshiftCreateTableSqlBuilder.java | 9 +- .../jdbc/catalog/saphana/SapHanaCatalog.java | 5 +- .../saphana/SapHanaCreateTableSqlBuilder.java | 9 +- .../catalog/sqlserver/SqlServerCatalog.java | 6 +- .../SqlServerCreateTableSqlBuilder.java | 12 +- .../jdbc/catalog/xugu/XuguCatalog.java | 5 +- .../xugu/XuguCreateTableSqlBuilder.java | 7 +- .../seatunnel/jdbc/config/JdbcSinkConfig.java | 3 + .../seatunnel/jdbc/sink/JdbcSink.java | 10 +- .../seatunnel/jdbc/sink/JdbcSinkFactory.java | 2 + .../sink/savemode/JdbcSaveModeHandler.java | 50 ++++++ .../IrisCreateTableSqlBuilderTest.java | 21 ++- .../MysqlCreateTableSqlBuilderTest.java | 22 ++- .../catalog/oracle/OracleCatalogTest.java | 6 +- .../OracleCreateTableSqlBuilderTest.java | 146 ++++++++++++++++ .../PostgresCreateTableSqlBuilderTest.java | 19 +- .../catalog/redshift/RedshiftCatalogTest.java | 2 +- .../RedshiftCreateTableSqlBuilderTest.java | 152 ++++++++++++++++ .../SapHanaCreateTableSqlBuilderTest.java | 16 +- .../SqlServerCreateTableSqlBuilderTest.java | 164 ++++++++++++++++++ .../xugu/XuguCreateTableSqlBuilderTest.java | 157 +++++++++++++++++ .../seatunnel/jdbc/AbstractJdbcIT.java | 73 +++++++- .../seatunnel/jdbc/JdbcMysqlIT.java | 3 +- .../seatunnel/jdbc/JdbcOracleIT.java | 21 +++ .../seatunnel/jdbc/JdbcOceanBaseMysqlIT.java | 3 +- .../seatunnel/jdbc/JdbcPostgresIT.java | 65 +++++++ .../seatunnel/jdbc/JdbcSqlServerIT.java | 4 +- .../jdbc/JdbcOracleLowercaseTableIT.java | 19 ++ .../jdbc/JdbcMysqlSaveModeHandlerIT.java | 3 +- 46 files changed, 1089 insertions(+), 84 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/savemode/JdbcSaveModeHandler.java rename seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/{sql => iris}/IrisCreateTableSqlBuilderTest.java (84%) rename seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/{sql => mysql}/MysqlCreateTableSqlBuilderTest.java (85%) create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilderTest.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCreateTableSqlBuilderTest.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCreateTableSqlBuilderTest.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCreateTableSqlBuilderTest.java diff --git a/docs/en/connector-v2/sink/Jdbc.md b/docs/en/connector-v2/sink/Jdbc.md index aa13c86c58f..c46933b486b 100644 --- a/docs/en/connector-v2/sink/Jdbc.md +++ b/docs/en/connector-v2/sink/Jdbc.md @@ -58,6 +58,7 @@ support `Xa transactions`. You can set `is_exactly_once=true` to enable it. | custom_sql | String | No | - | | enable_upsert | Boolean | No | true | | use_copy_statement | Boolean | No | false | +| create_index | Boolean | No | true | ### driver [string] @@ -205,6 +206,12 @@ Use `COPY ${table} FROM STDIN` statement to import data. Only drivers with `getC NOTICE: `MAP`, `ARRAY`, `ROW` types are not supported. +### create_index [boolean] + +Create the index(contains primary key and any other indexes) or not when auto-create table. You can use this option to improve the performance of jdbc writes when migrating large tables. + +Notice: Note that this will sacrifice read performance, so you'll need to manually create indexes after the table migration to improve read performance + ## tips In the case of is_exactly_once = "true", Xa transactions are used. This requires database support, and some databases require some setup : diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DefaultSaveModeHandler.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DefaultSaveModeHandler.java index e22dd7c99a5..051068dba03 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DefaultSaveModeHandler.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DefaultSaveModeHandler.java @@ -151,7 +151,7 @@ protected void dropTable() { catalog.dropTable(tablePath, true); } - protected void createTable() { + protected void createTablePreCheck() { if (!catalog.databaseExists(tablePath.getDatabaseName())) { try { log.info( @@ -175,6 +175,10 @@ protected void createTable() { } catch (UnsupportedOperationException ignore) { log.info("Creating table {}", tablePath); } + } + + protected void createTable() { + createTablePreCheck(); catalog.createTable(tablePath, catalogTable, true); } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/Catalog.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/Catalog.java index 05b7ab114c4..f75c012f8fe 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/Catalog.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/Catalog.java @@ -239,6 +239,25 @@ default void buildColumnsWithErrorCheck( void createTable(TablePath tablePath, CatalogTable table, boolean ignoreIfExists) throws TableAlreadyExistException, DatabaseNotExistException, CatalogException; + /** + * Create a new table in this catalog. + * + * @param tablePath Path of the table + * @param table The table definition + * @param ignoreIfExists Flag to specify behavior when a table with the given name already exist + * @param createIndex If you want to create index or not + * @throws TableAlreadyExistException thrown if the table already exists in the catalog and + * ignoreIfExists is false + * @throws DatabaseNotExistException thrown if the database in tablePath doesn't exist in the + * catalog + * @throws CatalogException in case of any runtime exception + */ + default void createTable( + TablePath tablePath, CatalogTable table, boolean ignoreIfExists, boolean createIndex) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + createTable(tablePath, table, ignoreIfExists); + } + /** * Drop an existing table in this catalog. * diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java index 8d0301b492e..210bb779e0e 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java @@ -373,6 +373,13 @@ && listTables(tablePath.getDatabaseName()) @Override public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreIfExists) throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + createTable(tablePath, table, ignoreIfExists, true); + } + + @Override + public void createTable( + TablePath tablePath, CatalogTable table, boolean ignoreIfExists, boolean createIndex) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { checkNotNull(tablePath, "Table path cannot be null"); if (!databaseExists(tablePath.getDatabaseName())) { @@ -393,22 +400,25 @@ public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreI throw new TableAlreadyExistException(catalogName, tablePath); } - createTableInternal(tablePath, table); + createTableInternal(tablePath, table, createIndex); } - protected String getCreateTableSql(TablePath tablePath, CatalogTable table) { + protected String getCreateTableSql( + TablePath tablePath, CatalogTable table, boolean createIndex) { throw new UnsupportedOperationException(); } - protected List getCreateTableSqls(TablePath tablePath, CatalogTable table) { - return Collections.singletonList(getCreateTableSql(tablePath, table)); + protected List getCreateTableSqls( + TablePath tablePath, CatalogTable table, boolean createIndex) { + return Collections.singletonList(getCreateTableSql(tablePath, table, createIndex)); } - protected void createTableInternal(TablePath tablePath, CatalogTable table) + protected void createTableInternal(TablePath tablePath, CatalogTable table, boolean createIndex) throws CatalogException { String dbUrl = getUrlFromDatabaseName(tablePath.getDatabaseName()); try { - final List createTableSqlList = getCreateTableSqls(tablePath, table); + final List createTableSqlList = + getCreateTableSqls(tablePath, table, createIndex); for (String sql : createTableSqlList) { executeInternal(dbUrl, sql); } @@ -646,7 +656,7 @@ public PreviewResult previewAction( ActionType actionType, TablePath tablePath, Optional catalogTable) { if (actionType == ActionType.CREATE_TABLE) { checkArgument(catalogTable.isPresent(), "CatalogTable cannot be null"); - return new SQLPreviewResult(getCreateTableSql(tablePath, catalogTable.get())); + return new SQLPreviewResult(getCreateTableSql(tablePath, catalogTable.get(), true)); } else if (actionType == ActionType.DROP_TABLE) { return new SQLPreviewResult(getDropTableSql(tablePath)); } else if (actionType == ActionType.TRUNCATE_TABLE) { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java index fc58a45c28b..c2f2405ee00 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java @@ -74,4 +74,10 @@ public interface JdbcCatalogOptions { .noDefaultValue() .withDescription( "The table suffix name added when the table is automatically created"); + + Option CREATE_INDEX = + Options.key("create_index") + .booleanType() + .defaultValue(true) + .withDescription("Create index or not when auto create table"); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/dm/DamengCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/dm/DamengCatalog.java index ede65bc8a0a..0525a47584c 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/dm/DamengCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/dm/DamengCatalog.java @@ -88,7 +88,8 @@ protected String getListDatabaseSql() { } @Override - protected String getCreateTableSql(TablePath tablePath, CatalogTable table) { + protected String getCreateTableSql( + TablePath tablePath, CatalogTable table, boolean createIndex) { throw new UnsupportedOperationException(); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCatalog.java index 02e58ea8573..5898d4e855f 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCatalog.java @@ -66,8 +66,9 @@ public IrisCatalog( } @Override - protected String getCreateTableSql(TablePath tablePath, CatalogTable table) { - return new IrisCreateTableSqlBuilder(table).build(tablePath); + protected String getCreateTableSql( + TablePath tablePath, CatalogTable table, boolean createIndex) { + return new IrisCreateTableSqlBuilder(table, createIndex).build(tablePath); } @Override @@ -224,7 +225,8 @@ public void createDatabase(TablePath tablePath, boolean ignoreIfExists) } @Override - public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreIfExists) + public void createTable( + TablePath tablePath, CatalogTable table, boolean ignoreIfExists, boolean createIndex) throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { checkNotNull(tablePath, "Table path cannot be null"); if (defaultSchema.isPresent()) { @@ -242,7 +244,7 @@ public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreI throw new TableAlreadyExistException(catalogName, tablePath); } - createTableInternal(tablePath, table); + createTableInternal(tablePath, table, createIndex); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCreateTableSqlBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCreateTableSqlBuilder.java index b4a6b8f08d6..819b3bb2160 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCreateTableSqlBuilder.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCreateTableSqlBuilder.java @@ -40,14 +40,16 @@ public class IrisCreateTableSqlBuilder { private String fieldIde; private String comment; + private boolean createIndex; - public IrisCreateTableSqlBuilder(CatalogTable catalogTable) { + public IrisCreateTableSqlBuilder(CatalogTable catalogTable, boolean createIndex) { this.columns = catalogTable.getTableSchema().getColumns(); this.primaryKey = catalogTable.getTableSchema().getPrimaryKey(); this.constraintKeys = catalogTable.getTableSchema().getConstraintKeys(); this.sourceCatalogName = catalogTable.getCatalogName(); this.fieldIde = catalogTable.getOptions().get("fieldIde"); this.comment = catalogTable.getComment(); + this.createIndex = createIndex; } public String build(TablePath tablePath) { @@ -64,12 +66,13 @@ public String build(TablePath tablePath) { .collect(Collectors.toList()); // Add primary key directly in the create table statement - if (primaryKey != null + if (createIndex + && primaryKey != null && primaryKey.getColumnNames() != null && primaryKey.getColumnNames().size() > 0) { columnSqls.add(buildPrimaryKeySql(primaryKey)); } - if (CollectionUtils.isNotEmpty(constraintKeys)) { + if (createIndex && CollectionUtils.isNotEmpty(constraintKeys)) { for (ConstraintKey constraintKey : constraintKeys) { if (StringUtils.isBlank(constraintKey.getConstraintName()) || (primaryKey != null diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/savemode/IrisSaveModeHandler.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/savemode/IrisSaveModeHandler.java index b2a7c9851e6..0ddf2968fff 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/savemode/IrisSaveModeHandler.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/savemode/IrisSaveModeHandler.java @@ -33,14 +33,18 @@ @Slf4j public class IrisSaveModeHandler extends DefaultSaveModeHandler { + public boolean createIndex; + public IrisSaveModeHandler( @Nonnull SchemaSaveMode schemaSaveMode, @Nonnull DataSaveMode dataSaveMode, @Nonnull Catalog catalog, @Nonnull TablePath tablePath, @Nullable CatalogTable catalogTable, - @Nullable String customSql) { + @Nullable String customSql, + boolean createIndex) { super(schemaSaveMode, dataSaveMode, catalog, tablePath, catalogTable, customSql); + this.createIndex = createIndex; } @Override @@ -53,7 +57,7 @@ protected void createTable() { Catalog.ActionType.CREATE_TABLE, tablePath, Optional.ofNullable(catalogTable))); - catalog.createTable(tablePath, catalogTable, true); + catalog.createTable(tablePath, catalogTable, true, createIndex); } catch (UnsupportedOperationException ignore) { log.info("Creating table {}", tablePath); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java index e2df8ab24b9..65922383ee2 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java @@ -181,8 +181,9 @@ protected Column buildColumn(ResultSet resultSet) throws SQLException { } @Override - protected String getCreateTableSql(TablePath tablePath, CatalogTable table) { - return MysqlCreateTableSqlBuilder.builder(tablePath, table, typeConverter) + protected String getCreateTableSql( + TablePath tablePath, CatalogTable table, boolean createIndex) { + return MysqlCreateTableSqlBuilder.builder(tablePath, table, typeConverter, createIndex) .build(table.getCatalogName()); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MysqlCreateTableSqlBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MysqlCreateTableSqlBuilder.java index 91879701204..ec9d4fc433f 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MysqlCreateTableSqlBuilder.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MysqlCreateTableSqlBuilder.java @@ -61,22 +61,28 @@ public class MysqlCreateTableSqlBuilder { private String fieldIde; private final MySqlTypeConverter typeConverter; + private boolean createIndex; - private MysqlCreateTableSqlBuilder(String tableName, MySqlTypeConverter typeConverter) { + private MysqlCreateTableSqlBuilder( + String tableName, MySqlTypeConverter typeConverter, boolean createIndex) { checkNotNull(tableName, "tableName must not be null"); this.tableName = tableName; this.typeConverter = typeConverter; + this.createIndex = createIndex; } public static MysqlCreateTableSqlBuilder builder( - TablePath tablePath, CatalogTable catalogTable, MySqlTypeConverter typeConverter) { + TablePath tablePath, + CatalogTable catalogTable, + MySqlTypeConverter typeConverter, + boolean createIndex) { checkNotNull(tablePath, "tablePath must not be null"); checkNotNull(catalogTable, "catalogTable must not be null"); TableSchema tableSchema = catalogTable.getTableSchema(); checkNotNull(tableSchema, "tableSchema must not be null"); - return new MysqlCreateTableSqlBuilder(tablePath.getTableName(), typeConverter) + return new MysqlCreateTableSqlBuilder(tablePath.getTableName(), typeConverter, createIndex) .comment(catalogTable.getComment()) // todo: set charset and collate .engine(null) @@ -156,10 +162,10 @@ private String buildColumnsIdentifySql(String catalogName) { for (Column column : columns) { columnSqls.add("\t" + buildColumnIdentifySql(column, catalogName, columnTypeMap)); } - if (primaryKey != null) { + if (createIndex && primaryKey != null) { columnSqls.add("\t" + buildPrimaryKeySql()); } - if (CollectionUtils.isNotEmpty(constraintKeys)) { + if (createIndex && CollectionUtils.isNotEmpty(constraintKeys)) { for (ConstraintKey constraintKey : constraintKeys) { if (StringUtils.isBlank(constraintKey.getConstraintName())) { continue; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseOracleCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseOracleCatalog.java index b98f4c4c2b2..7e8f844699b 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseOracleCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseOracleCatalog.java @@ -80,7 +80,8 @@ public List listTables(String databaseName) } @Override - public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreIfExists) + public void createTable( + TablePath tablePath, CatalogTable table, boolean ignoreIfExists, boolean createIndex) throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { checkNotNull(tablePath, "Table path cannot be null"); @@ -99,6 +100,6 @@ public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreI throw new TableAlreadyExistException(catalogName, tablePath); } - createTableInternal(tablePath, table); + createTableInternal(tablePath, table, createIndex); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java index 1430cb387af..338a894c08d 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java @@ -131,12 +131,14 @@ protected String getListDatabaseSql() { } @Override - protected String getCreateTableSql(TablePath tablePath, CatalogTable table) { - return new OracleCreateTableSqlBuilder(table).build(tablePath).get(0); + protected String getCreateTableSql( + TablePath tablePath, CatalogTable table, boolean createIndex) { + return new OracleCreateTableSqlBuilder(table, createIndex).build(tablePath).get(0); } - protected List getCreateTableSqls(TablePath tablePath, CatalogTable table) { - return new OracleCreateTableSqlBuilder(table).build(tablePath); + protected List getCreateTableSqls( + TablePath tablePath, CatalogTable table, boolean createIndex) { + return new OracleCreateTableSqlBuilder(table, createIndex).build(tablePath); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilder.java index 6afbfcfce29..369980b57da 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilder.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilder.java @@ -38,12 +38,14 @@ public class OracleCreateTableSqlBuilder { private PrimaryKey primaryKey; private String sourceCatalogName; private String fieldIde; + private boolean createIndex; - public OracleCreateTableSqlBuilder(CatalogTable catalogTable) { + public OracleCreateTableSqlBuilder(CatalogTable catalogTable, boolean createIndex) { this.columns = catalogTable.getTableSchema().getColumns(); this.primaryKey = catalogTable.getTableSchema().getPrimaryKey(); this.sourceCatalogName = catalogTable.getCatalogName(); this.fieldIde = catalogTable.getOptions().get("fieldIde"); + this.createIndex = createIndex; } public List build(TablePath tablePath) { @@ -60,7 +62,8 @@ public List build(TablePath tablePath) { .collect(Collectors.toList()); // Add primary key directly in the create table statement - if (primaryKey != null + if (createIndex + && primaryKey != null && primaryKey.getColumnNames() != null && primaryKey.getColumnNames().size() > 0) { columnSqls.add(buildPrimaryKeySql(primaryKey)); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java index d5261e16d59..c1cf2e4253f 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java @@ -169,10 +169,10 @@ protected Column buildColumn(ResultSet resultSet) throws SQLException { } @Override - protected void createTableInternal(TablePath tablePath, CatalogTable table) + protected void createTableInternal(TablePath tablePath, CatalogTable table, boolean createIndex) throws CatalogException { PostgresCreateTableSqlBuilder postgresCreateTableSqlBuilder = - new PostgresCreateTableSqlBuilder(table); + new PostgresCreateTableSqlBuilder(table, createIndex); String dbUrl = getUrlFromDatabaseName(tablePath.getDatabaseName()); try { String createTableSql = postgresCreateTableSqlBuilder.build(tablePath); @@ -199,9 +199,10 @@ protected void createTableInternal(TablePath tablePath, CatalogTable table) } @Override - protected String getCreateTableSql(TablePath tablePath, CatalogTable table) { + protected String getCreateTableSql( + TablePath tablePath, CatalogTable table, boolean createIndex) { PostgresCreateTableSqlBuilder postgresCreateTableSqlBuilder = - new PostgresCreateTableSqlBuilder(table); + new PostgresCreateTableSqlBuilder(table, createIndex); return postgresCreateTableSqlBuilder.build(tablePath); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCreateTableSqlBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCreateTableSqlBuilder.java index c3f414a0a5a..f7b98c1bb17 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCreateTableSqlBuilder.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCreateTableSqlBuilder.java @@ -44,13 +44,15 @@ public class PostgresCreateTableSqlBuilder { public Boolean isHaveConstraintKey = false; @Getter public List createIndexSqls = new ArrayList<>(); + private boolean createIndex; - public PostgresCreateTableSqlBuilder(CatalogTable catalogTable) { + public PostgresCreateTableSqlBuilder(CatalogTable catalogTable, boolean createIndex) { this.columns = catalogTable.getTableSchema().getColumns(); this.primaryKey = catalogTable.getTableSchema().getPrimaryKey(); this.sourceCatalogName = catalogTable.getCatalogName(); this.fieldIde = catalogTable.getOptions().get("fieldIde"); this.constraintKeys = catalogTable.getTableSchema().getConstraintKeys(); + this.createIndex = createIndex; } public String build(TablePath tablePath) { @@ -68,7 +70,7 @@ public String build(TablePath tablePath) { buildColumnSql(column), fieldIde)) .collect(Collectors.toList()); - if (CollectionUtils.isNotEmpty(constraintKeys)) { + if (createIndex && CollectionUtils.isNotEmpty(constraintKeys)) { for (ConstraintKey constraintKey : constraintKeys) { if (StringUtils.isBlank(constraintKey.getConstraintName()) || (primaryKey != null @@ -131,7 +133,9 @@ private String buildColumnSql(Column column) { } // Add primary key directly after the column if it is a primary key - if (primaryKey != null && primaryKey.getColumnNames().contains(column.getName())) { + if (createIndex + && primaryKey != null + && primaryKey.getColumnNames().contains(column.getName())) { columnSql.append(" PRIMARY KEY"); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalog.java index 064b2473371..b0fa834dc70 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalog.java @@ -122,9 +122,10 @@ protected String getTableName(ResultSet rs) throws SQLException { } @Override - protected String getCreateTableSql(TablePath tablePath, CatalogTable table) { + protected String getCreateTableSql( + TablePath tablePath, CatalogTable table, boolean createIndex) { String createTableSql = - new RedshiftCreateTableSqlBuilder(table) + new RedshiftCreateTableSqlBuilder(table, createIndex) .build(tablePath, table.getOptions().get("fieldIde")); return CatalogUtils.getFieldIde(createTableSql, table.getOptions().get("fieldIde")); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCreateTableSqlBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCreateTableSqlBuilder.java index 01e64397103..919adc10678 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCreateTableSqlBuilder.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCreateTableSqlBuilder.java @@ -35,11 +35,13 @@ public class RedshiftCreateTableSqlBuilder { private List columns; private PrimaryKey primaryKey; private String sourceCatalogName; + private boolean createIndex; - public RedshiftCreateTableSqlBuilder(CatalogTable catalogTable) { + public RedshiftCreateTableSqlBuilder(CatalogTable catalogTable, boolean createIndex) { this.columns = catalogTable.getTableSchema().getColumns(); this.primaryKey = catalogTable.getTableSchema().getPrimaryKey(); this.sourceCatalogName = catalogTable.getCatalogName(); + this.createIndex = createIndex; } public String build(TablePath tablePath) { @@ -61,7 +63,7 @@ public String build(TablePath tablePath, String fieldIde) { buildColumnSql(column), fieldIde)) .collect(Collectors.toList()); - if (primaryKey != null && primaryKey.getColumnNames().size() > 1) { + if (createIndex && primaryKey != null && primaryKey.getColumnNames().size() > 1) { columnSqls.add( CatalogUtils.quoteIdentifier( "PRIMARY KEY (" @@ -109,7 +111,8 @@ private String buildColumnSql(Column column) { columnSql.append(" NOT NULL"); } - if (primaryKey != null + if (createIndex + && primaryKey != null && primaryKey.getColumnNames().contains(column.getName()) && primaryKey.getColumnNames().size() == 1) { columnSql.append(" PRIMARY KEY"); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java index 19b8f668af9..56d68f02647 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCatalog.java @@ -137,8 +137,9 @@ protected String getDropDatabaseSql(String databaseName) { } @Override - protected String getCreateTableSql(TablePath tablePath, CatalogTable table) { - return new SapHanaCreateTableSqlBuilder(table).build(tablePath); + protected String getCreateTableSql( + TablePath tablePath, CatalogTable table, boolean createIndex) { + return new SapHanaCreateTableSqlBuilder(table, createIndex).build(tablePath); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCreateTableSqlBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCreateTableSqlBuilder.java index 49be1d3d7f1..27c591ab86c 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCreateTableSqlBuilder.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCreateTableSqlBuilder.java @@ -46,14 +46,16 @@ public class SapHanaCreateTableSqlBuilder extends AbstractJdbcCreateTableSqlBuil private final List constraintKeys; @Getter public List createIndexSqls = new ArrayList<>(); + private boolean createIndex; - public SapHanaCreateTableSqlBuilder(CatalogTable catalogTable) { + public SapHanaCreateTableSqlBuilder(CatalogTable catalogTable, boolean createIndex) { this.columns = catalogTable.getTableSchema().getColumns(); this.primaryKey = catalogTable.getTableSchema().getPrimaryKey(); this.sourceCatalogName = catalogTable.getCatalogName(); this.fieldIde = catalogTable.getOptions().get("fieldIde"); this.comment = catalogTable.getComment(); constraintKeys = catalogTable.getTableSchema().getConstraintKeys(); + this.createIndex = createIndex; } public String build(TablePath tablePath) { @@ -71,13 +73,14 @@ public String build(TablePath tablePath) { .collect(Collectors.toList()); // Add primary key directly in the create table statement - if (primaryKey != null + if (createIndex + && primaryKey != null && primaryKey.getColumnNames() != null && !primaryKey.getColumnNames().isEmpty()) { columnSqls.add(buildPrimaryKeySql(primaryKey)); } - if (CollectionUtils.isNotEmpty(constraintKeys)) { + if (createIndex && CollectionUtils.isNotEmpty(constraintKeys)) { for (ConstraintKey constraintKey : constraintKeys) { if (StringUtils.isBlank(constraintKey.getConstraintName()) || (primaryKey != null diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java index e4c63515220..dc6b42a4568 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java @@ -130,8 +130,10 @@ protected Column buildColumn(ResultSet resultSet) throws SQLException { } @Override - protected String getCreateTableSql(TablePath tablePath, CatalogTable table) { - return SqlServerCreateTableSqlBuilder.builder(tablePath, table).build(tablePath, table); + protected String getCreateTableSql( + TablePath tablePath, CatalogTable table, boolean createIndex) { + return SqlServerCreateTableSqlBuilder.builder(tablePath, table, createIndex) + .build(tablePath, table); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCreateTableSqlBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCreateTableSqlBuilder.java index 18d2da8d3f7..df258e364b7 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCreateTableSqlBuilder.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCreateTableSqlBuilder.java @@ -55,21 +55,23 @@ public class SqlServerCreateTableSqlBuilder { private List constraintKeys; private String fieldIde; + private boolean createIndex; - private SqlServerCreateTableSqlBuilder(String tableName) { + private SqlServerCreateTableSqlBuilder(String tableName, boolean createIndex) { checkNotNull(tableName, "tableName must not be null"); this.tableName = tableName; + this.createIndex = createIndex; } public static SqlServerCreateTableSqlBuilder builder( - TablePath tablePath, CatalogTable catalogTable) { + TablePath tablePath, CatalogTable catalogTable, boolean createIndex) { checkNotNull(tablePath, "tablePath must not be null"); checkNotNull(catalogTable, "catalogTable must not be null"); TableSchema tableSchema = catalogTable.getTableSchema(); checkNotNull(tableSchema, "tableSchema must not be null"); - return new SqlServerCreateTableSqlBuilder(tablePath.getTableName()) + return new SqlServerCreateTableSqlBuilder(tablePath.getTableName(), createIndex) .comment(catalogTable.getComment()) // todo: set charset and collate .engine(null) @@ -176,10 +178,10 @@ private String buildColumnsIdentifySql(String catalogName, Map c for (Column column : columns) { columnSqls.add("\t" + buildColumnIdentifySql(column, catalogName, columnComments)); } - if (primaryKey != null) { + if (createIndex && primaryKey != null) { columnSqls.add("\t" + buildPrimaryKeySql()); } - if (CollectionUtils.isNotEmpty(constraintKeys)) { + if (createIndex && CollectionUtils.isNotEmpty(constraintKeys)) { for (ConstraintKey constraintKey : constraintKeys) { if (StringUtils.isBlank(constraintKey.getConstraintName())) { continue; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java index a0b28e49abd..1ba14be9f5b 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java @@ -144,8 +144,9 @@ protected String getListDatabaseSql() { } @Override - protected String getCreateTableSql(TablePath tablePath, CatalogTable table) { - return new XuguCreateTableSqlBuilder(table).build(tablePath); + protected String getCreateTableSql( + TablePath tablePath, CatalogTable table, boolean createIndex) { + return new XuguCreateTableSqlBuilder(table, createIndex).build(tablePath); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCreateTableSqlBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCreateTableSqlBuilder.java index 19bce1a8ca0..8d0270a4461 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCreateTableSqlBuilder.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCreateTableSqlBuilder.java @@ -37,12 +37,14 @@ public class XuguCreateTableSqlBuilder { private PrimaryKey primaryKey; private String sourceCatalogName; private String fieldIde; + private boolean createIndex; - public XuguCreateTableSqlBuilder(CatalogTable catalogTable) { + public XuguCreateTableSqlBuilder(CatalogTable catalogTable, boolean createIndex) { this.columns = catalogTable.getTableSchema().getColumns(); this.primaryKey = catalogTable.getTableSchema().getPrimaryKey(); this.sourceCatalogName = catalogTable.getCatalogName(); this.fieldIde = catalogTable.getOptions().get("fieldIde"); + this.createIndex = createIndex; } public String build(TablePath tablePath) { @@ -58,7 +60,8 @@ public String build(TablePath tablePath) { .collect(Collectors.toList()); // Add primary key directly in the create table statement - if (primaryKey != null + if (createIndex + && primaryKey != null && primaryKey.getColumnNames() != null && primaryKey.getColumnNames().size() > 0) { columnSqls.add(buildPrimaryKeySql(primaryKey)); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSinkConfig.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSinkConfig.java index 8860703ca43..2b90c3a7a7d 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSinkConfig.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSinkConfig.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.config; import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions; import lombok.Builder; import lombok.Data; @@ -44,6 +45,7 @@ public class JdbcSinkConfig implements Serializable { @Builder.Default private boolean isPrimaryKeyUpdated = true; private boolean supportUpsertByInsertOnly; private boolean useCopyStatement; + @Builder.Default private boolean createIndex = true; public static JdbcSinkConfig of(ReadonlyConfig config) { JdbcSinkConfigBuilder builder = JdbcSinkConfig.builder(); @@ -57,6 +59,7 @@ public static JdbcSinkConfig of(ReadonlyConfig config) { builder.supportUpsertByInsertOnly(config.get(SUPPORT_UPSERT_BY_INSERT_ONLY)); builder.simpleSql(config.get(JdbcOptions.QUERY)); builder.useCopyStatement(config.get(JdbcOptions.USE_COPY_STATEMENT)); + builder.createIndex(config.get(JdbcCatalogOptions.CREATE_INDEX)); return builder.build(); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java index a6a162f472c..1ec9ab8883c 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java @@ -22,7 +22,6 @@ import org.apache.seatunnel.api.serialization.DefaultSerializer; import org.apache.seatunnel.api.serialization.Serializer; import org.apache.seatunnel.api.sink.DataSaveMode; -import org.apache.seatunnel.api.sink.DefaultSaveModeHandler; import org.apache.seatunnel.api.sink.SaveModeHandler; import org.apache.seatunnel.api.sink.SchemaSaveMode; import org.apache.seatunnel.api.sink.SeaTunnelSink; @@ -43,6 +42,7 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.dialectenum.FieldIdeEnum; +import org.apache.seatunnel.connectors.seatunnel.jdbc.sink.savemode.JdbcSaveModeHandler; import org.apache.seatunnel.connectors.seatunnel.jdbc.state.JdbcAggregatedCommitInfo; import org.apache.seatunnel.connectors.seatunnel.jdbc.state.JdbcSinkState; import org.apache.seatunnel.connectors.seatunnel.jdbc.state.XidInfo; @@ -219,16 +219,18 @@ public Optional getSaveModeHandler() { catalog, tablePath, catalogTable, - config.get(JdbcOptions.CUSTOM_SQL))); + config.get(JdbcOptions.CUSTOM_SQL), + jdbcSinkConfig.isCreateIndex())); } return Optional.of( - new DefaultSaveModeHandler( + new JdbcSaveModeHandler( schemaSaveMode, dataSaveMode, catalog, tablePath, catalogTable, - config.get(JdbcOptions.CUSTOM_SQL))); + config.get(JdbcOptions.CUSTOM_SQL), + jdbcSinkConfig.isCreateIndex())); } catch (Exception e) { throw new JdbcConnectorException(HANDLE_SAVE_MODE_FAILED, e); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java index 35e9a986ab8..214afcba068 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java @@ -52,6 +52,7 @@ import static org.apache.seatunnel.api.sink.SinkReplaceNameConstant.REPLACE_DATABASE_NAME_KEY; import static org.apache.seatunnel.api.sink.SinkReplaceNameConstant.REPLACE_SCHEMA_NAME_KEY; import static org.apache.seatunnel.api.sink.SinkReplaceNameConstant.REPLACE_TABLE_NAME_KEY; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions.CREATE_INDEX; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions.AUTO_COMMIT; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions.BATCH_SIZE; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions.COMPATIBLE_MODE; @@ -257,6 +258,7 @@ public OptionRule optionRule() { return OptionRule.builder() .required(URL, DRIVER, SCHEMA_SAVE_MODE, DATA_SAVE_MODE) .optional( + CREATE_INDEX, USER, PASSWORD, CONNECTION_CHECK_TIMEOUT_SEC, diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/savemode/JdbcSaveModeHandler.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/savemode/JdbcSaveModeHandler.java new file mode 100644 index 00000000000..87a2b7114db --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/savemode/JdbcSaveModeHandler.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.sink.savemode; + +import org.apache.seatunnel.api.sink.DataSaveMode; +import org.apache.seatunnel.api.sink.DefaultSaveModeHandler; +import org.apache.seatunnel.api.sink.SchemaSaveMode; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class JdbcSaveModeHandler extends DefaultSaveModeHandler { + public boolean createIndex; + + public JdbcSaveModeHandler( + SchemaSaveMode schemaSaveMode, + DataSaveMode dataSaveMode, + Catalog catalog, + TablePath tablePath, + CatalogTable catalogTable, + String customSql, + boolean createIndex) { + super(schemaSaveMode, dataSaveMode, catalog, tablePath, catalogTable, customSql); + this.createIndex = createIndex; + } + + @Override + protected void createTable() { + super.createTablePreCheck(); + catalog.createTable(tablePath, catalogTable, true, createIndex); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sql/IrisCreateTableSqlBuilderTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCreateTableSqlBuilderTest.java similarity index 84% rename from seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sql/IrisCreateTableSqlBuilderTest.java rename to seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCreateTableSqlBuilderTest.java index 20c65d06c25..0c1108b5760 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sql/IrisCreateTableSqlBuilderTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/iris/IrisCreateTableSqlBuilderTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.sql; +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.iris; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.ConstraintKey; @@ -26,7 +26,6 @@ import org.apache.seatunnel.api.table.catalog.TableSchema; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.LocalTimeType; -import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.iris.IrisCreateTableSqlBuilder; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -91,7 +90,7 @@ public void TestCreateTableSqlBuilder() { new ArrayList<>(), "User table"); - String createTableSql = new IrisCreateTableSqlBuilder(catalogTable).build(tablePath); + String createTableSql = new IrisCreateTableSqlBuilder(catalogTable, true).build(tablePath); // create table sql is change; The old unit tests are no longer applicable String expect = "CREATE TABLE \"test_schema\".\"test_table\" (\n" @@ -105,7 +104,21 @@ public void TestCreateTableSqlBuilder() { + "UNIQUE (\"name\")\n" + ");\n" + "CREATE INDEX test_table_age ON \"test_schema\".\"test_table\"(\"age\");"; - System.out.println(createTableSql); Assertions.assertEquals(expect, createTableSql); + + // skip index + String createTableSqlSkipIndex = + new IrisCreateTableSqlBuilder(catalogTable, false).build(tablePath); + // create table sql is change; The old unit tests are no longer applicable + String expectSkipIndex = + "CREATE TABLE \"test_schema\".\"test_table\" (\n" + + " %Description 'User table',\n" + + "\"id\" BIGINT NOT NULL %Description 'id',\n" + + "\"name\" VARCHAR(128) NOT NULL %Description 'name',\n" + + "\"age\" INTEGER %Description 'age',\n" + + "\"createTime\" TIMESTAMP2 %Description 'createTime',\n" + + "\"lastUpdateTime\" TIMESTAMP2 %Description 'lastUpdateTime'\n" + + ");\n"; + Assertions.assertEquals(expectSkipIndex, createTableSqlSkipIndex); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sql/MysqlCreateTableSqlBuilderTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MysqlCreateTableSqlBuilderTest.java similarity index 85% rename from seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sql/MysqlCreateTableSqlBuilderTest.java rename to seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MysqlCreateTableSqlBuilderTest.java index 745c7031f8d..3c433959316 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sql/MysqlCreateTableSqlBuilderTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MysqlCreateTableSqlBuilderTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.sql; +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.mysql; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.ConstraintKey; @@ -27,7 +27,6 @@ import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.LocalTimeType; import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; -import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.mysql.MysqlCreateTableSqlBuilder; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.mysql.MySqlTypeConverter; @@ -110,7 +109,7 @@ public void testBuild() { String createTableSql = MysqlCreateTableSqlBuilder.builder( - tablePath, catalogTable, MySqlTypeConverter.DEFAULT_INSTANCE) + tablePath, catalogTable, MySqlTypeConverter.DEFAULT_INSTANCE, true) .build(DatabaseIdentifier.MYSQL); // create table sql is change; The old unit tests are no longer applicable String expect = @@ -127,5 +126,22 @@ public void testBuild() { + ") COMMENT = 'User table';"; CONSOLE.println(expect); Assertions.assertEquals(expect, createTableSql); + + // skip index + String createTableSqlSkipIndex = + MysqlCreateTableSqlBuilder.builder( + tablePath, catalogTable, MySqlTypeConverter.DEFAULT_INSTANCE, false) + .build(DatabaseIdentifier.MYSQL); + String expectSkipIndex = + "CREATE TABLE `test_table` (\n" + + "\t`id` BIGINT NOT NULL COMMENT 'id', \n" + + "\t`name` VARCHAR(128) NOT NULL COMMENT 'name', \n" + + "\t`age` INT NULL COMMENT 'age', \n" + + "\t`blob_v` LONGBLOB NULL COMMENT 'blob_v', \n" + + "\t`createTime` DATETIME NULL COMMENT 'createTime', \n" + + "\t`lastUpdateTime` DATETIME NULL COMMENT 'lastUpdateTime'\n" + + ") COMMENT = 'User table';"; + CONSOLE.println(expectSkipIndex); + Assertions.assertEquals(expectSkipIndex, createTableSqlSkipIndex); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java index 75b22ec24dc..9f4d8e86198 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java @@ -37,9 +37,9 @@ static void before() { catalog = new OracleCatalog( "oracle", - "c##gguser", - "testdb", - OracleURLParser.parse("jdbc:oracle:thin:@127.0.0.1:1521/CDC_PDB"), + "test", + "oracle", + OracleURLParser.parse("jdbc:oracle:thin:@127.0.0.1:1521:xe"), null); catalog.open(); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilderTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilderTest.java new file mode 100644 index 00000000000..6005aa0b262 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilderTest.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import com.google.common.collect.Lists; + +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; + +public class OracleCreateTableSqlBuilderTest { + + private static final PrintStream CONSOLE = System.out; + + @Test + public void testBuild() { + String dataBaseName = "test_database"; + String tableName = "test_table"; + TablePath tablePath = TablePath.of(dataBaseName, tableName); + TableSchema tableSchema = + TableSchema.builder() + .column(PhysicalColumn.of("id", BasicType.LONG_TYPE, 22, false, null, "id")) + .column( + PhysicalColumn.of( + "name", BasicType.STRING_TYPE, 128, false, null, "name")) + .column( + PhysicalColumn.of( + "age", BasicType.INT_TYPE, (Long) null, true, null, "age")) + .column( + PhysicalColumn.of( + "blob_v", + PrimitiveByteArrayType.INSTANCE, + Long.MAX_VALUE, + true, + null, + "blob_v")) + .column( + PhysicalColumn.of( + "createTime", + LocalTimeType.LOCAL_DATE_TIME_TYPE, + 3, + true, + null, + "createTime")) + .column( + PhysicalColumn.of( + "lastUpdateTime", + LocalTimeType.LOCAL_DATE_TIME_TYPE, + 3, + true, + null, + "lastUpdateTime")) + .primaryKey(PrimaryKey.of("id", Lists.newArrayList("id"))) + .constraintKey( + Arrays.asList( + ConstraintKey.of( + ConstraintKey.ConstraintType.INDEX_KEY, + "name", + Lists.newArrayList( + ConstraintKey.ConstraintKeyColumn.of( + "name", null))), + ConstraintKey.of( + ConstraintKey.ConstraintType.INDEX_KEY, + "blob_v", + Lists.newArrayList( + ConstraintKey.ConstraintKeyColumn.of( + "blob_v", null))))) + .build(); + CatalogTable catalogTable = + CatalogTable.of( + TableIdentifier.of("test_catalog", dataBaseName, tableName), + tableSchema, + new HashMap<>(), + new ArrayList<>(), + "User table"); + + OracleCreateTableSqlBuilder oracleCreateTableSqlBuilder = + new OracleCreateTableSqlBuilder(catalogTable, true); + String createTableSql = oracleCreateTableSqlBuilder.build(tablePath).get(0); + // create table sql is change; The old unit tests are no longer applicable + String expect = + "CREATE TABLE \"test_table\" (\n" + + "\"id\" INTEGER NOT NULL,\n" + + "\"name\" VARCHAR2(128) NOT NULL,\n" + + "\"age\" INTEGER,\n" + + "\"blob_v\" BLOB,\n" + + "\"createTime\" TIMESTAMP WITH LOCAL TIME ZONE,\n" + + "\"lastUpdateTime\" TIMESTAMP WITH LOCAL TIME ZONE,\n" + + "CONSTRAINT id_9a8b PRIMARY KEY (\"id\")\n" + + ")"; + + // replace "CONSTRAINT id_xxxx" because it's dynamically generated(random) + String regex = "id_\\w+"; + String replacedStr1 = createTableSql.replaceAll(regex, "id_"); + String replacedStr2 = expect.replaceAll(regex, "id_"); + CONSOLE.println(replacedStr2); + Assertions.assertEquals(replacedStr2, replacedStr1); + + // skip index + OracleCreateTableSqlBuilder oracleCreateTableSqlBuilderSkipIndex = + new OracleCreateTableSqlBuilder(catalogTable, false); + String createTableSqlSkipIndex = + oracleCreateTableSqlBuilderSkipIndex.build(tablePath).get(0); + String expectSkipIndex = + "CREATE TABLE \"test_table\" (\n" + + "\"id\" INTEGER NOT NULL,\n" + + "\"name\" VARCHAR2(128) NOT NULL,\n" + + "\"age\" INTEGER,\n" + + "\"blob_v\" BLOB,\n" + + "\"createTime\" TIMESTAMP WITH LOCAL TIME ZONE,\n" + + "\"lastUpdateTime\" TIMESTAMP WITH LOCAL TIME ZONE\n" + + ")"; + CONSOLE.println(expectSkipIndex); + Assertions.assertEquals(expectSkipIndex, createTableSqlSkipIndex); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCreateTableSqlBuilderTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCreateTableSqlBuilderTest.java index 446fac45744..37049eced38 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCreateTableSqlBuilderTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCreateTableSqlBuilderTest.java @@ -45,7 +45,7 @@ void build() { otherDB -> { CatalogTable catalogTable = catalogTable(otherDB); PostgresCreateTableSqlBuilder postgresCreateTableSqlBuilder = - new PostgresCreateTableSqlBuilder(catalogTable); + new PostgresCreateTableSqlBuilder(catalogTable, true); String createTableSql = postgresCreateTableSqlBuilder.build( catalogTable.getTableId().toTablePath()); @@ -61,6 +61,23 @@ void build() { Lists.newArrayList( "CREATE INDEX test_index_age ON \"test\"(\"age\");"), postgresCreateTableSqlBuilder.getCreateIndexSqls()); + + // skip index + PostgresCreateTableSqlBuilder postgresCreateTableSqlBuilderSkipIndex = + new PostgresCreateTableSqlBuilder(catalogTable, false); + String createTableSqlSkipIndex = + postgresCreateTableSqlBuilderSkipIndex.build( + catalogTable.getTableId().toTablePath()); + Assertions.assertEquals( + "CREATE TABLE \"test\" (\n" + + "\"id\" int4 NOT NULL,\n" + + "\"name\" text NOT NULL,\n" + + "\"age\" int4 NOT NULL\n" + + ");", + createTableSqlSkipIndex); + Assertions.assertEquals( + Lists.newArrayList(), + postgresCreateTableSqlBuilderSkipIndex.getCreateIndexSqls()); }); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalogTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalogTest.java index 27439ec1ed7..6253e9cc88a 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalogTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCatalogTest.java @@ -89,7 +89,7 @@ void testCreateTableSqlWithPrimaryKeys() { put("password", "test"); } })); - String sql = catalog.getCreateTableSql(TablePath.of("test.test.test"), CATALOG_TABLE); + String sql = catalog.getCreateTableSql(TablePath.of("test.test.test"), CATALOG_TABLE, true); Assertions.assertEquals( "CREATE TABLE \"test\".\"test\" (\n" + "\"test\" CHARACTER VARYING(65535),\n" diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCreateTableSqlBuilderTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCreateTableSqlBuilderTest.java new file mode 100644 index 00000000000..84d9e937117 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/redshift/RedshiftCreateTableSqlBuilderTest.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.redshift; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import com.google.common.collect.Lists; + +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; + +public class RedshiftCreateTableSqlBuilderTest { + + private static final PrintStream CONSOLE = System.out; + + @Test + public void testBuild() { + String dataBaseName = "test_database"; + String tableName = "test_table"; + TablePath tablePath = TablePath.of(dataBaseName, tableName); + TableSchema tableSchema = + TableSchema.builder() + .column(PhysicalColumn.of("id", BasicType.LONG_TYPE, 22, false, null, "id")) + .column( + PhysicalColumn.of( + "name", BasicType.STRING_TYPE, 128, false, null, "name")) + .column( + PhysicalColumn.of( + "age", BasicType.INT_TYPE, (Long) null, true, null, "age")) + .column( + PhysicalColumn.of( + "blob_v", + PrimitiveByteArrayType.INSTANCE, + Long.MAX_VALUE, + true, + null, + "blob_v")) + .column( + PhysicalColumn.of( + "createTime", + LocalTimeType.LOCAL_DATE_TIME_TYPE, + 3, + true, + null, + "createTime")) + .column( + PhysicalColumn.of( + "lastUpdateTime", + LocalTimeType.LOCAL_DATE_TIME_TYPE, + 3, + true, + null, + "lastUpdateTime")) + .primaryKey(PrimaryKey.of("id", Lists.newArrayList("id"))) + .constraintKey( + Arrays.asList( + ConstraintKey.of( + ConstraintKey.ConstraintType.INDEX_KEY, + "name", + Lists.newArrayList( + ConstraintKey.ConstraintKeyColumn.of( + "name", null))), + ConstraintKey.of( + ConstraintKey.ConstraintType.INDEX_KEY, + "blob_v", + Lists.newArrayList( + ConstraintKey.ConstraintKeyColumn.of( + "blob_v", null))))) + .build(); + CatalogTable catalogTable = + CatalogTable.of( + TableIdentifier.of("test_catalog", dataBaseName, tableName), + tableSchema, + new HashMap<>(), + new ArrayList<>(), + "User table"); + + RedshiftCreateTableSqlBuilder redshiftCreateTableSqlBuilder = + new RedshiftCreateTableSqlBuilder(catalogTable, true); + String createTableSql = redshiftCreateTableSqlBuilder.build(tablePath); + // create table sql is change; The old unit tests are no longer applicable + String expect = + "CREATE TABLE \"test_table\" (\n" + + "\"id\" BIGINT NOT NULL PRIMARY KEY,\n" + + "\"name\" CHARACTER VARYING(128) NOT NULL,\n" + + "\"age\" INTEGER,\n" + + "\"blob_v\" BINARY VARYING(1024000),\n" + + "\"createTime\" TIMESTAMP WITHOUT TIME ZONE,\n" + + "\"lastUpdateTime\" TIMESTAMP WITHOUT TIME ZONE\n" + + ");\n" + + "COMMENT ON COLUMN \"test_table\".\"id\" IS 'id';\n" + + "COMMENT ON COLUMN \"test_table\".\"name\" IS 'name';\n" + + "COMMENT ON COLUMN \"test_table\".\"age\" IS 'age';\n" + + "COMMENT ON COLUMN \"test_table\".\"blob_v\" IS 'blob_v';\n" + + "COMMENT ON COLUMN \"test_table\".\"createTime\" IS 'createTime';\n" + + "COMMENT ON COLUMN \"test_table\".\"lastUpdateTime\" IS 'lastUpdateTime';"; + + CONSOLE.println(expect); + Assertions.assertEquals(expect, createTableSql); + + // skip index + RedshiftCreateTableSqlBuilder redshiftCreateTableSqlBuilderSkipIndex = + new RedshiftCreateTableSqlBuilder(catalogTable, false); + String createTableSqlSkipIndex = redshiftCreateTableSqlBuilderSkipIndex.build(tablePath); + String expectSkipIndex = + "CREATE TABLE \"test_table\" (\n" + + "\"id\" BIGINT NOT NULL,\n" + + "\"name\" CHARACTER VARYING(128) NOT NULL,\n" + + "\"age\" INTEGER,\n" + + "\"blob_v\" BINARY VARYING(1024000),\n" + + "\"createTime\" TIMESTAMP WITHOUT TIME ZONE,\n" + + "\"lastUpdateTime\" TIMESTAMP WITHOUT TIME ZONE\n" + + ");\n" + + "COMMENT ON COLUMN \"test_table\".\"id\" IS 'id';\n" + + "COMMENT ON COLUMN \"test_table\".\"name\" IS 'name';\n" + + "COMMENT ON COLUMN \"test_table\".\"age\" IS 'age';\n" + + "COMMENT ON COLUMN \"test_table\".\"blob_v\" IS 'blob_v';\n" + + "COMMENT ON COLUMN \"test_table\".\"createTime\" IS 'createTime';\n" + + "COMMENT ON COLUMN \"test_table\".\"lastUpdateTime\" IS 'lastUpdateTime';"; + CONSOLE.println(expectSkipIndex); + Assertions.assertEquals(expectSkipIndex, createTableSqlSkipIndex); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCreateTableSqlBuilderTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCreateTableSqlBuilderTest.java index a735e5c74a4..03699896b58 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCreateTableSqlBuilderTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/saphana/SapHanaCreateTableSqlBuilderTest.java @@ -84,7 +84,8 @@ public void testBuild() { new ArrayList<>(), "User table"); - String createTableSql = new SapHanaCreateTableSqlBuilder(catalogTable).build(tablePath); + String createTableSql = + new SapHanaCreateTableSqlBuilder(catalogTable, true).build(tablePath); String expect = "CREATE TABLE \"test_database\".\"test_table\" (\n" + "\"id\" BIGINT NOT NULL COMMENT 'id',\n" @@ -96,5 +97,18 @@ public void testBuild() { + "UNIQUE (\"name\")\n" + ") COMMENT 'User table'"; Assertions.assertEquals(expect, createTableSql); + + // skip index + String createTableSqlSkipIndex = + new SapHanaCreateTableSqlBuilder(catalogTable, false).build(tablePath); + String expectSkipIndex = + "CREATE TABLE \"test_database\".\"test_table\" (\n" + + "\"id\" BIGINT NOT NULL COMMENT 'id',\n" + + "\"name\" NVARCHAR(128) NOT NULL COMMENT 'name',\n" + + "\"age\" INTEGER NULL COMMENT 'age',\n" + + "\"createTime\" SECONDDATE NULL COMMENT 'createTime',\n" + + "\"lastUpdateTime\" SECONDDATE NULL COMMENT 'lastUpdateTime'\n" + + ") COMMENT 'User table'"; + Assertions.assertEquals(expectSkipIndex, createTableSqlSkipIndex); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCreateTableSqlBuilderTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCreateTableSqlBuilderTest.java new file mode 100644 index 00000000000..04f765f4e5a --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCreateTableSqlBuilderTest.java @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.sqlserver; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import com.google.common.collect.Lists; + +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; + +public class SqlServerCreateTableSqlBuilderTest { + + private static final PrintStream CONSOLE = System.out; + + @Test + public void testBuild() { + String dataBaseName = "test_database"; + String tableName = "test_table"; + TablePath tablePath = TablePath.of(dataBaseName, tableName); + TableSchema tableSchema = + TableSchema.builder() + .column(PhysicalColumn.of("id", BasicType.LONG_TYPE, 22, false, null, "id")) + .column( + PhysicalColumn.of( + "name", BasicType.STRING_TYPE, 128, false, null, "name")) + .column( + PhysicalColumn.of( + "age", BasicType.INT_TYPE, (Long) null, true, null, "age")) + .column( + PhysicalColumn.of( + "blob_v", + PrimitiveByteArrayType.INSTANCE, + Long.MAX_VALUE, + true, + null, + "blob_v")) + .column( + PhysicalColumn.of( + "createTime", + LocalTimeType.LOCAL_DATE_TIME_TYPE, + 3, + true, + null, + "createTime")) + .column( + PhysicalColumn.of( + "lastUpdateTime", + LocalTimeType.LOCAL_DATE_TIME_TYPE, + 3, + true, + null, + "lastUpdateTime")) + .primaryKey(PrimaryKey.of("id", Lists.newArrayList("id"))) + .constraintKey( + Arrays.asList( + ConstraintKey.of( + ConstraintKey.ConstraintType.INDEX_KEY, + "name", + Lists.newArrayList( + ConstraintKey.ConstraintKeyColumn.of( + "name", null))), + ConstraintKey.of( + ConstraintKey.ConstraintType.INDEX_KEY, + "blob_v", + Lists.newArrayList( + ConstraintKey.ConstraintKeyColumn.of( + "blob_v", null))))) + .build(); + CatalogTable catalogTable = + CatalogTable.of( + TableIdentifier.of("test_catalog", dataBaseName, tableName), + tableSchema, + new HashMap<>(), + new ArrayList<>(), + "User table"); + + SqlServerCreateTableSqlBuilder sqlServerCreateTableSqlBuilder = + SqlServerCreateTableSqlBuilder.builder(tablePath, catalogTable, true); + String createTableSql = sqlServerCreateTableSqlBuilder.build(tablePath, catalogTable); + // create table sql is change; The old unit tests are no longer applicable + String expect = + "IF OBJECT_ID('[test_database].[test_table]', 'U') IS NULL \n" + + "BEGIN \n" + + "CREATE TABLE [test_database].[test_table] ( \n" + + "\t[id] BIGINT NOT NULL, \n" + + "\t[name] NVARCHAR(128) NOT NULL, \n" + + "\t[age] INT NULL, \n" + + "\t[blob_v] VARBINARY(MAX) NULL, \n" + + "\t[createTime] DATETIME2 NULL, \n" + + "\t[lastUpdateTime] DATETIME2 NULL, \n" + + "\tPRIMARY KEY ([id])\n" + + ");\n" + + "EXEC test_database.sys.sp_addextendedproperty 'MS_Description', N'User table', 'schema', N'null', 'table', N'test_table';\n" + + "EXEC test_database.sys.sp_addextendedproperty 'MS_Description', N'blob_v', 'schema', N'null', 'table', N'test_table', 'column', N'blob_v';\n" + + "EXEC test_database.sys.sp_addextendedproperty 'MS_Description', N'createTime', 'schema', N'null', 'table', N'test_table', 'column', N'createTime';\n" + + "EXEC test_database.sys.sp_addextendedproperty 'MS_Description', N'name', 'schema', N'null', 'table', N'test_table', 'column', N'name';\n" + + "EXEC test_database.sys.sp_addextendedproperty 'MS_Description', N'id', 'schema', N'null', 'table', N'test_table', 'column', N'id';\n" + + "EXEC test_database.sys.sp_addextendedproperty 'MS_Description', N'age', 'schema', N'null', 'table', N'test_table', 'column', N'age';\n" + + "EXEC test_database.sys.sp_addextendedproperty 'MS_Description', N'lastUpdateTime', 'schema', N'null', 'table', N'test_table', 'column', N'lastUpdateTime';\n" + + "\n" + + "END"; + + CONSOLE.println(expect); + Assertions.assertEquals(expect, createTableSql); + + // skip index + SqlServerCreateTableSqlBuilder sqlServerCreateTableSqlBuilderSkipIndex = + SqlServerCreateTableSqlBuilder.builder(tablePath, catalogTable, false); + String createTableSqlSkipIndex = + sqlServerCreateTableSqlBuilderSkipIndex.build(tablePath, catalogTable); + String expectSkipIndex = + "IF OBJECT_ID('[test_database].[test_table]', 'U') IS NULL \n" + + "BEGIN \n" + + "CREATE TABLE [test_database].[test_table] ( \n" + + "\t[id] BIGINT NOT NULL, \n" + + "\t[name] NVARCHAR(128) NOT NULL, \n" + + "\t[age] INT NULL, \n" + + "\t[blob_v] VARBINARY(MAX) NULL, \n" + + "\t[createTime] DATETIME2 NULL, \n" + + "\t[lastUpdateTime] DATETIME2 NULL\n" + + ");\n" + + "EXEC test_database.sys.sp_addextendedproperty 'MS_Description', N'User table', 'schema', N'null', 'table', N'test_table';\n" + + "EXEC test_database.sys.sp_addextendedproperty 'MS_Description', N'blob_v', 'schema', N'null', 'table', N'test_table', 'column', N'blob_v';\n" + + "EXEC test_database.sys.sp_addextendedproperty 'MS_Description', N'createTime', 'schema', N'null', 'table', N'test_table', 'column', N'createTime';\n" + + "EXEC test_database.sys.sp_addextendedproperty 'MS_Description', N'name', 'schema', N'null', 'table', N'test_table', 'column', N'name';\n" + + "EXEC test_database.sys.sp_addextendedproperty 'MS_Description', N'id', 'schema', N'null', 'table', N'test_table', 'column', N'id';\n" + + "EXEC test_database.sys.sp_addextendedproperty 'MS_Description', N'age', 'schema', N'null', 'table', N'test_table', 'column', N'age';\n" + + "EXEC test_database.sys.sp_addextendedproperty 'MS_Description', N'lastUpdateTime', 'schema', N'null', 'table', N'test_table', 'column', N'lastUpdateTime';\n" + + "\n" + + "END"; + CONSOLE.println(expectSkipIndex); + Assertions.assertEquals(expectSkipIndex, createTableSqlSkipIndex); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCreateTableSqlBuilderTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCreateTableSqlBuilderTest.java new file mode 100644 index 00000000000..8c8de29cace --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCreateTableSqlBuilderTest.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.xugu; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import com.google.common.collect.Lists; + +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; + +public class XuguCreateTableSqlBuilderTest { + + private static final PrintStream CONSOLE = System.out; + + @Test + public void testBuild() { + String dataBaseName = "test_database"; + String tableName = "test_table"; + TablePath tablePath = TablePath.of(dataBaseName, tableName); + TableSchema tableSchema = + TableSchema.builder() + .column(PhysicalColumn.of("id", BasicType.LONG_TYPE, 22, false, null, "id")) + .column( + PhysicalColumn.of( + "name", BasicType.STRING_TYPE, 128, false, null, "name")) + .column( + PhysicalColumn.of( + "age", BasicType.INT_TYPE, (Long) null, true, null, "age")) + .column( + PhysicalColumn.of( + "blob_v", + PrimitiveByteArrayType.INSTANCE, + Long.MAX_VALUE, + true, + null, + "blob_v")) + .column( + PhysicalColumn.of( + "createTime", + LocalTimeType.LOCAL_DATE_TIME_TYPE, + 3, + true, + null, + "createTime")) + .column( + PhysicalColumn.of( + "lastUpdateTime", + LocalTimeType.LOCAL_DATE_TIME_TYPE, + 3, + true, + null, + "lastUpdateTime")) + .primaryKey(PrimaryKey.of("id", Lists.newArrayList("id"))) + .constraintKey( + Arrays.asList( + ConstraintKey.of( + ConstraintKey.ConstraintType.INDEX_KEY, + "name", + Lists.newArrayList( + ConstraintKey.ConstraintKeyColumn.of( + "name", null))), + ConstraintKey.of( + ConstraintKey.ConstraintType.INDEX_KEY, + "blob_v", + Lists.newArrayList( + ConstraintKey.ConstraintKeyColumn.of( + "blob_v", null))))) + .build(); + CatalogTable catalogTable = + CatalogTable.of( + TableIdentifier.of("test_catalog", dataBaseName, tableName), + tableSchema, + new HashMap<>(), + new ArrayList<>(), + "User table"); + + XuguCreateTableSqlBuilder xuguCreateTableSqlBuilder = + new XuguCreateTableSqlBuilder(catalogTable, true); + String createTableSql = xuguCreateTableSqlBuilder.build(tablePath); + // create table sql is change; The old unit tests are no longer applicable + String expect = + "CREATE TABLE \"test_table\" (\n" + + "\"id\" BIGINT NOT NULL,\n" + + "\"name\" VARCHAR(128) NOT NULL,\n" + + "\"age\" INTEGER,\n" + + "\"blob_v\" BLOB,\n" + + "\"createTime\" TIMESTAMP,\n" + + "\"lastUpdateTime\" TIMESTAMP,\n" + + "CONSTRAINT id_88a3 PRIMARY KEY (\"id\")\n" + + ");\n" + + "COMMENT ON COLUMN \"test_table\".\"id\" IS 'id';\n" + + "COMMENT ON COLUMN \"test_table\".\"name\" IS 'name';\n" + + "COMMENT ON COLUMN \"test_table\".\"age\" IS 'age';\n" + + "COMMENT ON COLUMN \"test_table\".\"blob_v\" IS 'blob_v';\n" + + "COMMENT ON COLUMN \"test_table\".\"createTime\" IS 'createTime';\n" + + "COMMENT ON COLUMN \"test_table\".\"lastUpdateTime\" IS 'lastUpdateTime'"; + + // replace "CONSTRAINT id_xxxx" because it's dynamically generated(random) + String regex = "id_\\w+"; + String replacedStr1 = createTableSql.replaceAll(regex, "id_"); + String replacedStr2 = expect.replaceAll(regex, "id_"); + CONSOLE.println(replacedStr2); + Assertions.assertEquals(replacedStr2, replacedStr1); + + // skip index + XuguCreateTableSqlBuilder xuguCreateTableSqlBuilderSkipIndex = + new XuguCreateTableSqlBuilder(catalogTable, false); + String createTableSqlSkipIndex = xuguCreateTableSqlBuilderSkipIndex.build(tablePath); + String expectSkipIndex = + "CREATE TABLE \"test_table\" (\n" + + "\"id\" BIGINT NOT NULL,\n" + + "\"name\" VARCHAR(128) NOT NULL,\n" + + "\"age\" INTEGER,\n" + + "\"blob_v\" BLOB,\n" + + "\"createTime\" TIMESTAMP,\n" + + "\"lastUpdateTime\" TIMESTAMP\n" + + ");\n" + + "COMMENT ON COLUMN \"test_table\".\"id\" IS 'id';\n" + + "COMMENT ON COLUMN \"test_table\".\"name\" IS 'name';\n" + + "COMMENT ON COLUMN \"test_table\".\"age\" IS 'age';\n" + + "COMMENT ON COLUMN \"test_table\".\"blob_v\" IS 'blob_v';\n" + + "COMMENT ON COLUMN \"test_table\".\"createTime\" IS 'createTime';\n" + + "COMMENT ON COLUMN \"test_table\".\"lastUpdateTime\" IS 'lastUpdateTime'"; + CONSOLE.println(expectSkipIndex); + Assertions.assertEquals(expectSkipIndex, createTableSqlSkipIndex); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java index e1cb5fb3055..7cd6be4fd95 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java @@ -22,10 +22,15 @@ import org.apache.seatunnel.api.table.catalog.Catalog; import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.common.utils.ExceptionUtils; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.iris.IrisCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleCatalog; import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; @@ -349,11 +354,77 @@ public void testJdbcDb(TestContainer container) protected void initCatalog() {} @Test - public void testCatalog() { + public void testCreateIndex() { if (catalog == null) { return; } + TablePath sourceTablePath = + new TablePath( + jdbcCase.getDatabase(), jdbcCase.getSchema(), jdbcCase.getSourceTable()); + // add suffix for target table + TablePath targetTablePath = + new TablePath( + jdbcCase.getDatabase(), + jdbcCase.getSchema(), + jdbcCase.getSinkTable() + + ((catalog instanceof OracleCatalog) ? "_INDEX" : "_index")); + boolean createdDb = false; + + if (!(catalog instanceof IrisCatalog) + && !catalog.databaseExists(targetTablePath.getDatabaseName())) { + catalog.createDatabase(targetTablePath, false); + Assertions.assertTrue(catalog.databaseExists(targetTablePath.getDatabaseName())); + createdDb = true; + } + + CatalogTable catalogTable = catalog.getTable(sourceTablePath); + + // not create index + createIndexOrNot(targetTablePath, catalogTable, false); + Assertions.assertFalse(hasIndex(catalog, targetTablePath)); + + dropTableWithAssert(targetTablePath); + // create index + createIndexOrNot(targetTablePath, catalogTable, true); + Assertions.assertTrue(hasIndex(catalog, targetTablePath)); + + dropTableWithAssert(targetTablePath); + + if (createdDb) { + catalog.dropDatabase(targetTablePath, false); + Assertions.assertFalse(catalog.databaseExists(targetTablePath.getDatabaseName())); + } + } + + private boolean hasIndex(Catalog catalog, TablePath targetTablePath) { + TableSchema tableSchema = catalog.getTable(targetTablePath).getTableSchema(); + PrimaryKey primaryKey = tableSchema.getPrimaryKey(); + List constraintKeys = tableSchema.getConstraintKeys(); + if (primaryKey != null && StringUtils.isNotBlank(primaryKey.getPrimaryKey())) { + return true; + } + if (!constraintKeys.isEmpty()) { + return true; + } + return false; + } + protected void dropTableWithAssert(TablePath targetTablePath) { + catalog.dropTable(targetTablePath, true); + Assertions.assertFalse(catalog.tableExists(targetTablePath)); + } + + protected void createIndexOrNot( + TablePath targetTablePath, CatalogTable catalogTable, boolean createIndex) { + catalog.createTable(targetTablePath, catalogTable, false, createIndex); + Assertions.assertTrue(catalog.tableExists(targetTablePath)); + } + + @Test + public void testCatalog() { + if (catalog == null) { + return; + } TablePath sourceTablePath = new TablePath( jdbcCase.getDatabase(), jdbcCase.getSchema(), jdbcCase.getSourceTable()); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlIT.java index 4c2ecc94e39..6bd97863b25 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlIT.java @@ -146,7 +146,8 @@ public class JdbcMysqlIT extends AbstractJdbcIT { + " `c_integer_unsigned` int(10) unsigned DEFAULT NULL,\n" + " `c_bigint_30` BIGINT(40) unsigned DEFAULT NULL,\n" + " `c_decimal_unsigned_30` DECIMAL(30) unsigned DEFAULT NULL,\n" - + " `c_decimal_30` DECIMAL(30) DEFAULT NULL\n" + + " `c_decimal_30` DECIMAL(30) DEFAULT NULL,\n" + + " UNIQUE (c_bigint_30)\n" + ");"; @Override diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java index 9d3597c435b..e4b4de39501 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java @@ -72,6 +72,26 @@ public class JdbcOracleIT extends AbstractJdbcIT { "/jdbc_oracle_source_to_sink_use_select3.conf"); private static final String CREATE_SQL = + "create table %s\n" + + "(\n" + + " VARCHAR_10_COL varchar2(10),\n" + + " CHAR_10_COL char(10),\n" + + " CLOB_COL clob,\n" + + " NUMBER_3_SF_2_DP number(3, 2),\n" + + " NUMBER_7_SF_N2_DP number(7, -2),\n" + + " INTEGER_COL integer,\n" + + " FLOAT_COL float(10),\n" + + " REAL_COL real,\n" + + " BINARY_FLOAT_COL binary_float,\n" + + " BINARY_DOUBLE_COL binary_double,\n" + + " DATE_COL date,\n" + + " TIMESTAMP_WITH_3_FRAC_SEC_COL timestamp(3),\n" + + " TIMESTAMP_WITH_LOCAL_TZ timestamp with local time zone,\n" + + " XML_TYPE_COL \"SYS\".\"XMLTYPE\",\n" + + " constraint PK_T_COL primary key (INTEGER_COL)" + + ")"; + + private static final String SINK_CREATE_SQL = "create table %s\n" + "(\n" + " VARCHAR_10_COL varchar2(10),\n" @@ -150,6 +170,7 @@ JdbcCase getJdbcCase() { .catalogSchema(SCHEMA) .catalogTable(CATALOG_TABLE) .createSql(CREATE_SQL) + .sinkCreateSql(SINK_CREATE_SQL) .configFile(CONFIG_FILE) .insertSql(insertSql) .testData(testDataSet) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOceanBaseMysqlIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOceanBaseMysqlIT.java index cb0958c705c..3208473d619 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOceanBaseMysqlIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOceanBaseMysqlIT.java @@ -173,7 +173,8 @@ String createSqlTemplate() { + " `c_integer_unsigned` int(10) unsigned DEFAULT NULL,\n" + " `c_bigint_30` BIGINT(40) unsigned DEFAULT NULL,\n" + " `c_decimal_unsigned_30` DECIMAL(30) unsigned DEFAULT NULL,\n" - + " `c_decimal_30` DECIMAL(30) DEFAULT NULL\n" + + " `c_decimal_30` DECIMAL(30) DEFAULT NULL,\n" + + " UNIQUE KEY (c_int)\n" + ");"; } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcPostgresIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcPostgresIT.java index 34c2082eb3d..6993b99336b 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcPostgresIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcPostgresIT.java @@ -19,7 +19,10 @@ import org.apache.seatunnel.api.table.catalog.Catalog; import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresCatalog; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; @@ -29,6 +32,8 @@ import org.apache.seatunnel.e2e.common.container.TestContainer; import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; +import org.apache.commons.lang3.StringUtils; + import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; @@ -256,6 +261,66 @@ public void startUp() throws Exception { log.info("pg data initialization succeeded. Procedure"); } + @Test + public void testCreateIndex() { + String schema = "public"; + String databaseName = POSTGRESQL_CONTAINER.getDatabaseName(); + TablePath sourceTablePath = TablePath.of(databaseName, "public", "pg_e2e_source_table"); + TablePath targetTablePath = TablePath.of(databaseName, "public", "pg_ide_sink_table_2"); + PostgresCatalog postgresCatalog = + new PostgresCatalog( + DatabaseIdentifier.POSTGRESQL, + POSTGRESQL_CONTAINER.getUsername(), + POSTGRESQL_CONTAINER.getPassword(), + JdbcUrlUtil.getUrlInfo(POSTGRESQL_CONTAINER.getJdbcUrl()), + schema); + postgresCatalog.open(); + + CatalogTable catalogTable = postgresCatalog.getTable(sourceTablePath); + + dropTableWithAssert(postgresCatalog, targetTablePath, true); + // not create index + createIndexOrNot(postgresCatalog, targetTablePath, catalogTable, false); + Assertions.assertFalse(hasIndex(postgresCatalog, targetTablePath)); + + dropTableWithAssert(postgresCatalog, targetTablePath, true); + // create index + createIndexOrNot(postgresCatalog, targetTablePath, catalogTable, true); + Assertions.assertTrue(hasIndex(postgresCatalog, targetTablePath)); + + dropTableWithAssert(postgresCatalog, targetTablePath, true); + + postgresCatalog.close(); + } + + protected boolean hasIndex(Catalog catalog, TablePath targetTablePath) { + TableSchema tableSchema = catalog.getTable(targetTablePath).getTableSchema(); + PrimaryKey primaryKey = tableSchema.getPrimaryKey(); + List constraintKeys = tableSchema.getConstraintKeys(); + if (primaryKey != null && StringUtils.isNotBlank(primaryKey.getPrimaryKey())) { + return true; + } + if (!constraintKeys.isEmpty()) { + return true; + } + return false; + } + + private void dropTableWithAssert( + PostgresCatalog postgresCatalog, TablePath targetTablePath, boolean ignoreIfNotExists) { + postgresCatalog.dropTable(targetTablePath, ignoreIfNotExists); + Assertions.assertFalse(postgresCatalog.tableExists(targetTablePath)); + } + + private void createIndexOrNot( + PostgresCatalog postgresCatalog, + TablePath targetTablePath, + CatalogTable catalogTable, + boolean createIndex) { + postgresCatalog.createTable(targetTablePath, catalogTable, false, createIndex); + Assertions.assertTrue(postgresCatalog.tableExists(targetTablePath)); + } + @TestTemplate public void testAutoGenerateSQL(TestContainer container) throws IOException, InterruptedException { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerIT.java index e82ca5b03ea..de5621899a4 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerIT.java @@ -59,7 +59,6 @@ public class JdbcSqlServerIT extends AbstractJdbcIT { private static final String SQLSERVER_DATABASE = "master"; private static final String SQLSERVER_SCHEMA = "dbo"; private static final String SQLSERVER_CATALOG_DATABASE = "catalog_test"; - private static final int SQLSERVER_CONTAINER_PORT = 1433; private static final String SQLSERVER_URL = "jdbc:sqlserver://" @@ -103,7 +102,8 @@ public class JdbcSqlServerIT extends AbstractJdbcIT { + "\tVARBINARY_MAX_TEST varbinary(MAX) NULL,\n" + "\tVARCHAR_TEST varchar(16) COLLATE Chinese_PRC_CS_AS NULL,\n" + "\tVARCHAR_MAX_TEST varchar(MAX) COLLATE Chinese_PRC_CS_AS DEFAULT NULL NULL,\n" - + "\tXML_TEST xml NULL\n" + + "\tXML_TEST xml NULL,\n" + + "\tCONSTRAINT PK_TEST_INDEX PRIMARY KEY (INT_IDENTITY_TEST)\n" + ");"; private static final String SINK_CREATE_SQL = diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-6/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleLowercaseTableIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-6/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleLowercaseTableIT.java index 717f72e1e9a..73feb9ab571 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-6/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleLowercaseTableIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-6/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleLowercaseTableIT.java @@ -65,6 +65,24 @@ public class JdbcOracleLowercaseTableIT extends AbstractJdbcIT { private static final List CONFIG_FILE = Lists.newArrayList(); private static final String CREATE_SQL = + "create table %s\n" + + "(\n" + + " VARCHAR_10_COL varchar2(10),\n" + + " CHAR_10_COL char(10),\n" + + " CLOB_COL clob,\n" + + " NUMBER_3_SF_2_DP number(3, 2),\n" + + " INTEGER_COL integer,\n" + + " FLOAT_COL float(10),\n" + + " REAL_COL real,\n" + + " BINARY_FLOAT_COL binary_float,\n" + + " BINARY_DOUBLE_COL binary_double,\n" + + " DATE_COL date,\n" + + " TIMESTAMP_WITH_3_FRAC_SEC_COL timestamp(3),\n" + + " TIMESTAMP_WITH_LOCAL_TZ timestamp with local time zone,\n" + + " constraint PK_T_COL1 primary key (INTEGER_COL)" + + ")"; + + private static final String SINK_CREATE_SQL = "create table %s\n" + "(\n" + " VARCHAR_10_COL varchar2(10),\n" @@ -113,6 +131,7 @@ JdbcCase getJdbcCase() { .catalogSchema(SCHEMA) .catalogTable(CATALOG_TABLE) .createSql(CREATE_SQL) + .sinkCreateSql(SINK_CREATE_SQL) .configFile(CONFIG_FILE) .insertSql(insertSql) .testData(testDataSet) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlSaveModeHandlerIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlSaveModeHandlerIT.java index bc1361aa267..a6a1c050c81 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlSaveModeHandlerIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlSaveModeHandlerIT.java @@ -119,7 +119,8 @@ public class JdbcMysqlSaveModeHandlerIT extends AbstractJdbcIT { + " `c_integer_unsigned` int(10) unsigned DEFAULT NULL,\n" + " `c_bigint_30` BIGINT(40) unsigned DEFAULT NULL,\n" + " `c_decimal_unsigned_30` DECIMAL(30) unsigned DEFAULT NULL,\n" - + " `c_decimal_30` DECIMAL(30) DEFAULT NULL\n" + + " `c_decimal_30` DECIMAL(30) DEFAULT NULL,\n" + + " UNIQUE (c_int)\n" + ");"; @Override From 921662722f01fd6e7bffb3b582e7624f21cddfee Mon Sep 17 00:00:00 2001 From: chaos <71205599+chaos-cn@users.noreply.github.com> Date: Fri, 9 Aug 2024 13:53:28 +0800 Subject: [PATCH 040/361] Update ConsoleSinkFactory.java (#7350) Add existing rules to the optionRule --- .../connectors/seatunnel/console/sink/ConsoleSinkFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkFactory.java b/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkFactory.java index 858357d282b..169a281fc19 100644 --- a/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkFactory.java +++ b/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkFactory.java @@ -52,7 +52,7 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { - return OptionRule.builder().build(); + return OptionRule.builder().optional(LOG_PRINT_DATA, LOG_PRINT_DELAY).build(); } @Override From 3130ae089e65bc71fa61784fad7d282f92aeadaf Mon Sep 17 00:00:00 2001 From: xxsc0529 <93303124+xxsc0529@users.noreply.github.com> Date: Fri, 9 Aug 2024 20:02:06 +0800 Subject: [PATCH 041/361] [Fix][Connector-V2][OceanBase] Remove OceanBase catalog's dependency on mysql driver (#7311) --- .../oceanbase/OceanBaseCatalogFactory.java | 5 + .../oceanbase/OceanBaseMySqlCatalog.java | 193 +++++- .../OceanBaseMysqlCreateTableSqlBuilder.java | 271 +++++++++ .../oceanbase/OceanBaseDialectFactory.java | 3 +- .../OceanBaseMySqlTypeConverter.java | 513 ++++++++++++++++ .../oceanbase/OceanBaseMySqlTypeMapper.java | 72 +++ .../oceanbase/OceanBaseMysqlDialect.java | 290 +++++++++ .../OceanBaseMysqlJdbcRowConverter.java | 42 ++ .../dialect/oceanbase/OceanBaseMysqlType.java | 567 ++++++++++++++++++ .../jdbc/utils/JdbcCatalogUtils.java | 2 + .../seatunnel/jdbc/JdbcOceanBaseMysqlIT.java | 8 +- 11 files changed, 1954 insertions(+), 12 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMysqlCreateTableSqlBuilder.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMySqlTypeConverter.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMySqlTypeMapper.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlDialect.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlJdbcRowConverter.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlType.java diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseCatalogFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseCatalogFactory.java index 58dfa5b884a..01d035e1677 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseCatalogFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseCatalogFactory.java @@ -31,6 +31,9 @@ import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import com.google.auto.service.AutoService; import java.util.Optional; @@ -38,6 +41,8 @@ @AutoService(Factory.class) public class OceanBaseCatalogFactory implements CatalogFactory { + private static final Logger log = LoggerFactory.getLogger(OceanBaseCatalogFactory.class); + @Override public String factoryIdentifier() { return DatabaseIdentifier.OCENABASE; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMySqlCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMySqlCatalog.java index 58cdb5c4131..08aa0faea08 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMySqlCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMySqlCatalog.java @@ -17,10 +17,44 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oceanbase; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.exception.CatalogException; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.common.utils.JdbcUrlUtil; -import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.mysql.MySqlCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.utils.CatalogUtils; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oceanbase.OceanBaseMySqlTypeConverter; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oceanbase.OceanBaseMySqlTypeMapper; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oceanbase.OceanBaseMysqlType; -public class OceanBaseMySqlCatalog extends MySqlCatalog { +import com.google.common.base.Preconditions; +import lombok.extern.slf4j.Slf4j; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; + +@Slf4j +public class OceanBaseMySqlCatalog extends AbstractJdbcCatalog { + + private static final String SELECT_COLUMNS_SQL_TEMPLATE = + "SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_SCHEMA = '%s' AND TABLE_NAME ='%s' ORDER BY ORDINAL_POSITION ASC"; + + private static final String SELECT_DATABASE_EXISTS = + "SELECT SCHEMA_NAME FROM information_schema.schemata WHERE SCHEMA_NAME = '%s'"; + + private static final String SELECT_TABLE_EXISTS = + "SELECT TABLE_SCHEMA,TABLE_NAME FROM information_schema.tables WHERE table_schema = '%s' AND table_name = '%s'"; static { SYS_DATABASES.clear(); @@ -32,8 +66,161 @@ public class OceanBaseMySqlCatalog extends MySqlCatalog { SYS_DATABASES.add("SYS"); } + private OceanBaseMySqlTypeConverter typeConverter; + public OceanBaseMySqlCatalog( String catalogName, String username, String pwd, JdbcUrlUtil.UrlInfo urlInfo) { - super(catalogName, username, pwd, urlInfo); + super(catalogName, username, pwd, urlInfo, null); + this.typeConverter = new OceanBaseMySqlTypeConverter(); + } + + @Override + protected String getDatabaseWithConditionSql(String databaseName) { + return String.format(SELECT_DATABASE_EXISTS, databaseName); + } + + @Override + protected String getTableWithConditionSql(TablePath tablePath) { + return String.format( + SELECT_TABLE_EXISTS, tablePath.getDatabaseName(), tablePath.getTableName()); + } + + @Override + protected String getListDatabaseSql() { + return "SHOW DATABASES;"; + } + + @Override + protected String getListTableSql(String databaseName) { + return "SHOW TABLES;"; + } + + @Override + protected String getTableName(ResultSet rs) throws SQLException { + return rs.getString(1); + } + + @Override + protected String getTableName(TablePath tablePath) { + return tablePath.getTableName(); + } + + @Override + protected String getSelectColumnsSql(TablePath tablePath) { + return String.format( + SELECT_COLUMNS_SQL_TEMPLATE, tablePath.getDatabaseName(), tablePath.getTableName()); + } + + @Override + protected TableIdentifier getTableIdentifier(TablePath tablePath) { + return TableIdentifier.of( + catalogName, tablePath.getDatabaseName(), tablePath.getTableName()); + } + + @Override + protected List getConstraintKeys(DatabaseMetaData metaData, TablePath tablePath) + throws SQLException { + List indexList = + super.getConstraintKeys( + metaData, + tablePath.getDatabaseName(), + tablePath.getSchemaName(), + tablePath.getTableName()); + for (Iterator it = indexList.iterator(); it.hasNext(); ) { + ConstraintKey index = it.next(); + if (ConstraintKey.ConstraintType.UNIQUE_KEY.equals(index.getConstraintType()) + && "PRIMARY".equals(index.getConstraintName())) { + it.remove(); + } + } + return indexList; + } + + @Override + protected Column buildColumn(ResultSet resultSet) throws SQLException { + String columnName = resultSet.getString("COLUMN_NAME"); + // e.g. tinyint(1) unsigned + String columnType = resultSet.getString("COLUMN_TYPE"); + // e.g. tinyint + String dataType = resultSet.getString("DATA_TYPE").toUpperCase(); + String comment = resultSet.getString("COLUMN_COMMENT"); + Object defaultValue = resultSet.getObject("COLUMN_DEFAULT"); + String isNullableStr = resultSet.getString("IS_NULLABLE"); + boolean isNullable = isNullableStr.equals("YES"); + // e.g. `decimal(10, 2)` is 10 + long numberPrecision = resultSet.getInt("NUMERIC_PRECISION"); + // e.g. `decimal(10, 2)` is 2 + int numberScale = resultSet.getInt("NUMERIC_SCALE"); + // e.g. `varchar(10)` is 40 + long charOctetLength = resultSet.getLong("CHARACTER_OCTET_LENGTH"); + // e.g. `timestamp(3)` is 3 + // int timePrecision = + // MySqlVersion.V_5_5.equals(version) ? 0 : + // resultSet.getInt("DATETIME_PRECISION"); + int timePrecision = resultSet.getInt("DATETIME_PRECISION"); + Preconditions.checkArgument(!(numberPrecision > 0 && charOctetLength > 0)); + Preconditions.checkArgument(!(numberScale > 0 && timePrecision > 0)); + + OceanBaseMysqlType oceanbaseMysqlType = OceanBaseMysqlType.getByName(columnType); + boolean unsigned = columnType.toLowerCase(Locale.ROOT).contains("unsigned"); + + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name(columnName) + .columnType(columnType) + .dataType(dataType) + .nativeType(oceanbaseMysqlType) + .unsigned(unsigned) + .length(Math.max(charOctetLength, numberPrecision)) + .precision(numberPrecision) + .scale(Math.max(numberScale, timePrecision)) + .nullable(isNullable) + .defaultValue(defaultValue) + .comment(comment) + .build(); + return typeConverter.convert(typeDefine); + } + + @Override + protected String getCreateTableSql(TablePath tablePath, CatalogTable table) { + return OceanBaseMysqlCreateTableSqlBuilder.builder(tablePath, table, typeConverter) + .build(table.getCatalogName()); + } + + @Override + protected String getDropTableSql(TablePath tablePath) { + return String.format( + "DROP TABLE `%s`.`%s`;", tablePath.getDatabaseName(), tablePath.getTableName()); + } + + @Override + protected String getCreateDatabaseSql(String databaseName) { + return String.format("CREATE DATABASE `%s`;", databaseName); + } + + @Override + protected String getDropDatabaseSql(String databaseName) { + return String.format("DROP DATABASE `%s`;", databaseName); + } + + @Override + public CatalogTable getTable(String sqlQuery) throws SQLException { + Connection defaultConnection = getConnection(defaultUrl); + Statement statement = defaultConnection.createStatement(); + ResultSetMetaData metaData = statement.executeQuery(sqlQuery).getMetaData(); + return CatalogUtils.getCatalogTable( + metaData, new OceanBaseMySqlTypeMapper(typeConverter), sqlQuery); + } + + @Override + protected String getTruncateTableSql(TablePath tablePath) throws CatalogException { + return String.format( + "TRUNCATE TABLE `%s`.`%s`;", tablePath.getDatabaseName(), tablePath.getTableName()); + } + + public String getExistDataSql(TablePath tablePath) { + return String.format( + "SELECT * FROM `%s`.`%s` LIMIT 1;", + tablePath.getDatabaseName(), tablePath.getTableName()); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMysqlCreateTableSqlBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMysqlCreateTableSqlBuilder.java new file mode 100644 index 00000000000..bc3413dbd82 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMysqlCreateTableSqlBuilder.java @@ -0,0 +1,271 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oceanbase; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.utils.CatalogUtils; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oceanbase.OceanBaseMySqlTypeConverter; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oceanbase.OceanBaseMysqlType; + +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.StringUtils; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkArgument; +import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkNotNull; + +public class OceanBaseMysqlCreateTableSqlBuilder { + + private final String tableName; + private List columns; + + private String comment; + + private String engine; + private String charset; + private String collate; + + private PrimaryKey primaryKey; + + private List constraintKeys; + + private String fieldIde; + + private final OceanBaseMySqlTypeConverter typeConverter; + + private OceanBaseMysqlCreateTableSqlBuilder( + String tableName, OceanBaseMySqlTypeConverter typeConverter) { + checkNotNull(tableName, "tableName must not be null"); + this.tableName = tableName; + this.typeConverter = typeConverter; + } + + public static OceanBaseMysqlCreateTableSqlBuilder builder( + TablePath tablePath, + CatalogTable catalogTable, + OceanBaseMySqlTypeConverter typeConverter) { + checkNotNull(tablePath, "tablePath must not be null"); + checkNotNull(catalogTable, "catalogTable must not be null"); + + TableSchema tableSchema = catalogTable.getTableSchema(); + checkNotNull(tableSchema, "tableSchema must not be null"); + + return new OceanBaseMysqlCreateTableSqlBuilder(tablePath.getTableName(), typeConverter) + .comment(catalogTable.getComment()) + // todo: set charset and collate + .engine(null) + .charset(null) + .primaryKey(tableSchema.getPrimaryKey()) + .constraintKeys(tableSchema.getConstraintKeys()) + .addColumn(tableSchema.getColumns()) + .fieldIde(catalogTable.getOptions().get("fieldIde")); + } + + public OceanBaseMysqlCreateTableSqlBuilder addColumn(List columns) { + checkArgument(CollectionUtils.isNotEmpty(columns), "columns must not be empty"); + this.columns = columns; + return this; + } + + public OceanBaseMysqlCreateTableSqlBuilder primaryKey(PrimaryKey primaryKey) { + this.primaryKey = primaryKey; + return this; + } + + public OceanBaseMysqlCreateTableSqlBuilder fieldIde(String fieldIde) { + this.fieldIde = fieldIde; + return this; + } + + public OceanBaseMysqlCreateTableSqlBuilder constraintKeys(List constraintKeys) { + this.constraintKeys = constraintKeys; + return this; + } + + public OceanBaseMysqlCreateTableSqlBuilder engine(String engine) { + this.engine = engine; + return this; + } + + public OceanBaseMysqlCreateTableSqlBuilder charset(String charset) { + this.charset = charset; + return this; + } + + public OceanBaseMysqlCreateTableSqlBuilder collate(String collate) { + this.collate = collate; + return this; + } + + public OceanBaseMysqlCreateTableSqlBuilder comment(String comment) { + this.comment = comment; + return this; + } + + public String build(String catalogName) { + List sqls = new ArrayList<>(); + sqls.add( + String.format( + "CREATE TABLE %s (\n%s\n)", + CatalogUtils.quoteIdentifier(tableName, fieldIde, "`"), + buildColumnsIdentifySql(catalogName))); + if (engine != null) { + sqls.add("ENGINE = " + engine); + } + if (charset != null) { + sqls.add("DEFAULT CHARSET = " + charset); + } + if (collate != null) { + sqls.add("COLLATE = " + collate); + } + if (comment != null) { + sqls.add("COMMENT = '" + comment + "'"); + } + return String.join(" ", sqls) + ";"; + } + + private String buildColumnsIdentifySql(String catalogName) { + List columnSqls = new ArrayList<>(); + Map columnTypeMap = new HashMap<>(); + for (Column column : columns) { + columnSqls.add("\t" + buildColumnIdentifySql(column, catalogName, columnTypeMap)); + } + if (primaryKey != null) { + columnSqls.add("\t" + buildPrimaryKeySql()); + } + if (CollectionUtils.isNotEmpty(constraintKeys)) { + for (ConstraintKey constraintKey : constraintKeys) { + if (StringUtils.isBlank(constraintKey.getConstraintName())) { + continue; + } + String constraintKeyStr = buildConstraintKeySql(constraintKey, columnTypeMap); + if (StringUtils.isNotBlank(constraintKeyStr)) { + columnSqls.add("\t" + constraintKeyStr); + } + } + } + return String.join(", \n", columnSqls); + } + + private String buildColumnIdentifySql( + Column column, String catalogName, Map columnTypeMap) { + final List columnSqls = new ArrayList<>(); + columnSqls.add(CatalogUtils.quoteIdentifier(column.getName(), fieldIde, "`")); + String type; + if ((SqlType.TIME.equals(column.getDataType().getSqlType()) + || SqlType.TIMESTAMP.equals(column.getDataType().getSqlType())) + && column.getScale() != null) { + BasicTypeDefine typeDefine = typeConverter.reconvert(column); + type = typeDefine.getColumnType(); + } else if (StringUtils.equals(catalogName, DatabaseIdentifier.MYSQL) + && StringUtils.isNotBlank(column.getSourceType())) { + type = column.getSourceType(); + } else { + BasicTypeDefine typeDefine = typeConverter.reconvert(column); + type = typeDefine.getColumnType(); + } + columnSqls.add(type); + columnTypeMap.put(column.getName(), type); + // nullable + if (column.isNullable()) { + columnSqls.add("NULL"); + } else { + columnSqls.add("NOT NULL"); + } + + if (column.getComment() != null) { + columnSqls.add( + "COMMENT '" + + column.getComment().replace("'", "''").replace("\\", "\\\\") + + "'"); + } + + return String.join(" ", columnSqls); + } + + private String buildPrimaryKeySql() { + String key = + primaryKey.getColumnNames().stream() + .map(columnName -> "`" + columnName + "`") + .collect(Collectors.joining(", ")); + // add sort type + return String.format("PRIMARY KEY (%s)", CatalogUtils.quoteIdentifier(key, fieldIde)); + } + + private String buildConstraintKeySql( + ConstraintKey constraintKey, Map columnTypeMap) { + ConstraintKey.ConstraintType constraintType = constraintKey.getConstraintType(); + String indexColumns = + constraintKey.getColumnNames().stream() + .map( + constraintKeyColumn -> { + String columnName = constraintKeyColumn.getColumnName(); + boolean withLength = false; + if (columnTypeMap.containsKey(columnName)) { + String columnType = columnTypeMap.get(columnName); + if (columnType.endsWith("BLOB") + || columnType.endsWith("TEXT")) { + withLength = true; + } + } + if (constraintKeyColumn.getSortType() == null) { + return String.format( + "`%s`%s", + CatalogUtils.getFieldIde(columnName, fieldIde), + withLength ? "(255)" : ""); + } + return String.format( + "`%s`%s %s", + CatalogUtils.getFieldIde(columnName, fieldIde), + withLength ? "(255)" : "", + constraintKeyColumn.getSortType().name()); + }) + .collect(Collectors.joining(", ")); + String keyName = null; + switch (constraintType) { + case INDEX_KEY: + keyName = "KEY"; + break; + case UNIQUE_KEY: + keyName = "UNIQUE KEY"; + break; + case FOREIGN_KEY: + keyName = "FOREIGN KEY"; + // todo: + break; + default: + throw new UnsupportedOperationException( + "Unsupported constraint type: " + constraintType); + } + return String.format( + "%s `%s` (%s)", keyName, constraintKey.getConstraintName(), indexColumns); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseDialectFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseDialectFactory.java index b3a456870cc..d25d48b4f2c 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseDialectFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseDialectFactory.java @@ -19,7 +19,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectFactory; -import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.mysql.MysqlDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oracle.OracleDialect; import com.google.auto.service.AutoService; @@ -44,6 +43,6 @@ public JdbcDialect create(@Nonnull String compatibleMode, String fieldIde) { if ("oracle".equalsIgnoreCase(compatibleMode)) { return new OracleDialect(); } - return new MysqlDialect(); + return new OceanBaseMysqlDialect(); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMySqlTypeConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMySqlTypeConverter.java new file mode 100644 index 00000000000..4e9fa04d0d3 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMySqlTypeConverter.java @@ -0,0 +1,513 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oceanbase; + +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.api.table.converter.TypeConverter; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; +import org.apache.seatunnel.common.exception.CommonError; +import org.apache.seatunnel.connectors.seatunnel.common.source.TypeDefineUtils; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; + +import com.google.auto.service.AutoService; +import com.google.common.base.Preconditions; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +@AutoService(TypeConverter.class) +public class OceanBaseMySqlTypeConverter + implements TypeConverter> { + + // ============================data types===================== + static final String MYSQL_NULL = "NULL"; + static final String MYSQL_BIT = "BIT"; + + // -------------------------number---------------------------- + static final String MYSQL_TINYINT = "TINYINT"; + static final String MYSQL_TINYINT_UNSIGNED = "TINYINT UNSIGNED"; + static final String MYSQL_SMALLINT = "SMALLINT"; + static final String MYSQL_SMALLINT_UNSIGNED = "SMALLINT UNSIGNED"; + static final String MYSQL_MEDIUMINT = "MEDIUMINT"; + static final String MYSQL_MEDIUMINT_UNSIGNED = "MEDIUMINT UNSIGNED"; + static final String MYSQL_INT = "INT"; + static final String MYSQL_INT_UNSIGNED = "INT UNSIGNED"; + static final String MYSQL_INTEGER = "INTEGER"; + static final String MYSQL_INTEGER_UNSIGNED = "INTEGER UNSIGNED"; + static final String MYSQL_BIGINT = "BIGINT"; + static final String MYSQL_BIGINT_UNSIGNED = "BIGINT UNSIGNED"; + static final String MYSQL_DECIMAL = "DECIMAL"; + static final String MYSQL_DECIMAL_UNSIGNED = "DECIMAL UNSIGNED"; + static final String MYSQL_FLOAT = "FLOAT"; + static final String MYSQL_FLOAT_UNSIGNED = "FLOAT UNSIGNED"; + static final String MYSQL_DOUBLE = "DOUBLE"; + static final String MYSQL_DOUBLE_UNSIGNED = "DOUBLE UNSIGNED"; + + // -------------------------string---------------------------- + public static final String MYSQL_CHAR = "CHAR"; + public static final String MYSQL_VARCHAR = "VARCHAR"; + static final String MYSQL_TINYTEXT = "TINYTEXT"; + static final String MYSQL_MEDIUMTEXT = "MEDIUMTEXT"; + static final String MYSQL_TEXT = "TEXT"; + static final String MYSQL_LONGTEXT = "LONGTEXT"; + static final String MYSQL_JSON = "JSON"; + static final String MYSQL_ENUM = "ENUM"; + + // ------------------------------time------------------------- + static final String MYSQL_DATE = "DATE"; + public static final String MYSQL_DATETIME = "DATETIME"; + public static final String MYSQL_TIME = "TIME"; + public static final String MYSQL_TIMESTAMP = "TIMESTAMP"; + static final String MYSQL_YEAR = "YEAR"; + + // ------------------------------blob------------------------- + static final String MYSQL_TINYBLOB = "TINYBLOB"; + static final String MYSQL_MEDIUMBLOB = "MEDIUMBLOB"; + static final String MYSQL_BLOB = "BLOB"; + static final String MYSQL_LONGBLOB = "LONGBLOB"; + static final String MYSQL_BINARY = "BINARY"; + static final String MYSQL_VARBINARY = "VARBINARY"; + static final String MYSQL_GEOMETRY = "GEOMETRY"; + + public static final int DEFAULT_PRECISION = 38; + public static final int MAX_PRECISION = 65; + public static final int DEFAULT_SCALE = 18; + public static final int MAX_SCALE = 30; + public static final int MAX_TIME_SCALE = 6; + public static final int MAX_TIMESTAMP_SCALE = 6; + public static final long POWER_2_8 = (long) Math.pow(2, 8); + public static final long POWER_2_16 = (long) Math.pow(2, 16); + public static final long POWER_2_24 = (long) Math.pow(2, 24); + public static final long POWER_2_32 = (long) Math.pow(2, 32); + public static final long MAX_VARBINARY_LENGTH = POWER_2_16 - 4; + + @Override + public String identifier() { + return DatabaseIdentifier.OCENABASE; + } + + @Override + public Column convert(BasicTypeDefine typeDefine) { + PhysicalColumn.PhysicalColumnBuilder builder = + PhysicalColumn.builder() + .name(typeDefine.getName()) + .sourceType(typeDefine.getColumnType()) + .nullable(typeDefine.isNullable()) + .defaultValue(typeDefine.getDefaultValue()) + .comment(typeDefine.getComment()); + + String mysqlDataType = typeDefine.getDataType().toUpperCase(); + if (typeDefine.isUnsigned() && !(mysqlDataType.endsWith(" UNSIGNED"))) { + mysqlDataType = mysqlDataType + " UNSIGNED"; + } + switch (mysqlDataType) { + case MYSQL_NULL: + builder.dataType(BasicType.VOID_TYPE); + break; + case MYSQL_BIT: + if (typeDefine.getLength() == null || typeDefine.getLength() <= 0) { + builder.dataType(BasicType.BOOLEAN_TYPE); + } else if (typeDefine.getLength() == 1) { + builder.dataType(BasicType.BOOLEAN_TYPE); + } else { + builder.dataType(PrimitiveByteArrayType.INSTANCE); + // BIT(M) -> BYTE(M/8) + long byteLength = typeDefine.getLength() / 8; + byteLength += typeDefine.getLength() % 8 > 0 ? 1 : 0; + builder.columnLength(byteLength); + } + break; + case MYSQL_TINYINT: + if (typeDefine.getColumnType().equalsIgnoreCase("tinyint(1)")) { + builder.dataType(BasicType.BOOLEAN_TYPE); + } else { + builder.dataType(BasicType.BYTE_TYPE); + } + break; + case MYSQL_TINYINT_UNSIGNED: + case MYSQL_SMALLINT: + builder.dataType(BasicType.SHORT_TYPE); + break; + case MYSQL_SMALLINT_UNSIGNED: + case MYSQL_MEDIUMINT: + case MYSQL_MEDIUMINT_UNSIGNED: + case MYSQL_INT: + case MYSQL_INTEGER: + case MYSQL_YEAR: + builder.dataType(BasicType.INT_TYPE); + break; + case MYSQL_INT_UNSIGNED: + case MYSQL_INTEGER_UNSIGNED: + case MYSQL_BIGINT: + builder.dataType(BasicType.LONG_TYPE); + break; + case MYSQL_BIGINT_UNSIGNED: + DecimalType intDecimalType = new DecimalType(20, 0); + builder.dataType(intDecimalType); + builder.columnLength(Long.valueOf(intDecimalType.getPrecision())); + builder.scale(intDecimalType.getScale()); + break; + case MYSQL_FLOAT: + builder.dataType(BasicType.FLOAT_TYPE); + break; + case MYSQL_FLOAT_UNSIGNED: + log.warn("{} will probably cause value overflow.", MYSQL_FLOAT_UNSIGNED); + builder.dataType(BasicType.FLOAT_TYPE); + break; + case MYSQL_DOUBLE: + builder.dataType(BasicType.DOUBLE_TYPE); + break; + case MYSQL_DOUBLE_UNSIGNED: + log.warn("{} will probably cause value overflow.", MYSQL_DOUBLE_UNSIGNED); + builder.dataType(BasicType.DOUBLE_TYPE); + break; + case MYSQL_DECIMAL: + Preconditions.checkArgument(typeDefine.getPrecision() > 0); + + DecimalType decimalType; + if (typeDefine.getPrecision() > DEFAULT_PRECISION) { + log.warn("{} will probably cause value overflow.", MYSQL_DECIMAL); + decimalType = new DecimalType(DEFAULT_PRECISION, DEFAULT_SCALE); + } else { + decimalType = + new DecimalType( + typeDefine.getPrecision().intValue(), + typeDefine.getScale() == null + ? 0 + : typeDefine.getScale().intValue()); + } + builder.dataType(decimalType); + builder.columnLength(Long.valueOf(decimalType.getPrecision())); + builder.scale(decimalType.getScale()); + break; + case MYSQL_DECIMAL_UNSIGNED: + Preconditions.checkArgument(typeDefine.getPrecision() > 0); + + log.warn("{} will probably cause value overflow.", MYSQL_DECIMAL_UNSIGNED); + DecimalType decimalUnsignedType = + new DecimalType( + typeDefine.getPrecision().intValue() + 1, + typeDefine.getScale() == null + ? 0 + : typeDefine.getScale().intValue()); + builder.dataType(decimalUnsignedType); + builder.columnLength(Long.valueOf(decimalUnsignedType.getPrecision())); + builder.scale(decimalUnsignedType.getScale()); + break; + case MYSQL_ENUM: + builder.dataType(BasicType.STRING_TYPE); + if (typeDefine.getLength() == null || typeDefine.getLength() <= 0) { + builder.columnLength(100L); + } else { + builder.columnLength(typeDefine.getLength()); + } + break; + case MYSQL_CHAR: + case MYSQL_VARCHAR: + if (typeDefine.getLength() == null || typeDefine.getLength() <= 0) { + builder.columnLength(TypeDefineUtils.charTo4ByteLength(1L)); + } else { + builder.columnLength(typeDefine.getLength()); + } + builder.dataType(BasicType.STRING_TYPE); + break; + case MYSQL_TINYTEXT: + builder.dataType(BasicType.STRING_TYPE); + builder.columnLength(POWER_2_8 - 1); + break; + case MYSQL_TEXT: + builder.dataType(BasicType.STRING_TYPE); + builder.columnLength(POWER_2_16 - 1); + break; + case MYSQL_MEDIUMTEXT: + builder.dataType(BasicType.STRING_TYPE); + builder.columnLength(POWER_2_24 - 1); + break; + case MYSQL_LONGTEXT: + builder.dataType(BasicType.STRING_TYPE); + builder.columnLength(POWER_2_32 - 1); + break; + case MYSQL_JSON: + builder.dataType(BasicType.STRING_TYPE); + break; + case MYSQL_BINARY: + case MYSQL_VARBINARY: + if (typeDefine.getLength() == null || typeDefine.getLength() <= 0) { + builder.columnLength(1L); + } else { + builder.columnLength(typeDefine.getLength()); + } + builder.dataType(PrimitiveByteArrayType.INSTANCE); + break; + case MYSQL_TINYBLOB: + builder.dataType(PrimitiveByteArrayType.INSTANCE); + builder.columnLength(POWER_2_8 - 1); + break; + case MYSQL_BLOB: + builder.dataType(PrimitiveByteArrayType.INSTANCE); + builder.columnLength(POWER_2_16 - 1); + break; + case MYSQL_MEDIUMBLOB: + builder.dataType(PrimitiveByteArrayType.INSTANCE); + builder.columnLength(POWER_2_24 - 1); + break; + case MYSQL_LONGBLOB: + builder.dataType(PrimitiveByteArrayType.INSTANCE); + builder.columnLength(POWER_2_32 - 1); + break; + case MYSQL_GEOMETRY: + builder.dataType(PrimitiveByteArrayType.INSTANCE); + break; + case MYSQL_DATE: + builder.dataType(LocalTimeType.LOCAL_DATE_TYPE); + break; + case MYSQL_TIME: + builder.dataType(LocalTimeType.LOCAL_TIME_TYPE); + builder.scale(typeDefine.getScale()); + break; + case MYSQL_DATETIME: + case MYSQL_TIMESTAMP: + builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); + builder.scale(typeDefine.getScale()); + break; + default: + throw CommonError.convertToSeaTunnelTypeError( + DatabaseIdentifier.OCENABASE, mysqlDataType, typeDefine.getName()); + } + return builder.build(); + } + + @Override + public BasicTypeDefine reconvert(Column column) { + BasicTypeDefine.BasicTypeDefineBuilder builder = + BasicTypeDefine.builder() + .name(column.getName()) + .nullable(column.isNullable()) + .comment(column.getComment()) + .defaultValue(column.getDefaultValue()); + switch (column.getDataType().getSqlType()) { + case NULL: + builder.nativeType(OceanBaseMysqlType.NULL); + builder.columnType(MYSQL_NULL); + builder.dataType(MYSQL_NULL); + break; + case BOOLEAN: + builder.nativeType(OceanBaseMysqlType.BOOLEAN); + builder.columnType(String.format("%s(%s)", MYSQL_TINYINT, 1)); + builder.dataType(MYSQL_TINYINT); + builder.length(1L); + break; + case TINYINT: + builder.nativeType(OceanBaseMysqlType.TINYINT); + builder.columnType(MYSQL_TINYINT); + builder.dataType(MYSQL_TINYINT); + break; + case SMALLINT: + builder.nativeType(OceanBaseMysqlType.SMALLINT); + builder.columnType(MYSQL_SMALLINT); + builder.dataType(MYSQL_SMALLINT); + break; + case INT: + builder.nativeType(OceanBaseMysqlType.INT); + builder.columnType(MYSQL_INT); + builder.dataType(MYSQL_INT); + break; + case BIGINT: + builder.nativeType(OceanBaseMysqlType.BIGINT); + builder.columnType(MYSQL_BIGINT); + builder.dataType(MYSQL_BIGINT); + break; + case FLOAT: + builder.nativeType(OceanBaseMysqlType.FLOAT); + builder.columnType(MYSQL_FLOAT); + builder.dataType(MYSQL_FLOAT); + break; + case DOUBLE: + builder.nativeType(OceanBaseMysqlType.DOUBLE); + builder.columnType(MYSQL_DOUBLE); + builder.dataType(MYSQL_DOUBLE); + break; + case DECIMAL: + DecimalType decimalType = (DecimalType) column.getDataType(); + long precision = decimalType.getPrecision(); + int scale = decimalType.getScale(); + if (precision <= 0) { + precision = DEFAULT_PRECISION; + scale = DEFAULT_SCALE; + log.warn( + "The decimal column {} type decimal({},{}) is out of range, " + + "which is precision less than 0, " + + "it will be converted to decimal({},{})", + column.getName(), + decimalType.getPrecision(), + decimalType.getScale(), + precision, + scale); + } else if (precision > MAX_PRECISION) { + scale = (int) Math.max(0, scale - (precision - MAX_PRECISION)); + precision = MAX_PRECISION; + log.warn( + "The decimal column {} type decimal({},{}) is out of range, " + + "which exceeds the maximum precision of {}, " + + "it will be converted to decimal({},{})", + column.getName(), + decimalType.getPrecision(), + decimalType.getScale(), + MAX_PRECISION, + precision, + scale); + } + if (scale < 0) { + scale = 0; + log.warn( + "The decimal column {} type decimal({},{}) is out of range, " + + "which is scale less than 0, " + + "it will be converted to decimal({},{})", + column.getName(), + decimalType.getPrecision(), + decimalType.getScale(), + precision, + scale); + } else if (scale > MAX_SCALE) { + scale = MAX_SCALE; + log.warn( + "The decimal column {} type decimal({},{}) is out of range, " + + "which exceeds the maximum scale of {}, " + + "it will be converted to decimal({},{})", + column.getName(), + decimalType.getPrecision(), + decimalType.getScale(), + MAX_SCALE, + precision, + scale); + } + + builder.nativeType(OceanBaseMysqlType.DECIMAL); + builder.columnType(String.format("%s(%s,%s)", MYSQL_DECIMAL, precision, scale)); + builder.dataType(MYSQL_DECIMAL); + builder.precision(precision); + builder.scale(scale); + break; + case BYTES: + if (column.getColumnLength() == null || column.getColumnLength() <= 0) { + builder.nativeType(OceanBaseMysqlType.VARBINARY); + builder.columnType( + String.format("%s(%s)", MYSQL_VARBINARY, MAX_VARBINARY_LENGTH / 2)); + builder.dataType(MYSQL_VARBINARY); + } else if (column.getColumnLength() < MAX_VARBINARY_LENGTH) { + builder.nativeType(OceanBaseMysqlType.VARBINARY); + builder.columnType( + String.format("%s(%s)", MYSQL_VARBINARY, column.getColumnLength())); + builder.dataType(MYSQL_VARBINARY); + } else if (column.getColumnLength() < POWER_2_24) { + builder.nativeType(OceanBaseMysqlType.MEDIUMBLOB); + builder.columnType(MYSQL_MEDIUMBLOB); + builder.dataType(MYSQL_MEDIUMBLOB); + } else { + builder.nativeType(OceanBaseMysqlType.LONGBLOB); + builder.columnType(MYSQL_LONGBLOB); + builder.dataType(MYSQL_LONGBLOB); + } + break; + case STRING: + if (column.getColumnLength() == null || column.getColumnLength() <= 0) { + builder.nativeType(OceanBaseMysqlType.LONGTEXT); + builder.columnType(MYSQL_LONGTEXT); + builder.dataType(MYSQL_LONGTEXT); + } else if (column.getColumnLength() < POWER_2_8) { + builder.nativeType(OceanBaseMysqlType.VARCHAR); + builder.columnType( + String.format("%s(%s)", MYSQL_VARCHAR, column.getColumnLength())); + builder.dataType(MYSQL_VARCHAR); + } else if (column.getColumnLength() < POWER_2_16) { + builder.nativeType(OceanBaseMysqlType.TEXT); + builder.columnType(MYSQL_TEXT); + builder.dataType(MYSQL_TEXT); + } else if (column.getColumnLength() < POWER_2_24) { + builder.nativeType(OceanBaseMysqlType.MEDIUMTEXT); + builder.columnType(MYSQL_MEDIUMTEXT); + builder.dataType(MYSQL_MEDIUMTEXT); + } else { + builder.nativeType(OceanBaseMysqlType.LONGTEXT); + builder.columnType(MYSQL_LONGTEXT); + builder.dataType(MYSQL_LONGTEXT); + } + break; + case DATE: + builder.nativeType(OceanBaseMysqlType.DATE); + builder.columnType(MYSQL_DATE); + builder.dataType(MYSQL_DATE); + break; + case TIME: + builder.nativeType(OceanBaseMysqlType.TIME); + builder.dataType(MYSQL_TIME); + if (column.getScale() != null && column.getScale() > 0) { + int timeScale = column.getScale(); + if (timeScale > MAX_TIME_SCALE) { + timeScale = MAX_TIME_SCALE; + log.warn( + "The time column {} type time({}) is out of range, " + + "which exceeds the maximum scale of {}, " + + "it will be converted to time({})", + column.getName(), + column.getScale(), + MAX_SCALE, + timeScale); + } + builder.columnType(String.format("%s(%s)", MYSQL_TIME, timeScale)); + builder.scale(timeScale); + } else { + builder.columnType(MYSQL_TIME); + } + break; + case TIMESTAMP: + builder.nativeType(OceanBaseMysqlType.DATETIME); + builder.dataType(MYSQL_DATETIME); + if (column.getScale() != null && column.getScale() > 0) { + int timestampScale = column.getScale(); + if (timestampScale > MAX_TIMESTAMP_SCALE) { + timestampScale = MAX_TIMESTAMP_SCALE; + log.warn( + "The timestamp column {} type timestamp({}) is out of range, " + + "which exceeds the maximum scale of {}, " + + "it will be converted to timestamp({})", + column.getName(), + column.getScale(), + MAX_TIMESTAMP_SCALE, + timestampScale); + } + builder.columnType(String.format("%s(%s)", MYSQL_DATETIME, timestampScale)); + builder.scale(timestampScale); + } else { + builder.columnType(MYSQL_DATETIME); + } + break; + default: + throw CommonError.convertToConnectorTypeError( + DatabaseIdentifier.OCENABASE, + column.getDataType().getSqlType().name(), + column.getName()); + } + + return builder.build(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMySqlTypeMapper.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMySqlTypeMapper.java new file mode 100644 index 00000000000..e4d6e8b9739 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMySqlTypeMapper.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oceanbase; + +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.connectors.seatunnel.common.source.TypeDefineUtils; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; + +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.util.Arrays; + +public class OceanBaseMySqlTypeMapper implements JdbcDialectTypeMapper { + + private OceanBaseMySqlTypeConverter typeConverter; + + public OceanBaseMySqlTypeMapper() { + this.typeConverter = new OceanBaseMySqlTypeConverter(); + } + + public OceanBaseMySqlTypeMapper(OceanBaseMySqlTypeConverter typeConverter) { + this.typeConverter = typeConverter; + } + + @Override + public Column mappingColumn(BasicTypeDefine typeDefine) { + return typeConverter.convert(typeDefine); + } + + @Override + public Column mappingColumn(ResultSetMetaData metadata, int colIndex) throws SQLException { + String columnName = metadata.getColumnLabel(colIndex); + // e.g. tinyint unsigned + String nativeType = metadata.getColumnTypeName(colIndex); + int isNullable = metadata.isNullable(colIndex); + int precision = metadata.getPrecision(colIndex); + int scale = metadata.getScale(colIndex); + + if (Arrays.asList("CHAR", "VARCHAR", "ENUM").contains(nativeType)) { + long octetLength = TypeDefineUtils.charTo4ByteLength((long) precision); + precision = (int) Math.max(precision, octetLength); + } + + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name(columnName) + .columnType(nativeType) + .dataType(nativeType) + .nullable(isNullable == ResultSetMetaData.columnNullable) + .length((long) precision) + .precision((long) precision) + .scale(scale) + .build(); + return mappingColumn(typeDefine); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlDialect.java new file mode 100644 index 00000000000..83d3220b129 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlDialect.java @@ -0,0 +1,290 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oceanbase; + +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.api.table.event.AlterTableColumnEvent; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.JdbcRowConverter; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.SQLUtils; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.dialectenum.FieldIdeEnum; +import org.apache.seatunnel.connectors.seatunnel.jdbc.source.JdbcSourceTable; +import org.apache.seatunnel.connectors.seatunnel.jdbc.utils.MysqlDefaultValueUtils; + +import org.apache.commons.lang3.StringUtils; + +import lombok.extern.slf4j.Slf4j; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +@Slf4j +public class OceanBaseMysqlDialect implements JdbcDialect { + + private static final List NOT_SUPPORTED_DEFAULT_VALUES = + Arrays.asList( + OceanBaseMysqlType.BLOB, + OceanBaseMysqlType.TEXT, + OceanBaseMysqlType.JSON, + OceanBaseMysqlType.GEOMETRY); + + public String fieldIde = FieldIdeEnum.ORIGINAL.getValue(); + + public OceanBaseMysqlDialect() {} + + public OceanBaseMysqlDialect(String fieldIde) { + this.fieldIde = fieldIde; + } + + @Override + public String dialectName() { + return DatabaseIdentifier.OCENABASE; + } + + @Override + public JdbcRowConverter getRowConverter() { + return new OceanBaseMysqlJdbcRowConverter(); + } + + @Override + public JdbcDialectTypeMapper getJdbcDialectTypeMapper() { + return new OceanBaseMySqlTypeMapper(); + } + + @Override + public String quoteIdentifier(String identifier) { + return "`" + getFieldIde(identifier, fieldIde) + "`"; + } + + @Override + public String quoteDatabaseIdentifier(String identifier) { + return "`" + identifier + "`"; + } + + @Override + public Optional getUpsertStatement( + String database, String tableName, String[] fieldNames, String[] uniqueKeyFields) { + String updateClause = + Arrays.stream(fieldNames) + .map( + fieldName -> + quoteIdentifier(fieldName) + + "=VALUES(" + + quoteIdentifier(fieldName) + + ")") + .collect(Collectors.joining(", ")); + String upsertSQL = + getInsertIntoStatement(database, tableName, fieldNames) + + " ON DUPLICATE KEY UPDATE " + + updateClause; + return Optional.of(upsertSQL); + } + + @Override + public PreparedStatement creatPreparedStatement( + Connection connection, String queryTemplate, int fetchSize) throws SQLException { + PreparedStatement statement = + connection.prepareStatement( + queryTemplate, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY); + statement.setFetchSize(Integer.MIN_VALUE); + return statement; + } + + @Override + public String extractTableName(TablePath tablePath) { + return tablePath.getTableName(); + } + + @Override + public Map defaultParameter() { + HashMap map = new HashMap<>(); + map.put("rewriteBatchedStatements", "true"); + return map; + } + + @Override + public TablePath parse(String tablePath) { + return TablePath.of(tablePath, false); + } + + @Override + public Object[] sampleDataFromColumn( + Connection connection, + JdbcSourceTable table, + String columnName, + int samplingRate, + int fetchSize) + throws Exception { + String sampleQuery; + if (StringUtils.isNotBlank(table.getQuery())) { + sampleQuery = + String.format( + "SELECT %s FROM (%s) AS T", + quoteIdentifier(columnName), table.getQuery()); + } else { + sampleQuery = + String.format( + "SELECT %s FROM %s", + quoteIdentifier(columnName), tableIdentifier(table.getTablePath())); + } + + try (Statement stmt = + connection.createStatement( + ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY)) { + stmt.setFetchSize(Integer.MIN_VALUE); + try (ResultSet rs = stmt.executeQuery(sampleQuery)) { + int count = 0; + List results = new ArrayList<>(); + + while (rs.next()) { + count++; + if (count % samplingRate == 0) { + results.add(rs.getObject(1)); + } + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException("Thread interrupted"); + } + } + Object[] resultsArray = results.toArray(); + Arrays.sort(resultsArray); + return resultsArray; + } + } + } + + @Override + public Long approximateRowCntStatement(Connection connection, JdbcSourceTable table) + throws SQLException { + + // 1. If no query is configured, use TABLE STATUS. + // 2. If a query is configured but does not contain a WHERE clause and tablePath is + // configured , use TABLE STATUS. + // 3. If a query is configured with a WHERE clause, or a query statement is configured but + // tablePath is TablePath.DEFAULT, use COUNT(*). + + boolean useTableStats = + StringUtils.isBlank(table.getQuery()) + || (!table.getQuery().toLowerCase().contains("where") + && table.getTablePath() != null + && !TablePath.DEFAULT + .getFullName() + .equals(table.getTablePath().getFullName())); + + if (useTableStats) { + // The statement used to get approximate row count which is less + // accurate than COUNT(*), but is more efficient for large table. + TablePath tablePath = table.getTablePath(); + String useDatabaseStatement = + String.format("USE %s;", quoteDatabaseIdentifier(tablePath.getDatabaseName())); + String rowCountQuery = + String.format("SHOW TABLE STATUS LIKE '%s';", tablePath.getTableName()); + + try (Statement stmt = connection.createStatement()) { + log.info("Split Chunk, approximateRowCntStatement: {}", useDatabaseStatement); + stmt.execute(useDatabaseStatement); + log.info("Split Chunk, approximateRowCntStatement: {}", rowCountQuery); + try (ResultSet rs = stmt.executeQuery(rowCountQuery)) { + if (!rs.next() || rs.getMetaData().getColumnCount() < 5) { + throw new SQLException( + String.format( + "No result returned after running query [%s]", + rowCountQuery)); + } + return rs.getLong(5); + } + } + } + + return SQLUtils.countForSubquery(connection, table.getQuery()); + } + + @Override + public void refreshTableSchemaBySchemaChangeEvent( + String sourceDialectName, + AlterTableColumnEvent event, + JdbcConnectionProvider refreshTableSchemaConnectionProvider, + TablePath sinkTablePath) { + try (Connection connection = + refreshTableSchemaConnectionProvider.getOrEstablishConnection(); + Statement stmt = connection.createStatement()) { + String alterTableSql = generateAlterTableSql(sourceDialectName, event, sinkTablePath); + log.info("Apply schema change with sql: {}", alterTableSql); + stmt.execute(alterTableSql); + } catch (Exception e) { + throw new JdbcConnectorException( + JdbcConnectorErrorCode.REFRESH_PHYSICAL_TABLESCHEMA_BY_SCHEMA_CHANGE_EVENT, e); + } + } + + @Override + public String decorateWithComment(String basicSql, BasicTypeDefine typeBasicTypeDefine) { + OceanBaseMysqlType nativeType = (OceanBaseMysqlType) typeBasicTypeDefine.getNativeType(); + if (NOT_SUPPORTED_DEFAULT_VALUES.contains(nativeType)) { + return basicSql; + } + return JdbcDialect.super.decorateWithComment(basicSql, typeBasicTypeDefine); + } + + @Override + public boolean needsQuotesWithDefaultValue(String sqlType) { + OceanBaseMysqlType mysqlType = OceanBaseMysqlType.getByName(sqlType); + switch (mysqlType) { + case CHAR: + case VARCHAR: + case TEXT: + case TINYTEXT: + case MEDIUMTEXT: + case LONGTEXT: + case ENUM: + case SET: + case BLOB: + case TINYBLOB: + case MEDIUMBLOB: + case LONGBLOB: + case DATE: + case DATETIME: + case TIMESTAMP: + case TIME: + case YEAR: + return true; + default: + return false; + } + } + + @Override + public boolean isSpecialDefaultValue(Object defaultValue) { + return MysqlDefaultValueUtils.isSpecialDefaultValue(defaultValue); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlJdbcRowConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlJdbcRowConverter.java new file mode 100644 index 00000000000..2033518108c --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlJdbcRowConverter.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oceanbase; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.AbstractJdbcRowConverter; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; + +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; + +public class OceanBaseMysqlJdbcRowConverter extends AbstractJdbcRowConverter { + @Override + public String converterName() { + return DatabaseIdentifier.OCENABASE; + } + + @Override + protected void writeTime(PreparedStatement statement, int index, LocalTime time) + throws SQLException { + // Write to time column using timestamp retains milliseconds + statement.setTimestamp( + index, java.sql.Timestamp.valueOf(LocalDateTime.of(LocalDate.now(), time))); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlType.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlType.java new file mode 100644 index 00000000000..01f8141c392 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oceanbase/OceanBaseMysqlType.java @@ -0,0 +1,567 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oceanbase; + +import org.apache.commons.lang3.StringUtils; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Date; +import java.sql.SQLType; +import java.sql.Time; +import java.sql.Timestamp; +import java.sql.Types; +import java.time.LocalDateTime; + +public enum OceanBaseMysqlType implements SQLType { + DECIMAL( + "DECIMAL", + Types.DECIMAL, + BigDecimal.class, + OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 65L, + "[(M[,D])] [UNSIGNED] [ZEROFILL]"), + + DECIMAL_UNSIGNED( + "DECIMAL UNSIGNED", + Types.DECIMAL, + BigDecimal.class, + OceanBaseMysqlType.FIELD_FLAG_UNSIGNED | OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 65L, + "[(M[,D])] [UNSIGNED] [ZEROFILL]"), + + TINYINT( + "TINYINT", + Types.TINYINT, + Integer.class, + OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 3L, + "[(M)] [UNSIGNED] [ZEROFILL]"), + + TINYINT_UNSIGNED( + "TINYINT UNSIGNED", + Types.TINYINT, + Integer.class, + OceanBaseMysqlType.FIELD_FLAG_UNSIGNED | OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 3L, + "[(M)] [UNSIGNED] [ZEROFILL]"), + + BOOLEAN("BOOLEAN", Types.BOOLEAN, Boolean.class, 0, OceanBaseMysqlType.IS_NOT_DECIMAL, 3L, ""), + + SMALLINT( + "SMALLINT", + Types.SMALLINT, + Integer.class, + OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 5L, + "[(M)] [UNSIGNED] [ZEROFILL]"), + + SMALLINT_UNSIGNED( + "SMALLINT UNSIGNED", + Types.SMALLINT, + Integer.class, + OceanBaseMysqlType.FIELD_FLAG_UNSIGNED | OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 5L, + "[(M)] [UNSIGNED] [ZEROFILL]"), + + INT( + "INT", + Types.INTEGER, + Integer.class, + OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 10L, + "[(M)] [UNSIGNED] [ZEROFILL]"), + + INT_UNSIGNED( + "INT UNSIGNED", + Types.INTEGER, + Long.class, + OceanBaseMysqlType.FIELD_FLAG_UNSIGNED | OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 10L, + "[(M)] [UNSIGNED] [ZEROFILL]"), + + FLOAT( + "FLOAT", + Types.REAL, + Float.class, + OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 12L, + "[(M,D)] [UNSIGNED] [ZEROFILL]"), + + FLOAT_UNSIGNED( + "FLOAT UNSIGNED", + Types.REAL, + Float.class, + OceanBaseMysqlType.FIELD_FLAG_UNSIGNED | OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 12L, + "[(M,D)] [UNSIGNED] [ZEROFILL]"), + + DOUBLE( + "DOUBLE", + Types.DOUBLE, + Double.class, + OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 22L, + "[(M,D)] [UNSIGNED] [ZEROFILL]"), + + DOUBLE_UNSIGNED( + "DOUBLE UNSIGNED", + Types.DOUBLE, + Double.class, + OceanBaseMysqlType.FIELD_FLAG_UNSIGNED | OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 22L, + "[(M,D)] [UNSIGNED] [ZEROFILL]"), + /** FIELD_TYPE_NULL = 6 */ + NULL("NULL", Types.NULL, Object.class, 0, OceanBaseMysqlType.IS_NOT_DECIMAL, 0L, ""), + + TIMESTAMP( + "TIMESTAMP", + Types.TIMESTAMP, + Timestamp.class, + 0, + OceanBaseMysqlType.IS_NOT_DECIMAL, + 26L, + "[(fsp)]"), + + BIGINT( + "BIGINT", + Types.BIGINT, + Long.class, + OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 19L, + "[(M)] [UNSIGNED] [ZEROFILL]"), + + BIGINT_UNSIGNED( + "BIGINT UNSIGNED", + Types.BIGINT, + BigInteger.class, + OceanBaseMysqlType.FIELD_FLAG_UNSIGNED | OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 20L, + "[(M)] [UNSIGNED] [ZEROFILL]"), + + MEDIUMINT( + "MEDIUMINT", + Types.INTEGER, + Integer.class, + OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 7L, + "[(M)] [UNSIGNED] [ZEROFILL]"), + + MEDIUMINT_UNSIGNED( + "MEDIUMINT UNSIGNED", + Types.INTEGER, + Integer.class, + OceanBaseMysqlType.FIELD_FLAG_UNSIGNED | OceanBaseMysqlType.FIELD_FLAG_ZEROFILL, + OceanBaseMysqlType.IS_DECIMAL, + 8L, + "[(M)] [UNSIGNED] [ZEROFILL]"), + + DATE("DATE", Types.DATE, Date.class, 0, OceanBaseMysqlType.IS_NOT_DECIMAL, 10L, ""), + + TIME("TIME", Types.TIME, Time.class, 0, OceanBaseMysqlType.IS_NOT_DECIMAL, 16L, "[(fsp)]"), + + DATETIME( + "DATETIME", + Types.TIMESTAMP, + LocalDateTime.class, + 0, + OceanBaseMysqlType.IS_NOT_DECIMAL, + 26L, + "[(fsp)]"), + + YEAR("YEAR", Types.DATE, Date.class, 0, OceanBaseMysqlType.IS_NOT_DECIMAL, 4L, "[(4)]"), + + VARCHAR( + "VARCHAR", + Types.VARCHAR, + String.class, + 0, + OceanBaseMysqlType.IS_NOT_DECIMAL, + 65535L, + "(M) [CHARACTER SET charset_name] [COLLATE collation_name]"), + + VARBINARY( + "VARBINARY", + Types.VARBINARY, + null, + 0, + OceanBaseMysqlType.IS_NOT_DECIMAL, + 65535L, + "(M)"), + + BIT("BIT", Types.BIT, Boolean.class, 0, OceanBaseMysqlType.IS_DECIMAL, 1L, "[(M)]"), + + JSON( + "JSON", + Types.LONGVARCHAR, + String.class, + 0, + OceanBaseMysqlType.IS_NOT_DECIMAL, + 1073741824L, + ""), + + ENUM( + "ENUM", + Types.CHAR, + String.class, + 0, + OceanBaseMysqlType.IS_NOT_DECIMAL, + 65535L, + "('value1','value2',...) [CHARACTER SET charset_name] [COLLATE collation_name]"), + + SET( + "SET", + Types.CHAR, + String.class, + 0, + OceanBaseMysqlType.IS_NOT_DECIMAL, + 64L, + "('value1','value2',...) [CHARACTER SET charset_name] [COLLATE collation_name]"), + + TINYBLOB("TINYBLOB", Types.VARBINARY, null, 0, OceanBaseMysqlType.IS_NOT_DECIMAL, 255L, ""), + + TINYTEXT( + "TINYTEXT", + Types.VARCHAR, + String.class, + 0, + OceanBaseMysqlType.IS_NOT_DECIMAL, + 255L, + " [CHARACTER SET charset_name] [COLLATE collation_name]"), + + MEDIUMBLOB( + "MEDIUMBLOB", + Types.LONGVARBINARY, + null, + 0, + OceanBaseMysqlType.IS_NOT_DECIMAL, + 16777215L, + ""), + + MEDIUMTEXT( + "MEDIUMTEXT", + Types.LONGVARCHAR, + String.class, + 0, + OceanBaseMysqlType.IS_NOT_DECIMAL, + 16777215L, + " [CHARACTER SET charset_name] [COLLATE collation_name]"), + + LONGBLOB( + "LONGBLOB", + Types.LONGVARBINARY, + null, + 0, + OceanBaseMysqlType.IS_NOT_DECIMAL, + 4294967295L, + ""), + + LONGTEXT( + "LONGTEXT", + Types.LONGVARCHAR, + String.class, + 0, + OceanBaseMysqlType.IS_NOT_DECIMAL, + 4294967295L, + " [CHARACTER SET charset_name] [COLLATE collation_name]"), + + BLOB("BLOB", Types.LONGVARBINARY, null, 0, OceanBaseMysqlType.IS_NOT_DECIMAL, 65535L, "[(M)]"), + + TEXT( + "TEXT", + Types.LONGVARCHAR, + String.class, + 0, + OceanBaseMysqlType.IS_NOT_DECIMAL, + 65535L, + "[(M)] [CHARACTER SET charset_name] [COLLATE collation_name]"), + + CHAR( + "CHAR", + Types.CHAR, + String.class, + 0, + OceanBaseMysqlType.IS_NOT_DECIMAL, + 255L, + "[(M)] [CHARACTER SET charset_name] [COLLATE collation_name]"), + + BINARY("BINARY", Types.BINARY, null, 0, OceanBaseMysqlType.IS_NOT_DECIMAL, 255L, "(M)"), + + GEOMETRY("GEOMETRY", Types.BINARY, null, 0, OceanBaseMysqlType.IS_NOT_DECIMAL, 65535L, ""), + // is represented by BLOB + UNKNOWN("UNKNOWN", Types.OTHER, null, 0, OceanBaseMysqlType.IS_NOT_DECIMAL, 65535L, ""); + + private final String name; + protected int jdbcType; + protected final Class javaClass; + private final int flagsMask; + private final boolean isDecimal; + private final Long precision; + private final String createParams; + + private OceanBaseMysqlType( + String oceanBaseMysqlTypeName, + int jdbcType, + Class javaClass, + int allowedFlags, + boolean isDec, + Long precision, + String createParams) { + this.name = oceanBaseMysqlTypeName; + this.jdbcType = jdbcType; + this.javaClass = javaClass; + this.flagsMask = allowedFlags; + this.isDecimal = isDec; + this.precision = precision; + this.createParams = createParams; + } + + public static final int FIELD_FLAG_UNSIGNED = 32; + public static final int FIELD_FLAG_ZEROFILL = 64; + + private static final boolean IS_DECIMAL = true; + private static final boolean IS_NOT_DECIMAL = false; + + public static OceanBaseMysqlType getByName(String fullMysqlTypeName) { + + String typeName = ""; + + if (fullMysqlTypeName.indexOf("(") != -1) { + typeName = fullMysqlTypeName.substring(0, fullMysqlTypeName.indexOf("(")).trim(); + } else { + typeName = fullMysqlTypeName; + } + + // the order of checks is important because some short names could match parts of longer + // names + if (StringUtils.indexOfIgnoreCase(typeName, "DECIMAL") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "DEC") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "NUMERIC") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "FIXED") != -1) { + return StringUtils.indexOfIgnoreCase(fullMysqlTypeName, "UNSIGNED") != -1 + ? DECIMAL_UNSIGNED + : DECIMAL; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "TINYBLOB") != -1) { + // IMPORTANT: "TINYBLOB" must be checked before "TINY" + return TINYBLOB; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "TINYTEXT") != -1) { + // IMPORTANT: "TINYTEXT" must be checked before "TINY" + return TINYTEXT; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "TINYINT") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "TINY") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "INT1") != -1) { + return StringUtils.indexOfIgnoreCase(fullMysqlTypeName, "UNSIGNED") != -1 + || StringUtils.indexOfIgnoreCase(fullMysqlTypeName, "ZEROFILL") != -1 + ? TINYINT_UNSIGNED + : TINYINT; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "MEDIUMINT") != -1 + // IMPORTANT: "INT24" must be checked before "INT2" + || StringUtils.indexOfIgnoreCase(typeName, "INT24") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "INT3") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "MIDDLEINT") != -1) { + return StringUtils.indexOfIgnoreCase(fullMysqlTypeName, "UNSIGNED") != -1 + || StringUtils.indexOfIgnoreCase(fullMysqlTypeName, "ZEROFILL") != -1 + ? MEDIUMINT_UNSIGNED + : MEDIUMINT; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "SMALLINT") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "INT2") != -1) { + return StringUtils.indexOfIgnoreCase(fullMysqlTypeName, "UNSIGNED") != -1 + || StringUtils.indexOfIgnoreCase(fullMysqlTypeName, "ZEROFILL") != -1 + ? SMALLINT_UNSIGNED + : SMALLINT; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "BIGINT") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "SERIAL") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "INT8") != -1) { + // SERIAL is an alias for BIGINT UNSIGNED NOT NULL AUTO_INCREMENT UNIQUE. + return StringUtils.indexOfIgnoreCase(fullMysqlTypeName, "UNSIGNED") != -1 + || StringUtils.indexOfIgnoreCase(fullMysqlTypeName, "ZEROFILL") != -1 + ? BIGINT_UNSIGNED + : BIGINT; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "POINT") != -1) { + // also covers "MULTIPOINT" + // IMPORTANT: "POINT" must be checked before "INT" + } else if (StringUtils.indexOfIgnoreCase(typeName, "INT") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "INTEGER") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "INT4") != -1) { + // IMPORTANT: "INT" must be checked after all "*INT*" types + return StringUtils.indexOfIgnoreCase(fullMysqlTypeName, "UNSIGNED") != -1 + || StringUtils.indexOfIgnoreCase(fullMysqlTypeName, "ZEROFILL") != -1 + ? INT_UNSIGNED + : INT; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "DOUBLE") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "REAL") != -1 + /* || StringUtils.indexOfIgnoreCase(name, "DOUBLE PRECISION") != -1 is caught by "DOUBLE" check */ + // IMPORTANT: "FLOAT8" must be checked before "FLOAT" + || StringUtils.indexOfIgnoreCase(typeName, "FLOAT8") != -1) { + return StringUtils.indexOfIgnoreCase(fullMysqlTypeName, "UNSIGNED") != -1 + || StringUtils.indexOfIgnoreCase(fullMysqlTypeName, "ZEROFILL") != -1 + ? DOUBLE_UNSIGNED + : DOUBLE; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "FLOAT") != -1 /* + * || StringUtils.indexOfIgnoreCase(name, "FLOAT4") != -1 is caught by + * "FLOAT" check + */) { + return StringUtils.indexOfIgnoreCase(fullMysqlTypeName, "UNSIGNED") != -1 + || StringUtils.indexOfIgnoreCase(fullMysqlTypeName, "ZEROFILL") != -1 + ? FLOAT_UNSIGNED + : FLOAT; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "NULL") != -1) { + return NULL; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "TIMESTAMP") != -1) { + // IMPORTANT: "TIMESTAMP" must be checked before "TIME" + return TIMESTAMP; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "DATETIME") != -1) { + // IMPORTANT: "DATETIME" must be checked before "DATE" and "TIME" + return DATETIME; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "DATE") != -1) { + return DATE; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "TIME") != -1) { + return TIME; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "YEAR") != -1) { + return YEAR; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "LONGBLOB") != -1) { + // IMPORTANT: "LONGBLOB" must be checked before "LONG" and "BLOB" + return LONGBLOB; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "LONGTEXT") != -1) { + // IMPORTANT: "LONGTEXT" must be checked before "LONG" and "TEXT" + return LONGTEXT; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "MEDIUMBLOB") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "LONG VARBINARY") != -1) { + // IMPORTANT: "MEDIUMBLOB" must be checked before "BLOB" + // IMPORTANT: "LONG VARBINARY" must be checked before "LONG" and "VARBINARY" + return MEDIUMBLOB; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "MEDIUMTEXT") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "LONG VARCHAR") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "LONG") != -1) { + // IMPORTANT: "MEDIUMTEXT" must be checked before "TEXT" + // IMPORTANT: "LONG VARCHAR" must be checked before "VARCHAR" + return MEDIUMTEXT; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "VARCHAR") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "NVARCHAR") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "NATIONAL VARCHAR") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "CHARACTER VARYING") != -1) { + // IMPORTANT: "CHARACTER VARYING" must be checked before "CHARACTER" and "CHAR" + return VARCHAR; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "VARBINARY") != -1) { + return VARBINARY; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "BINARY") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "CHAR BYTE") != -1) { + // IMPORTANT: "BINARY" must be checked after all "*BINARY" types + // IMPORTANT: "CHAR BYTE" must be checked before "CHAR" + return BINARY; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "LINESTRING") != -1) { + // also covers "MULTILINESTRING" + // IMPORTANT: "LINESTRING" must be checked before "STRING" + return GEOMETRY; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "STRING") != -1 + // IMPORTANT: "CHAR" must be checked after all "*CHAR*" types + || StringUtils.indexOfIgnoreCase(typeName, "CHAR") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "NCHAR") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "NATIONAL CHAR") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "CHARACTER") != -1) { + return CHAR; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "BOOLEAN") != -1 + || StringUtils.indexOfIgnoreCase(typeName, "BOOL") != -1) { + return BOOLEAN; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "BIT") != -1) { + return BIT; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "JSON") != -1) { + return JSON; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "ENUM") != -1) { + return ENUM; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "SET") != -1) { + return SET; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "BLOB") != -1) { + return BLOB; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "TEXT") != -1) { + return TEXT; + + } else if (StringUtils.indexOfIgnoreCase(typeName, "GEOM") + != -1 // covers "GEOMETRY", "GEOMETRYCOLLECTION" and "GEOMCOLLECTION" + || StringUtils.indexOfIgnoreCase(typeName, "POINT") + != -1 // also covers "MULTIPOINT" + || StringUtils.indexOfIgnoreCase(typeName, "POLYGON") + != -1 // also covers "MULTIPOLYGON" + ) { + return GEOMETRY; + } + + return UNKNOWN; + } + + @Override + public String getVendor() { + return "com.oceanbase"; + } + + @Override + public Integer getVendorTypeNumber() { + return this.jdbcType; + } + + @Override + public String getName() { + return this.name; + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java index a6896322065..860131041a9 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java @@ -393,6 +393,8 @@ private static ReadonlyConfig extractCatalogConfig(JdbcConnectionConfig config) .ifPresent(val -> catalogConfig.put(JdbcCatalogOptions.USERNAME.key(), val)); config.getPassword() .ifPresent(val -> catalogConfig.put(JdbcCatalogOptions.PASSWORD.key(), val)); + Optional.ofNullable(config.getCompatibleMode()) + .ifPresent(val -> catalogConfig.put(JdbcCatalogOptions.COMPATIBLE_MODE.key(), val)); return ReadonlyConfig.fromMap(catalogConfig); } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOceanBaseMysqlIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOceanBaseMysqlIT.java index 3208473d619..a747058391b 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOceanBaseMysqlIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOceanBaseMysqlIT.java @@ -66,16 +66,10 @@ public class JdbcOceanBaseMysqlIT extends JdbcOceanBaseITBase { "bash", "-c", "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " - + driverUrl() - + " && wget " - + mysqlDriverUrl()); + + driverUrl()); Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); }; - String mysqlDriverUrl() { - return "https://repo1.maven.org/maven2/com/mysql/mysql-connector-j/8.0.32/mysql-connector-j-8.0.32.jar"; - } - @Override List configFile() { return Lists.newArrayList("/jdbc_oceanbase_mysql_source_and_sink.conf"); From 2769ed5029efc44891552725437e35d86a0eb93f Mon Sep 17 00:00:00 2001 From: hailin0 Date: Sat, 10 Aug 2024 09:58:03 +0800 Subject: [PATCH 042/361] [Hotfix][Connector] Fix jdbc compile error (#7359) --- .../catalog/oceanbase/OceanBaseMySqlCatalog.java | 6 ++++-- .../OceanBaseMysqlCreateTableSqlBuilder.java | 14 +++++++++----- 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMySqlCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMySqlCatalog.java index 08aa0faea08..b876e33cc8c 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMySqlCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMySqlCatalog.java @@ -182,8 +182,10 @@ protected Column buildColumn(ResultSet resultSet) throws SQLException { } @Override - protected String getCreateTableSql(TablePath tablePath, CatalogTable table) { - return OceanBaseMysqlCreateTableSqlBuilder.builder(tablePath, table, typeConverter) + protected String getCreateTableSql( + TablePath tablePath, CatalogTable table, boolean createIndex) { + return OceanBaseMysqlCreateTableSqlBuilder.builder( + tablePath, table, typeConverter, createIndex) .build(table.getCatalogName()); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMysqlCreateTableSqlBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMysqlCreateTableSqlBuilder.java index bc3413dbd82..9707ff23acc 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMysqlCreateTableSqlBuilder.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oceanbase/OceanBaseMysqlCreateTableSqlBuilder.java @@ -60,25 +60,29 @@ public class OceanBaseMysqlCreateTableSqlBuilder { private String fieldIde; private final OceanBaseMySqlTypeConverter typeConverter; + private boolean createIndex; private OceanBaseMysqlCreateTableSqlBuilder( - String tableName, OceanBaseMySqlTypeConverter typeConverter) { + String tableName, OceanBaseMySqlTypeConverter typeConverter, boolean createIndex) { checkNotNull(tableName, "tableName must not be null"); this.tableName = tableName; this.typeConverter = typeConverter; + this.createIndex = createIndex; } public static OceanBaseMysqlCreateTableSqlBuilder builder( TablePath tablePath, CatalogTable catalogTable, - OceanBaseMySqlTypeConverter typeConverter) { + OceanBaseMySqlTypeConverter typeConverter, + boolean createIndex) { checkNotNull(tablePath, "tablePath must not be null"); checkNotNull(catalogTable, "catalogTable must not be null"); TableSchema tableSchema = catalogTable.getTableSchema(); checkNotNull(tableSchema, "tableSchema must not be null"); - return new OceanBaseMysqlCreateTableSqlBuilder(tablePath.getTableName(), typeConverter) + return new OceanBaseMysqlCreateTableSqlBuilder( + tablePath.getTableName(), typeConverter, createIndex) .comment(catalogTable.getComment()) // todo: set charset and collate .engine(null) @@ -158,10 +162,10 @@ private String buildColumnsIdentifySql(String catalogName) { for (Column column : columns) { columnSqls.add("\t" + buildColumnIdentifySql(column, catalogName, columnTypeMap)); } - if (primaryKey != null) { + if (createIndex && primaryKey != null) { columnSqls.add("\t" + buildPrimaryKeySql()); } - if (CollectionUtils.isNotEmpty(constraintKeys)) { + if (createIndex && CollectionUtils.isNotEmpty(constraintKeys)) { for (ConstraintKey constraintKey : constraintKeys) { if (StringUtils.isBlank(constraintKey.getConstraintName())) { continue; From 862e2055cec6a4428ec4cf67b472ba41ca364d05 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Sat, 10 Aug 2024 15:21:36 +0800 Subject: [PATCH 043/361] [Improve][Document] Change deprecated connector name in setup.md (#7366) --- docs/en/contribution/setup.md | 38 +++++++++++++++++++++-------------- docs/zh/contribution/setup.md | 38 +++++++++++++++++++++-------------- 2 files changed, 46 insertions(+), 30 deletions(-) diff --git a/docs/en/contribution/setup.md b/docs/en/contribution/setup.md index 094799e6f56..b2579e1ee1e 100644 --- a/docs/en/contribution/setup.md +++ b/docs/en/contribution/setup.md @@ -80,16 +80,21 @@ After all the above things are done, you just finish the environment setup and c of box. All examples are in module `seatunnel-examples`, you could pick one you are interested in, [Running Or Debugging It In IDEA](https://www.jetbrains.com/help/idea/run-debug-configuration.html) as you wish. -Here we use `seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java` +Here we use `seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java` as an example, when you run it successfully you can see the output as below: ```log -+I[Ricky Huo, 71] -+I[Gary, 12] -+I[Ricky Huo, 93] -... -... -+I[Ricky Huo, 83] +2024-08-10 11:45:32,839 INFO org.apache.seatunnel.core.starter.seatunnel.command.ClientExecuteCommand - +*********************************************** + Job Statistic Information +*********************************************** +Start Time : 2024-08-10 11:45:30 +End Time : 2024-08-10 11:45:32 +Total Time(s) : 2 +Total Read Count : 5 +Total Write Count : 5 +Total Failed Count : 0 +*********************************************** ``` ## What's More @@ -97,23 +102,26 @@ as an example, when you run it successfully you can see the output as below: All our examples use simple source and sink to make it less dependent and easy to run. You can change the example configuration in `resources/examples`. You can change your configuration as below, if you want to use PostgreSQL as the source and sink to console. +Please note that when using connectors other than FakeSource and Console, you need to modify the dependencies in the `pom.xml` file of the corresponding submodule of seatunnel-example. ```conf env { parallelism = 1 + job.mode = "BATCH" } - source { - JdbcSource { - driver = org.postgresql.Driver - url = "jdbc:postgresql://host:port/database" - username = postgres - query = "select * from test" - } + Jdbc { + driver = org.postgresql.Driver + url = "jdbc:postgresql://host:port/database" + username = postgres + password = "123456" + query = "select * from test" + table_path = "database.test" + } } sink { - ConsoleSink {} + Console {} } ``` diff --git a/docs/zh/contribution/setup.md b/docs/zh/contribution/setup.md index 3527f72c1dc..c00c3132c22 100644 --- a/docs/zh/contribution/setup.md +++ b/docs/zh/contribution/setup.md @@ -75,39 +75,47 @@ Apache SeaTunnel 使用 `Spotless` 来统一代码风格和格式检查。可以 完成上面所有的工作后,环境搭建已经完成, 可以直接运行我们的示例了。 所有的示例在 `seatunnel-examples` 模块里, 你可以随意选择进行编译和调试,参考 [running or debugging it in IDEA](https://www.jetbrains.com/help/idea/run-debug-configuration.html)。 -我们使用 `seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java` +我们使用 `seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java` 作为示例, 运行成功后的输出如下: ```log -+I[Ricky Huo, 71] -+I[Gary, 12] -+I[Ricky Huo, 93] -... -... -+I[Ricky Huo, 83] +2024-08-10 11:45:32,839 INFO org.apache.seatunnel.core.starter.seatunnel.command.ClientExecuteCommand - +*********************************************** + Job Statistic Information +*********************************************** +Start Time : 2024-08-10 11:45:30 +End Time : 2024-08-10 11:45:32 +Total Time(s) : 2 +Total Read Count : 5 +Total Write Count : 5 +Total Failed Count : 0 +*********************************************** ``` ## 更多信息 所有的实例都用了简单的 source 和 sink, 这样可以使得运行更独立和更简单。 你可以修改 `resources/examples` 中的示例的配置。 例如下面的配置使用 PostgreSQL 作为源,并且输出到控制台。 +请注意引用FakeSource 和 Console 以外的连接器时,需要修改seatunnel-example对应子模块下的`pom.xml`文件中的依赖。 ```conf env { parallelism = 1 + job.mode = "BATCH" } - source { - JdbcSource { - driver = org.postgresql.Driver - url = "jdbc:postgresql://host:port/database" - username = postgres - query = "select * from test" - } + Jdbc { + driver = org.postgresql.Driver + url = "jdbc:postgresql://host:port/database" + username = postgres + password = "123456" + query = "select * from test" + table_path = "database.test" + } } sink { - ConsoleSink {} + Console {} } ``` From d46cf16e5a4bb0e4820feaf4dd03e7e9038d6281 Mon Sep 17 00:00:00 2001 From: Jarvis Date: Sat, 10 Aug 2024 23:47:12 +0800 Subject: [PATCH 044/361] [Feature] Split transform and move jar into connectors directory (#7218) --- plugin-mapping.properties | 13 ++++- .../core/starter/execution/PluginUtil.java | 16 +++--- .../execution/TransformExecuteProcessor.java | 14 ++++- .../execution/TransformExecuteProcessor.java | 11 +++- .../src/main/assembly/assembly-bin-ci.xml | 48 +++++++---------- .../src/main/assembly/assembly-bin.xml | 6 +-- .../e2e/common/util/ContainerUtil.java | 4 +- .../parse/MultipleTableJobConfigParser.java | 36 +++++++++---- .../SeaTunnelTransformPluginDiscovery.java | 2 +- .../AbstractCatalogSupportTransform.java | 16 +++++- .../common/AbstractSeaTunnelTransform.java | 51 ------------------- .../seatunnel/transform/sql/SQLTransform.java | 15 ++---- 12 files changed, 108 insertions(+), 124 deletions(-) delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java diff --git a/plugin-mapping.properties b/plugin-mapping.properties index 1942f875d7c..579bf2dac04 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -129,4 +129,15 @@ seatunnel.source.ObsFile = connector-file-obs seatunnel.sink.ObsFile = connector-file-obs seatunnel.source.Milvus = connector-milvus seatunnel.sink.Milvus = connector-milvus -seatunnel.sink.ActiveMQ = connector-activemq \ No newline at end of file +seatunnel.sink.ActiveMQ = connector-activemq + +seatunnel.transform.Sql = seatunnel-transforms-v2 +seatunnel.transform.FieldMapper = seatunnel-transforms-v2 +seatunnel.transform.Filter = seatunnel-transforms-v2 +seatunnel.transform.FilterRowKind = seatunnel-transforms-v2 +seatunnel.transform.JsonPath = seatunnel-transforms-v2 +seatunnel.transform.Replace = seatunnel-transforms-v2 +seatunnel.transform.Split = seatunnel-transforms-v2 +seatunnel.transform.Copy = seatunnel-transforms-v2 +seatunnel.transform.DynamicCompile = seatunnel-transforms-v2 +seatunnel.transform.LLM = seatunnel-transforms-v2 \ No newline at end of file diff --git a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/PluginUtil.java b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/PluginUtil.java index 0dc4209a8b6..166e581e2d9 100644 --- a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/PluginUtil.java +++ b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/PluginUtil.java @@ -31,7 +31,6 @@ import org.apache.seatunnel.api.table.factory.FactoryException; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; -import org.apache.seatunnel.api.table.factory.TableTransformFactory; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.common.constants.JobMode; import org.apache.seatunnel.common.utils.SeaTunnelException; @@ -49,7 +48,6 @@ import static org.apache.seatunnel.api.common.CommonOptions.PLUGIN_NAME; import static org.apache.seatunnel.api.table.factory.FactoryUtil.DEFAULT_ID; -import static org.apache.seatunnel.api.table.factory.FactoryUtil.discoverFactory; /** The util used for Spark/Flink to create to SeaTunnelSource etc. */ public class PluginUtil { @@ -130,21 +128,21 @@ private static SeaTunnelSource fallbackCreate( return source; } - public static TableTransformFactory createTransformFactory( + public static Optional createTransformFactory( + SeaTunnelFactoryDiscovery factoryDiscovery, SeaTunnelTransformPluginDiscovery transformPluginDiscovery, Config transformConfig, List pluginJars) { PluginIdentifier pluginIdentifier = PluginIdentifier.of( ENGINE_TYPE, "transform", transformConfig.getString(PLUGIN_NAME.key())); - final ReadonlyConfig readonlyConfig = ReadonlyConfig.fromConfig(transformConfig); - final String factoryId = readonlyConfig.get(PLUGIN_NAME); - ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - final TableTransformFactory factory = - discoverFactory(classLoader, TableTransformFactory.class, factoryId); pluginJars.addAll( transformPluginDiscovery.getPluginJarPaths(Lists.newArrayList(pluginIdentifier))); - return factory; + try { + return factoryDiscovery.createOptionalPluginInstance(pluginIdentifier); + } catch (FactoryException e) { + return Optional.empty(); + } } public static Optional createSinkFactory( diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java index d91bb9d3da7..1ff2cf64372 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java @@ -29,6 +29,7 @@ import org.apache.seatunnel.api.transform.SeaTunnelTransform; import org.apache.seatunnel.core.starter.exception.TaskExecuteException; import org.apache.seatunnel.core.starter.execution.PluginUtil; +import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelFactoryDiscovery; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelTransformPluginDiscovery; import org.apache.seatunnel.translation.flink.serialization.FlinkRowConverter; import org.apache.seatunnel.translation.flink.utils.TypeConverterUtils; @@ -41,6 +42,7 @@ import java.net.URL; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; import static org.apache.seatunnel.api.common.CommonOptions.RESULT_TABLE_NAME; @@ -59,15 +61,23 @@ protected TransformExecuteProcessor( @Override protected List initializePlugins( List jarPaths, List pluginConfigs) { + + SeaTunnelFactoryDiscovery factoryDiscovery = + new SeaTunnelFactoryDiscovery(TableTransformFactory.class, ADD_URL_TO_CLASSLOADER); SeaTunnelTransformPluginDiscovery transformPluginDiscovery = new SeaTunnelTransformPluginDiscovery(); - return pluginConfigs.stream() .map( transformConfig -> PluginUtil.createTransformFactory( - transformPluginDiscovery, transformConfig, jarPaths)) + factoryDiscovery, + transformPluginDiscovery, + transformConfig, + jarPaths)) .distinct() + .filter(Optional::isPresent) + .map(Optional::get) + .map(e -> (TableTransformFactory) e) .collect(Collectors.toList()); } diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java index bc7cd5cdbed..fc4a9e00d0d 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java @@ -29,6 +29,7 @@ import org.apache.seatunnel.api.transform.SeaTunnelTransform; import org.apache.seatunnel.core.starter.exception.TaskExecuteException; import org.apache.seatunnel.core.starter.execution.PluginUtil; +import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelFactoryDiscovery; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelTransformPluginDiscovery; import org.apache.seatunnel.translation.spark.serialization.SeaTunnelRowConverter; import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; @@ -50,6 +51,7 @@ import java.util.Iterator; import java.util.List; import java.util.Objects; +import java.util.Optional; import java.util.stream.Collectors; import static org.apache.seatunnel.api.common.CommonOptions.RESULT_TABLE_NAME; @@ -69,16 +71,23 @@ protected TransformExecuteProcessor( protected List initializePlugins(List pluginConfigs) { SeaTunnelTransformPluginDiscovery transformPluginDiscovery = new SeaTunnelTransformPluginDiscovery(); + + SeaTunnelFactoryDiscovery factoryDiscovery = + new SeaTunnelFactoryDiscovery(TableTransformFactory.class); List pluginJars = new ArrayList<>(); List transforms = pluginConfigs.stream() .map( transformConfig -> PluginUtil.createTransformFactory( + factoryDiscovery, transformPluginDiscovery, transformConfig, - pluginJars)) + new ArrayList<>())) .distinct() + .filter(Optional::isPresent) + .map(Optional::get) + .map(e -> (TableTransformFactory) e) .collect(Collectors.toList()); sparkRuntimeEnvironment.registerPlugin(pluginJars); return transforms; diff --git a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml index cc48ac86a2c..4510579d811 100644 --- a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml +++ b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml @@ -140,7 +140,7 @@ provided - + false @@ -148,6 +148,7 @@ false org.apache.seatunnel:connector-*:jar + org.apache.seatunnel:seatunnel-transforms-v2:jar org.apache.seatunnel:connector-common @@ -160,36 +161,7 @@ provided - - - false - true - false - - org.apache.seatunnel:seatunnel-transforms-v2:jar - org.apache.hadoop:hadoop-aws:jar - com.amazonaws:aws-java-sdk-bundle:jar - org.apache.seatunnel:seatunnel-hadoop3-3.1.4-uber:jar:*:optional - - org.apache.hadoop:hadoop-aliyun:jar - com.aliyun.oss:aliyun-sdk-oss:jar - org.jdom:jdom:jar - - - io.netty:netty-buffer:jar - io.netty:netty-common:jar - - - org.apache.hive:hive-exec:jar - org.apache.hive:hive-service:jar - org.apache.thrift:libfb303:jar - - ${artifact.file.name} - /lib - provided - - - + false true @@ -209,6 +181,20 @@ com.amazon.redshift:redshift-jdbc42:jar net.snowflake.snowflake-jdbc:jar com.xugudb:xugu-jdbc:jar + org.apache.hadoop:hadoop-aws:jar + com.amazonaws:aws-java-sdk-bundle:jar + org.apache.seatunnel:seatunnel-hadoop3-3.1.4-uber:jar:*:optional + + org.apache.hadoop:hadoop-aliyun:jar + com.aliyun.oss:aliyun-sdk-oss:jar + org.jdom:jdom:jar + + io.netty:netty-buffer:jar + io.netty:netty-common:jar + + org.apache.hive:hive-exec:jar + org.apache.hive:hive-service:jar + org.apache.thrift:libfb303:jar ${artifact.file.name} /lib diff --git a/seatunnel-dist/src/main/assembly/assembly-bin.xml b/seatunnel-dist/src/main/assembly/assembly-bin.xml index 30fc5a6336a..f16841f7a95 100644 --- a/seatunnel-dist/src/main/assembly/assembly-bin.xml +++ b/seatunnel-dist/src/main/assembly/assembly-bin.xml @@ -161,13 +161,12 @@ provided - + false true false - org.apache.seatunnel:seatunnel-transforms-v2:jar org.apache.seatunnel:seatunnel-hadoop3-3.1.4-uber:jar:*:optional ${artifact.file.name} @@ -175,7 +174,7 @@ provided - + false @@ -184,6 +183,7 @@ org.apache.seatunnel:connector-fake:jar org.apache.seatunnel:connector-console:jar + org.apache.seatunnel:seatunnel-transforms-v2:jar /connectors provided diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/util/ContainerUtil.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/util/ContainerUtil.java index 1c590bb69ab..6c6a8e5cddd 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/util/ContainerUtil.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/util/ContainerUtil.java @@ -195,13 +195,13 @@ public static void copySeaTunnelStarterToContainer( MountableFile.forHostPath(startJarPath), Paths.get(seatunnelHomeInContainer, "starter", startJarName).toString()); - // copy lib + // copy transform String transformJar = "seatunnel-transforms-v2.jar"; Path transformJarPath = Paths.get(PROJECT_ROOT_PATH, "seatunnel-transforms-v2", "target", transformJar); container.withCopyFileToContainer( MountableFile.forHostPath(transformJarPath), - Paths.get(seatunnelHomeInContainer, "lib", transformJar).toString()); + Paths.get(seatunnelHomeInContainer, "connectors", transformJar).toString()); // copy bin final String startBinPath = startModulePath + File.separator + "src/main/bin/"; diff --git a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java index 40a6640c358..d02a76a4c51 100644 --- a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java +++ b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java @@ -182,7 +182,7 @@ public ImmutablePair, Set> parse(ClassLoaderService classLoade TypesafeConfigUtils.getConfigList( seaTunnelJobConfig, "sink", Collections.emptyList()); - List connectorJars = getConnectorJarList(sourceConfigs, sinkConfigs); + List connectorJars = getConnectorJarList(sourceConfigs, transformConfigs, sinkConfigs); if (!commonPluginJars.isEmpty()) { connectorJars.addAll(commonPluginJars); } @@ -238,18 +238,32 @@ public Set getUsedFactoryUrls(List sinkActions) { } private List getConnectorJarList( - List sourceConfigs, List sinkConfigs) { + List sourceConfigs, + List transformConfigs, + List sinkConfigs) { List factoryIds = Stream.concat( - sourceConfigs.stream() - .map(ConfigParserUtil::getFactoryId) - .map( - factory -> - PluginIdentifier.of( - CollectionConstants - .SEATUNNEL_PLUGIN, - CollectionConstants.SOURCE_PLUGIN, - factory)), + Stream.concat( + sourceConfigs.stream() + .map(ConfigParserUtil::getFactoryId) + .map( + factory -> + PluginIdentifier.of( + CollectionConstants + .SEATUNNEL_PLUGIN, + CollectionConstants + .SOURCE_PLUGIN, + factory)), + transformConfigs.stream() + .map(ConfigParserUtil::getFactoryId) + .map( + factory -> + PluginIdentifier.of( + CollectionConstants + .SEATUNNEL_PLUGIN, + CollectionConstants + .TRANSFORM_PLUGIN, + factory))), sinkConfigs.stream() .map(ConfigParserUtil::getFactoryId) .map( diff --git a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelTransformPluginDiscovery.java b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelTransformPluginDiscovery.java index 445bf14628d..606cd0d7cae 100644 --- a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelTransformPluginDiscovery.java +++ b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelTransformPluginDiscovery.java @@ -34,7 +34,7 @@ public class SeaTunnelTransformPluginDiscovery extends AbstractPluginDiscovery { public SeaTunnelTransformPluginDiscovery() { - super(Common.libDir()); + super(Common.connectorDir()); } @Override diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportTransform.java index 5670bcc1296..632d3af1e41 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportTransform.java @@ -20,10 +20,12 @@ import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.transform.SeaTunnelTransform; import lombok.NonNull; -public abstract class AbstractCatalogSupportTransform extends AbstractSeaTunnelTransform { +public abstract class AbstractCatalogSupportTransform implements SeaTunnelTransform { protected CatalogTable inputCatalogTable; protected volatile CatalogTable outputCatalogTable; @@ -32,6 +34,18 @@ public AbstractCatalogSupportTransform(@NonNull CatalogTable inputCatalogTable) this.inputCatalogTable = inputCatalogTable; } + @Override + public SeaTunnelRow map(SeaTunnelRow row) { + return transformRow(row); + } + + /** + * Outputs transformed row data. + * + * @param inputRow upstream input row data + */ + protected abstract SeaTunnelRow transformRow(SeaTunnelRow inputRow); + @Override public CatalogTable getProducedCatalogTable() { if (outputCatalogTable == null) { diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java deleted file mode 100644 index 1892881c277..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.transform.common; - -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.api.transform.SeaTunnelTransform; - -public abstract class AbstractSeaTunnelTransform implements SeaTunnelTransform { - - protected String inputTableName; - protected SeaTunnelRowType inputRowType; - - protected SeaTunnelRowType outputRowType; - - @Override - public SeaTunnelRow map(SeaTunnelRow row) { - return transformRow(row); - } - - /** - * Outputs transformed row data. - * - * @param inputRow upstream input row data - */ - protected abstract SeaTunnelRow transformRow(SeaTunnelRow inputRow); - - @Override - public CatalogTable getProducedCatalogTable() { - throw new UnsupportedOperationException( - String.format( - "Connector %s must implement TableTransformFactory.createTransform method", - getPluginName())); - } -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java index a9d04b07396..00316bba8e7 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java @@ -62,6 +62,8 @@ public class SQLTransform extends AbstractCatalogSupportTransform { private transient SQLEngine sqlEngine; + private final String inputTableName; + public SQLTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable catalogTable) { super(catalogTable); this.query = config.get(KEY_QUERY); @@ -77,15 +79,6 @@ public SQLTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable catalo } else { this.inputTableName = catalogTable.getTableId().getTableName(); } - List columns = catalogTable.getTableSchema().getColumns(); - String[] fieldNames = new String[columns.size()]; - SeaTunnelDataType[] fieldTypes = new SeaTunnelDataType[columns.size()]; - for (int i = 0; i < columns.size(); i++) { - Column column = columns.get(i); - fieldNames[i] = column.getName(); - fieldTypes[i] = column.getDataType(); - } - this.inputRowType = new SeaTunnelRowType(fieldNames, fieldTypes); } @Override @@ -98,8 +91,8 @@ public void open() { sqlEngine = SQLEngineFactory.getSQLEngine(engineType); sqlEngine.init( inputTableName, - inputCatalogTable != null ? inputCatalogTable.getTableId().getTableName() : null, - inputRowType, + inputCatalogTable.getTableId().getTableName(), + inputCatalogTable.getSeaTunnelRowType(), query); } From e23e3ac4ed5f83735797c4a9d5ffc0606cbe94ef Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 11 Aug 2024 17:40:02 +0800 Subject: [PATCH 045/361] Bump org.apache.activemq:activemq-client (#7323) Bumps org.apache.activemq:activemq-client from 5.14.5 to 5.15.16. --- seatunnel-connectors-v2/connector-activemq/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-activemq/pom.xml b/seatunnel-connectors-v2/connector-activemq/pom.xml index 7a72a3b1c4c..b905e055713 100644 --- a/seatunnel-connectors-v2/connector-activemq/pom.xml +++ b/seatunnel-connectors-v2/connector-activemq/pom.xml @@ -30,7 +30,7 @@ SeaTunnel : Connectors V2 : Activemq - 5.14.5 + 5.15.16 From 068c5e3e3e8518e8cef2bd7599d85455d1adbaef Mon Sep 17 00:00:00 2001 From: Tyrantlucifer Date: Mon, 12 Aug 2024 12:55:41 +0800 Subject: [PATCH 046/361] [Core][Flink] refactor flink proxy source/sink (#7355) --- .../api/table/type/SeaTunnelRow.java | 15 ++ .../core/starter/execution/PluginUtil.java | 1 + .../starter/execution/SourceTableInfo.java | 1 + .../execution/FlinkRuntimeEnvironment.java | 25 --- .../flink/execution/SinkExecuteProcessor.java | 12 +- .../AbstractFlinkRuntimeEnvironment.java | 18 -- .../flink/execution/DataStreamTableInfo.java | 8 +- .../FlinkAbstractPluginExecuteProcessor.java | 22 +- .../execution/FlinkRuntimeEnvironment.java | 25 --- .../flink/execution/SinkExecuteProcessor.java | 13 +- .../execution/SourceExecuteProcessor.java | 8 +- .../execution/TransformExecuteProcessor.java | 53 ++--- .../engine/e2e/UnifyEnvParameterIT.java | 16 -- .../pom.xml | 6 + .../resources/examples/fake_to_console.conf | 12 +- .../discovery/AbstractPluginDiscovery.java | 1 + .../flink/utils/TypeConverterUtilsTest.java | 161 -------------- .../serialization/FlinkRowConverter.java | 154 ------------- .../flink/sink/FlinkSinkWriter.java | 16 +- .../flink/source/FlinkRowCollector.java | 21 +- .../translation/flink/source/FlinkSource.java | 17 +- .../flink/source/FlinkSourceReader.java | 12 +- .../flink/utils/TypeConverterUtils.java | 210 ------------------ 23 files changed, 103 insertions(+), 724 deletions(-) delete mode 100644 seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-13/src/test/java/org/apache/seatunnel/translation/flink/utils/TypeConverterUtilsTest.java delete mode 100644 seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/serialization/FlinkRowConverter.java delete mode 100644 seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/utils/TypeConverterUtils.java diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java index 95a36b796c4..11388dbb6a7 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java @@ -50,7 +50,16 @@ public void setTableId(String tableId) { this.tableId = tableId; } + /** + * The method will be removed in the future, please use {@link #setKind(RowKind)} instanced of + * it. + */ + @Deprecated public void setRowKind(RowKind kind) { + setKind(kind); + } + + public void setKind(RowKind kind) { this.kind = kind; } @@ -62,7 +71,13 @@ public String getTableId() { return tableId; } + /** The method will be removed in the future, please use {@link #getKind()} instanced of it. */ + @Deprecated public RowKind getRowKind() { + return getKind(); + } + + public RowKind getKind() { return this.kind; } diff --git a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/PluginUtil.java b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/PluginUtil.java index 166e581e2d9..c47ea0b1215 100644 --- a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/PluginUtil.java +++ b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/PluginUtil.java @@ -50,6 +50,7 @@ import static org.apache.seatunnel.api.table.factory.FactoryUtil.DEFAULT_ID; /** The util used for Spark/Flink to create to SeaTunnelSource etc. */ +@SuppressWarnings("rawtypes") public class PluginUtil { protected static final String ENGINE_TYPE = "seatunnel"; diff --git a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/SourceTableInfo.java b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/SourceTableInfo.java index 529b9b42078..43642f57352 100644 --- a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/SourceTableInfo.java +++ b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/SourceTableInfo.java @@ -27,6 +27,7 @@ @Data @AllArgsConstructor +@SuppressWarnings("rawtypes") public class SourceTableInfo { private SeaTunnelSource source; diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkRuntimeEnvironment.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkRuntimeEnvironment.java index fcc25a6b9e6..e3428c751e6 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkRuntimeEnvironment.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkRuntimeEnvironment.java @@ -21,13 +21,6 @@ import org.apache.seatunnel.common.constants.JobMode; import org.apache.seatunnel.core.starter.execution.RuntimeEnvironment; -import org.apache.seatunnel.core.starter.flink.utils.ConfigKeyName; -import org.apache.seatunnel.core.starter.flink.utils.EnvironmentUtil; - -import org.apache.flink.api.common.time.Time; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableConfig; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import lombok.extern.slf4j.Slf4j; @@ -50,7 +43,6 @@ public FlinkRuntimeEnvironment setConfig(Config config) { @Override public FlinkRuntimeEnvironment prepare() { createStreamEnvironment(); - createStreamTableEnvironment(); if (config.hasPath("job.name")) { jobName = config.getString("job.name"); } @@ -63,23 +55,6 @@ public FlinkRuntimeEnvironment setJobMode(JobMode jobMode) { return this; } - private void createStreamTableEnvironment() { - EnvironmentSettings environmentSettings = - EnvironmentSettings.newInstance().inStreamingMode().useBlinkPlanner().build(); - tableEnvironment = - StreamTableEnvironment.create(getStreamExecutionEnvironment(), environmentSettings); - TableConfig config = tableEnvironment.getConfig(); - if (EnvironmentUtil.hasPathAndWaring(this.config, ConfigKeyName.MAX_STATE_RETENTION_TIME) - && EnvironmentUtil.hasPathAndWaring( - this.config, ConfigKeyName.MIN_STATE_RETENTION_TIME)) { - long max = this.config.getLong(ConfigKeyName.MAX_STATE_RETENTION_TIME); - long min = this.config.getLong(ConfigKeyName.MIN_STATE_RETENTION_TIME); - config.setIdleStateRetentionTime(Time.seconds(min), Time.seconds(max)); - } - // init flink table env config - EnvironmentUtil.initTableEnvironmentConfiguration(this.config, config.getConfiguration()); - } - public static FlinkRuntimeEnvironment getInstance(Config config) { if (INSTANCE == null) { synchronized (FlinkRuntimeEnvironment.class) { diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java index 6a272aadb21..51586beaf0f 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java @@ -51,6 +51,7 @@ import static org.apache.seatunnel.api.common.CommonOptions.PLUGIN_NAME; import static org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode.HANDLE_SAVE_MODE_FAILED; +@SuppressWarnings({"unchecked", "rawtypes"}) public class SinkExecuteProcessor extends FlinkAbstractPluginExecuteProcessor> { @@ -107,12 +108,15 @@ public List execute(List upstreamDataS sinkConfig.getString(PLUGIN_NAME.key())), sinkConfig); sink.setJobContext(jobContext); - SeaTunnelRowType sourceType = stream.getCatalogTable().getSeaTunnelRowType(); + // TODO support sink multi sink + SeaTunnelRowType sourceType = + stream.getCatalogTables().get(0).getSeaTunnelRowType(); sink.setTypeInfo(sourceType); } else { + // TODO support sink multi sink TableSinkFactoryContext context = TableSinkFactoryContext.replacePlaceholderAndCreate( - stream.getCatalogTable(), + stream.getCatalogTables().get(0), ReadonlyConfig.fromConfig(sinkConfig), classLoader, ((TableSinkFactory) factory.get()) @@ -134,8 +138,8 @@ public List execute(List upstreamDataS } DataStreamSink dataStreamSink = stream.getDataStream() - .sinkTo(new FlinkSink<>(sink, stream.getCatalogTable())) - .name(sink.getPluginName()); + .sinkTo(new FlinkSink<>(sink, stream.getCatalogTables().get(0))) + .name(String.format("%s-Sink", sink.getPluginName())); if (sinkConfig.hasPath(CommonOptions.PARALLELISM.key())) { int parallelism = sinkConfig.getInt(CommonOptions.PARALLELISM.key()); dataStreamSink.setParallelism(parallelism); diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractFlinkRuntimeEnvironment.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractFlinkRuntimeEnvironment.java index d805c286f86..34d91842771 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractFlinkRuntimeEnvironment.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractFlinkRuntimeEnvironment.java @@ -27,7 +27,6 @@ import org.apache.seatunnel.core.starter.execution.RuntimeEnvironment; import org.apache.seatunnel.core.starter.flink.utils.ConfigKeyName; import org.apache.seatunnel.core.starter.flink.utils.EnvironmentUtil; -import org.apache.seatunnel.core.starter.flink.utils.TableUtil; import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.configuration.Configuration; @@ -37,11 +36,8 @@ import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.types.Row; import org.apache.flink.util.TernaryBoolean; import lombok.extern.slf4j.Slf4j; @@ -55,10 +51,8 @@ @Slf4j public abstract class AbstractFlinkRuntimeEnvironment implements RuntimeEnvironment { - protected static final String RESULT_TABLE_NAME = "result_table_name"; protected Config config; protected StreamExecutionEnvironment environment; - protected StreamTableEnvironment tableEnvironment; protected JobMode jobMode; protected String jobName = Constants.LOGO; @@ -78,10 +72,6 @@ public CheckResult checkConfig() { return EnvironmentUtil.checkRestartStrategy(config); } - public StreamTableEnvironment getStreamTableEnvironment() { - return tableEnvironment; - } - public StreamExecutionEnvironment getStreamExecutionEnvironment() { return environment; } @@ -228,14 +218,6 @@ private void setTimeCharacteristic() { } } - public void registerResultTable(Config config, DataStream dataStream, String name) { - StreamTableEnvironment tableEnvironment = this.getStreamTableEnvironment(); - if (!TableUtil.tableExists(tableEnvironment, name)) { - tableEnvironment.createTemporaryView( - name, tableEnvironment.fromChangelogStream(dataStream)); - } - } - public boolean isStreaming() { return JobMode.STREAMING.equals(jobMode); } diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/DataStreamTableInfo.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/DataStreamTableInfo.java index 7b158ee60b9..a80a09b5067 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/DataStreamTableInfo.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/DataStreamTableInfo.java @@ -18,20 +18,22 @@ package org.apache.seatunnel.core.starter.flink.execution; import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.types.Row; import lombok.AllArgsConstructor; import lombok.Data; +import java.util.List; + @Data @AllArgsConstructor public class DataStreamTableInfo { - private DataStream dataStream; + private DataStream dataStream; - private CatalogTable catalogTable; + private List catalogTables; private String tableName; } diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkAbstractPluginExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkAbstractPluginExecuteProcessor.java index 565b7379bf2..57956db56c1 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkAbstractPluginExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkAbstractPluginExecuteProcessor.java @@ -23,12 +23,6 @@ import org.apache.seatunnel.common.utils.ReflectionUtils; import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.core.starter.execution.PluginExecuteProcessor; -import org.apache.seatunnel.core.starter.flink.utils.TableUtil; - -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.types.Row; import java.net.URL; import java.net.URLClassLoader; @@ -36,8 +30,6 @@ import java.util.Optional; import java.util.function.BiConsumer; -import static org.apache.seatunnel.api.common.CommonOptions.RESULT_TABLE_NAME; - public abstract class FlinkAbstractPluginExecuteProcessor implements PluginExecuteProcessor { @@ -84,10 +76,7 @@ public void setRuntimeEnvironment(FlinkRuntimeEnvironment flinkRuntimeEnvironmen protected Optional fromSourceTable( Config pluginConfig, List upstreamDataStreams) { if (pluginConfig.hasPath(SOURCE_TABLE_NAME)) { - StreamTableEnvironment tableEnvironment = - flinkRuntimeEnvironment.getStreamTableEnvironment(); String tableName = pluginConfig.getString(SOURCE_TABLE_NAME); - Table table = tableEnvironment.from(tableName); DataStreamTableInfo dataStreamTableInfo = upstreamDataStreams.stream() .filter(info -> tableName.equals(info.getTableName())) @@ -99,20 +88,13 @@ protected Optional fromSourceTable( "table %s not found", tableName))); return Optional.of( new DataStreamTableInfo( - TableUtil.tableToDataStream(tableEnvironment, table), - dataStreamTableInfo.getCatalogTable(), + dataStreamTableInfo.getDataStream(), + dataStreamTableInfo.getCatalogTables(), tableName)); } return Optional.empty(); } - protected void registerResultTable(Config pluginConfig, DataStream dataStream) { - if (pluginConfig.hasPath(RESULT_TABLE_NAME.key())) { - String resultTable = pluginConfig.getString(RESULT_TABLE_NAME.key()); - flinkRuntimeEnvironment.registerResultTable(pluginConfig, dataStream, resultTable); - } - } - protected abstract List initializePlugins( List jarPaths, List pluginConfigs); } diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkRuntimeEnvironment.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkRuntimeEnvironment.java index c1de8ff4f71..e3428c751e6 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkRuntimeEnvironment.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkRuntimeEnvironment.java @@ -21,13 +21,6 @@ import org.apache.seatunnel.common.constants.JobMode; import org.apache.seatunnel.core.starter.execution.RuntimeEnvironment; -import org.apache.seatunnel.core.starter.flink.utils.ConfigKeyName; -import org.apache.seatunnel.core.starter.flink.utils.EnvironmentUtil; - -import org.apache.flink.api.common.time.Time; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableConfig; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import lombok.extern.slf4j.Slf4j; @@ -50,7 +43,6 @@ public FlinkRuntimeEnvironment setConfig(Config config) { @Override public FlinkRuntimeEnvironment prepare() { createStreamEnvironment(); - createStreamTableEnvironment(); if (config.hasPath("job.name")) { jobName = config.getString("job.name"); } @@ -63,23 +55,6 @@ public FlinkRuntimeEnvironment setJobMode(JobMode jobMode) { return this; } - private void createStreamTableEnvironment() { - EnvironmentSettings environmentSettings = - EnvironmentSettings.newInstance().inStreamingMode().build(); - tableEnvironment = - StreamTableEnvironment.create(getStreamExecutionEnvironment(), environmentSettings); - TableConfig config = tableEnvironment.getConfig(); - if (EnvironmentUtil.hasPathAndWaring(this.config, ConfigKeyName.MAX_STATE_RETENTION_TIME) - && EnvironmentUtil.hasPathAndWaring( - this.config, ConfigKeyName.MIN_STATE_RETENTION_TIME)) { - long max = this.config.getLong(ConfigKeyName.MAX_STATE_RETENTION_TIME); - long min = this.config.getLong(ConfigKeyName.MIN_STATE_RETENTION_TIME); - config.setIdleStateRetentionTime(Time.seconds(min), Time.seconds(max)); - } - // init flink table env config - EnvironmentUtil.initTableEnvironmentConfiguration(this.config, config.getConfiguration()); - } - public static FlinkRuntimeEnvironment getInstance(Config config) { if (INSTANCE == null) { synchronized (FlinkRuntimeEnvironment.class) { diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java index 14247464551..c713593821e 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java @@ -52,6 +52,7 @@ import static org.apache.seatunnel.api.common.CommonOptions.PLUGIN_NAME; import static org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode.HANDLE_SAVE_MODE_FAILED; +@SuppressWarnings("unchecked,rawtypes") public class SinkExecuteProcessor extends FlinkAbstractPluginExecuteProcessor> { @@ -108,12 +109,15 @@ public List execute(List upstreamDataS sinkConfig.getString(PLUGIN_NAME.key())), sinkConfig); sink.setJobContext(jobContext); - SeaTunnelRowType sourceType = stream.getCatalogTable().getSeaTunnelRowType(); + // TODO sink support multi table + SeaTunnelRowType sourceType = + stream.getCatalogTables().get(0).getSeaTunnelRowType(); sink.setTypeInfo(sourceType); } else { + // TODO sink support multi table TableSinkFactoryContext context = TableSinkFactoryContext.replacePlaceholderAndCreate( - stream.getCatalogTable(), + stream.getCatalogTables().get(0), ReadonlyConfig.fromConfig(sinkConfig), classLoader, ((TableSinkFactory) factory.get()) @@ -137,8 +141,9 @@ public List execute(List upstreamDataS stream.getDataStream() .sinkTo( SinkV1Adapter.wrap( - new FlinkSink<>(sink, stream.getCatalogTable()))) - .name(sink.getPluginName()); + new FlinkSink<>( + sink, stream.getCatalogTables().get(0)))) + .name(String.format("%s-Sink", sink.getPluginName())); if (sinkConfig.hasPath(CommonOptions.PARALLELISM.key())) { int parallelism = sinkConfig.getInt(CommonOptions.PARALLELISM.key()); dataStreamSink.setParallelism(parallelism); diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java index 20b74f4b71e..eeb757a8536 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.core.starter.enums.PluginType; import org.apache.seatunnel.core.starter.execution.PluginUtil; import org.apache.seatunnel.core.starter.execution.SourceTableInfo; @@ -71,21 +72,20 @@ public List execute(List upstreamDataS Config pluginConfig = pluginConfigs.get(i); FlinkSource flinkSource = new FlinkSource<>(internalSource, envConfig); - DataStreamSource sourceStream = + DataStreamSource sourceStream = executionEnvironment.fromSource( flinkSource, WatermarkStrategy.noWatermarks(), - String.format("%s-source", internalSource.getPluginName())); + String.format("%s-Source", internalSource.getPluginName())); if (pluginConfig.hasPath(CommonOptions.PARALLELISM.key())) { int parallelism = pluginConfig.getInt(CommonOptions.PARALLELISM.key()); sourceStream.setParallelism(parallelism); } - registerResultTable(pluginConfig, sourceStream); sources.add( new DataStreamTableInfo( sourceStream, - sourceTableInfo.getCatalogTables().get(0), + sourceTableInfo.getCatalogTables(), pluginConfig.hasPath(RESULT_TABLE_NAME.key()) ? pluginConfig.getString(RESULT_TABLE_NAME.key()) : null)); diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java index 1ff2cf64372..c92eaf42a9a 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java @@ -25,19 +25,15 @@ import org.apache.seatunnel.api.table.factory.TableTransformFactory; import org.apache.seatunnel.api.table.factory.TableTransformFactoryContext; import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.api.transform.SeaTunnelTransform; import org.apache.seatunnel.core.starter.exception.TaskExecuteException; import org.apache.seatunnel.core.starter.execution.PluginUtil; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelFactoryDiscovery; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelTransformPluginDiscovery; -import org.apache.seatunnel.translation.flink.serialization.FlinkRowConverter; -import org.apache.seatunnel.translation.flink.utils.TypeConverterUtils; -import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.types.Row; +import org.apache.flink.streaming.api.operators.StreamMap; import java.net.URL; import java.util.Collections; @@ -47,6 +43,7 @@ import static org.apache.seatunnel.api.common.CommonOptions.RESULT_TABLE_NAME; +@SuppressWarnings("unchecked,rawtypes") public class TransformExecuteProcessor extends FlinkAbstractPluginExecuteProcessor { @@ -97,21 +94,20 @@ public List execute(List upstreamDataS TableTransformFactory factory = plugins.get(i); TableTransformFactoryContext context = new TableTransformFactoryContext( - Collections.singletonList(stream.getCatalogTable()), + stream.getCatalogTables(), ReadonlyConfig.fromConfig(pluginConfig), classLoader); ConfigValidator.of(context.getOptions()).validate(factory.optionRule()); SeaTunnelTransform transform = factory.createTransform(context).createTransform(); - SeaTunnelRowType sourceType = stream.getCatalogTable().getSeaTunnelRowType(); transform.setJobContext(jobContext); - DataStream inputStream = - flinkTransform(sourceType, transform, stream.getDataStream()); - registerResultTable(pluginConfig, inputStream); + DataStream inputStream = + flinkTransform(transform, stream.getDataStream()); + // TODO transform support multi tables upstreamDataStreams.add( new DataStreamTableInfo( inputStream, - transform.getProducedCatalogTable(), + Collections.singletonList(transform.getProducedCatalogTable()), pluginConfig.hasPath(RESULT_TABLE_NAME.key()) ? pluginConfig.getString(RESULT_TABLE_NAME.key()) : null)); @@ -126,28 +122,17 @@ public List execute(List upstreamDataS return upstreamDataStreams; } - protected DataStream flinkTransform( - SeaTunnelRowType sourceType, SeaTunnelTransform transform, DataStream stream) { - TypeInformation rowTypeInfo = - TypeConverterUtils.convert( - transform.getProducedCatalogTable().getSeaTunnelRowType()); - FlinkRowConverter transformInputRowConverter = new FlinkRowConverter(sourceType); - FlinkRowConverter transformOutputRowConverter = - new FlinkRowConverter(transform.getProducedCatalogTable().getSeaTunnelRowType()); - DataStream output = - stream.flatMap( - (FlatMapFunction) - (value, out) -> { - SeaTunnelRow seaTunnelRow = - transformInputRowConverter.reconvert(value); - SeaTunnelRow dataRow = - (SeaTunnelRow) transform.map(seaTunnelRow); - if (dataRow != null) { - Row copy = transformOutputRowConverter.convert(dataRow); - out.collect(copy); - } - }, - rowTypeInfo); - return output; + protected DataStream flinkTransform( + SeaTunnelTransform transform, DataStream stream) { + return stream.transform( + String.format("%s-Transform", transform.getPluginName()), + TypeInformation.of(SeaTunnelRow.class), + new StreamMap<>( + flinkRuntimeEnvironment + .getStreamExecutionEnvironment() + .clean( + row -> + ((SeaTunnelTransform) transform) + .map(row)))); } } diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/UnifyEnvParameterIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/UnifyEnvParameterIT.java index ad5c6365f2f..48485eb69c3 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/UnifyEnvParameterIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/UnifyEnvParameterIT.java @@ -126,22 +126,6 @@ public void testUnifiedFlinkTableEnvParam(AbstractTestFlinkContainer container) } Assertions.assertNotNull(jobInfoReference.get()); }); - Map jobInfo = jobInfoReference.get(); - - /** - * 'table.exec.resource.default-parallelism' has a higher priority than 'parallelism', so - * one of these nodes must have a parallelism of 2. - */ - Map plan = (Map) jobInfo.get("plan"); - List> nodes = (List>) plan.get("nodes"); - boolean tableExecParallelism = false; - for (Map node : nodes) { - int parallelism = (int) node.get("parallelism"); - if (!tableExecParallelism && parallelism == 2) { - tableExecParallelism = true; - } - } - Assertions.assertTrue(tableExecParallelism); } public void genericTest(String configPath, AbstractTestFlinkContainer container) diff --git a/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml b/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml index 99c75d324a8..ef801bdb9c0 100644 --- a/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml +++ b/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml @@ -112,6 +112,12 @@ ${flink.1.15.3.version} + + org.apache.flink + flink-runtime-web + ${flink.1.15.3.version} + + com.squareup.okhttp3 mockwebserver diff --git a/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/resources/examples/fake_to_console.conf b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/resources/examples/fake_to_console.conf index 799dac79960..12c1f9f2811 100644 --- a/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/resources/examples/fake_to_console.conf +++ b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/resources/examples/fake_to_console.conf @@ -19,7 +19,7 @@ ###### env { - job.mode = "BATCH" + job.mode = "STREAMING" parallelism = 2 } @@ -41,15 +41,21 @@ source { } transform { - + Copy { + source_table_name = "fake" + result_table_name = "fake1" + fields { + name1 = name + } + } # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, # please go to https://seatunnel.apache.org/docs/category/transform-v2 } sink { Console { + source_table_name = "fake1" } - # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, # please go to https://seatunnel.apache.org/docs/category/sink-v2 } diff --git a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java index d4bd43c3d1c..4b62895f18c 100644 --- a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java +++ b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java @@ -66,6 +66,7 @@ import java.util.stream.Collectors; @Slf4j +@SuppressWarnings("unchecked") public abstract class AbstractPluginDiscovery implements PluginDiscovery { private static final String PLUGIN_MAPPING_FILE = "plugin-mapping.properties"; diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-13/src/test/java/org/apache/seatunnel/translation/flink/utils/TypeConverterUtilsTest.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-13/src/test/java/org/apache/seatunnel/translation/flink/utils/TypeConverterUtilsTest.java deleted file mode 100644 index 95cfa335e7d..00000000000 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-13/src/test/java/org/apache/seatunnel/translation/flink/utils/TypeConverterUtilsTest.java +++ /dev/null @@ -1,161 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.translation.flink.utils; - -import org.apache.seatunnel.api.table.type.ArrayType; -import org.apache.seatunnel.api.table.type.BasicType; -import org.apache.seatunnel.api.table.type.DecimalType; - -import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -public class TypeConverterUtilsTest { - // -------------------------------------------------------------- - // basic types test - // -------------------------------------------------------------- - - @Test - public void convertStringType() { - Assertions.assertEquals( - BasicTypeInfo.STRING_TYPE_INFO, TypeConverterUtils.convert(BasicType.STRING_TYPE)); - } - - @Test - public void convertIntegerType() { - Assertions.assertEquals( - BasicTypeInfo.INT_TYPE_INFO, TypeConverterUtils.convert(BasicType.INT_TYPE)); - } - - @Test - public void convertBooleanType() { - Assertions.assertEquals( - BasicTypeInfo.BOOLEAN_TYPE_INFO, - TypeConverterUtils.convert(BasicType.BOOLEAN_TYPE)); - } - - @Test - public void convertDoubleType() { - Assertions.assertEquals( - BasicTypeInfo.DOUBLE_TYPE_INFO, TypeConverterUtils.convert(BasicType.DOUBLE_TYPE)); - } - - @Test - public void convertLongType() { - Assertions.assertEquals( - BasicTypeInfo.LONG_TYPE_INFO, TypeConverterUtils.convert(BasicType.LONG_TYPE)); - } - - @Test - public void convertFloatType() { - Assertions.assertEquals( - BasicTypeInfo.FLOAT_TYPE_INFO, TypeConverterUtils.convert(BasicType.FLOAT_TYPE)); - } - - @Test - public void convertByteType() { - Assertions.assertEquals( - BasicTypeInfo.BYTE_TYPE_INFO, TypeConverterUtils.convert(BasicType.BYTE_TYPE)); - } - - @Test - public void convertShortType() { - Assertions.assertEquals( - BasicTypeInfo.SHORT_TYPE_INFO, TypeConverterUtils.convert(BasicType.SHORT_TYPE)); - } - - @Test - public void convertBigDecimalType() { - /** - * To solve lost precision and scale of {@link - * org.apache.seatunnel.api.table.type.DecimalType}, use {@link - * org.apache.flink.api.common.typeinfo.BasicTypeInfo#STRING_TYPE_INFO} as the convert - * result of {@link org.apache.seatunnel.api.table.type.DecimalType} instance. - */ - Assertions.assertEquals( - BasicTypeInfo.STRING_TYPE_INFO, TypeConverterUtils.convert(new DecimalType(30, 2))); - } - - @Test - public void convertNullType() { - Assertions.assertEquals( - BasicTypeInfo.VOID_TYPE_INFO, TypeConverterUtils.convert(BasicType.VOID_TYPE)); - } - - // -------------------------------------------------------------- - // array types test - // -------------------------------------------------------------- - - @Test - public void convertBooleanArrayType() { - Assertions.assertEquals( - BasicArrayTypeInfo.BOOLEAN_ARRAY_TYPE_INFO, - TypeConverterUtils.convert(ArrayType.BOOLEAN_ARRAY_TYPE)); - } - - @Test - public void convertStringArrayType() { - Assertions.assertEquals( - BasicArrayTypeInfo.STRING_ARRAY_TYPE_INFO, - TypeConverterUtils.convert(ArrayType.STRING_ARRAY_TYPE)); - } - - @Test - public void convertDoubleArrayType() { - Assertions.assertEquals( - BasicArrayTypeInfo.DOUBLE_ARRAY_TYPE_INFO, - TypeConverterUtils.convert(ArrayType.DOUBLE_ARRAY_TYPE)); - } - - @Test - public void convertIntegerArrayType() { - Assertions.assertEquals( - BasicArrayTypeInfo.INT_ARRAY_TYPE_INFO, - TypeConverterUtils.convert(ArrayType.INT_ARRAY_TYPE)); - } - - @Test - public void convertLongArrayType() { - Assertions.assertEquals( - BasicArrayTypeInfo.LONG_ARRAY_TYPE_INFO, - TypeConverterUtils.convert(ArrayType.LONG_ARRAY_TYPE)); - } - - @Test - public void convertFloatArrayType() { - Assertions.assertEquals( - BasicArrayTypeInfo.FLOAT_ARRAY_TYPE_INFO, - TypeConverterUtils.convert(ArrayType.FLOAT_ARRAY_TYPE)); - } - - @Test - public void convertByteArrayType() { - Assertions.assertEquals( - BasicArrayTypeInfo.BYTE_ARRAY_TYPE_INFO, - TypeConverterUtils.convert(ArrayType.BYTE_ARRAY_TYPE)); - } - - @Test - public void convertShortArrayType() { - Assertions.assertEquals( - BasicArrayTypeInfo.SHORT_ARRAY_TYPE_INFO, - TypeConverterUtils.convert(ArrayType.SHORT_ARRAY_TYPE)); - } -} diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/serialization/FlinkRowConverter.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/serialization/FlinkRowConverter.java deleted file mode 100644 index b24cb96dfef..00000000000 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/serialization/FlinkRowConverter.java +++ /dev/null @@ -1,154 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.translation.flink.serialization; - -import org.apache.seatunnel.api.table.type.DecimalType; -import org.apache.seatunnel.api.table.type.MapType; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.api.table.type.SqlType; -import org.apache.seatunnel.translation.serialization.RowConverter; - -import org.apache.flink.types.Row; -import org.apache.flink.types.RowKind; - -import lombok.extern.slf4j.Slf4j; - -import java.io.IOException; -import java.math.BigDecimal; -import java.math.RoundingMode; -import java.util.HashMap; -import java.util.Map; -import java.util.function.BiFunction; - -/** - * The row converter between {@link Row} and {@link SeaTunnelRow}, used to convert or reconvert - * between flink row and seatunnel row - */ -@Slf4j -public class FlinkRowConverter extends RowConverter { - - public FlinkRowConverter(SeaTunnelDataType dataType) { - super(dataType); - } - - @Override - public Row convert(SeaTunnelRow seaTunnelRow) throws IOException { - validate(seaTunnelRow); - return (Row) convert(seaTunnelRow, dataType); - } - - private static Object convert(Object field, SeaTunnelDataType dataType) { - if (field == null) { - return null; - } - SqlType sqlType = dataType.getSqlType(); - switch (sqlType) { - case ROW: - SeaTunnelRow seaTunnelRow = (SeaTunnelRow) field; - SeaTunnelRowType rowType = (SeaTunnelRowType) dataType; - int arity = rowType.getTotalFields(); - Row engineRow = new Row(arity); - for (int i = 0; i < arity; i++) { - engineRow.setField( - i, convert(seaTunnelRow.getField(i), rowType.getFieldType(i))); - } - engineRow.setKind(RowKind.fromByteValue(seaTunnelRow.getRowKind().toByteValue())); - return engineRow; - case MAP: - return convertMap( - (Map) field, (MapType) dataType, FlinkRowConverter::convert); - - /** - * To solve lost precision and scale of {@link - * org.apache.seatunnel.api.table.type.DecimalType}, use {@link java.lang.String} as - * the convert result of {@link java.math.BigDecimal} instance. - */ - case DECIMAL: - BigDecimal decimal = (BigDecimal) field; - return decimal.toString(); - default: - return field; - } - } - - private static Object convertMap( - Map mapData, - MapType mapType, - BiFunction, Object> convertFunction) { - if (mapData == null || mapData.isEmpty()) { - return mapData; - } - - Map newMap = new HashMap<>(mapData.size()); - mapData.forEach( - (key, value) -> { - SeaTunnelDataType keyType = mapType.getKeyType(); - SeaTunnelDataType valueType = mapType.getValueType(); - newMap.put( - convertFunction.apply(key, keyType), - convertFunction.apply(value, valueType)); - }); - return newMap; - } - - @Override - public SeaTunnelRow reconvert(Row engineRow) throws IOException { - return (SeaTunnelRow) reconvert(engineRow, dataType); - } - - private static Object reconvert(Object field, SeaTunnelDataType dataType) { - if (field == null) { - return null; - } - SqlType sqlType = dataType.getSqlType(); - switch (sqlType) { - case ROW: - Row engineRow = (Row) field; - SeaTunnelRowType rowType = (SeaTunnelRowType) dataType; - int arity = rowType.getTotalFields(); - SeaTunnelRow seaTunnelRow = new SeaTunnelRow(arity); - for (int i = 0; i < arity; i++) { - seaTunnelRow.setField( - i, reconvert(engineRow.getField(i), rowType.getFieldType(i))); - } - seaTunnelRow.setRowKind( - org.apache.seatunnel.api.table.type.RowKind.fromByteValue( - engineRow.getKind().toByteValue())); - return seaTunnelRow; - case MAP: - return convertMap( - (Map) field, (MapType) dataType, FlinkRowConverter::reconvert); - - /** - * To solve lost precision and scale of {@link - * org.apache.seatunnel.api.table.type.DecimalType}, create {@link - * java.math.BigDecimal} instance from {@link java.lang.String} type field. - */ - case DECIMAL: - DecimalType decimalType = (DecimalType) dataType; - String decimalData = (String) field; - BigDecimal decimal = new BigDecimal(decimalData); - decimal.setScale(decimalType.getScale(), RoundingMode.HALF_UP); - return decimal; - default: - return field; - } - } -} diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java index 3c949f64802..725bf606f93 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java @@ -25,11 +25,9 @@ import org.apache.seatunnel.api.sink.SupportResourceShare; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.translation.flink.serialization.FlinkRowConverter; import org.apache.flink.api.connector.sink.Sink; import org.apache.flink.api.connector.sink.SinkWriter; -import org.apache.flink.types.Row; import lombok.extern.slf4j.Slf4j; @@ -54,7 +52,6 @@ public class FlinkSinkWriter private final org.apache.seatunnel.api.sink.SinkWriter sinkWriter; - private final FlinkRowConverter rowSerialization; private final Counter sinkWriteCount; @@ -73,7 +70,6 @@ public class FlinkSinkWriter MetricsContext metricsContext) { this.sinkWriter = sinkWriter; this.checkpointId = checkpointId; - this.rowSerialization = new FlinkRowConverter(dataType); this.sinkWriteCount = metricsContext.counter(MetricNames.SINK_WRITE_COUNT); this.sinkWriteBytes = metricsContext.counter(MetricNames.SINK_WRITE_BYTES); this.sinkWriterQPS = metricsContext.meter(MetricNames.SINK_WRITE_QPS); @@ -86,15 +82,17 @@ public class FlinkSinkWriter @Override public void write(InputT element, SinkWriter.Context context) throws IOException { - if (element instanceof Row) { - SeaTunnelRow seaTunnelRow = rowSerialization.reconvert((Row) element); - sinkWriter.write(seaTunnelRow); + if (element == null) { + return; + } + if (element instanceof SeaTunnelRow) { + sinkWriter.write((SeaTunnelRow) element); sinkWriteCount.inc(); - sinkWriteBytes.inc(seaTunnelRow.getBytesSize()); + sinkWriteBytes.inc(((SeaTunnelRow) element).getBytesSize()); sinkWriterQPS.markEvent(); } else { throw new InvalidClassException( - "only support Flink Row at now, the element Class is " + element.getClass()); + "only support SeaTunnelRow at now, the element Class is " + element.getClass()); } } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkRowCollector.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkRowCollector.java index 39b14d17d03..2ea584029e5 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkRowCollector.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkRowCollector.java @@ -25,26 +25,18 @@ import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.core.starter.flowcontrol.FlowControlGate; import org.apache.seatunnel.core.starter.flowcontrol.FlowControlStrategy; -import org.apache.seatunnel.translation.flink.serialization.FlinkRowConverter; import org.apache.flink.api.connector.source.ReaderOutput; -import org.apache.flink.types.Row; import lombok.extern.slf4j.Slf4j; -/** - * The implementation of {@link Collector} for flink engine, as a container for {@link SeaTunnelRow} - * and convert {@link SeaTunnelRow} to {@link Row}. - */ +/** The implementation of {@link Collector} for flink engine. */ @Slf4j public class FlinkRowCollector implements Collector { - private ReaderOutput readerOutput; - - private final FlinkRowConverter rowSerialization; + private ReaderOutput readerOutput; private final FlowControlGate flowControlGate; @@ -54,9 +46,7 @@ public class FlinkRowCollector implements Collector { private final Meter sourceReadQPS; - public FlinkRowCollector( - SeaTunnelRowType seaTunnelRowType, Config envConfig, MetricsContext metricsContext) { - this.rowSerialization = new FlinkRowConverter(seaTunnelRowType); + public FlinkRowCollector(Config envConfig, MetricsContext metricsContext) { this.flowControlGate = FlowControlGate.create(FlowControlStrategy.fromConfig(envConfig)); this.sourceReadCount = metricsContext.counter(MetricNames.SOURCE_RECEIVED_COUNT); this.sourceReadBytes = metricsContext.counter(MetricNames.SOURCE_RECEIVED_BYTES); @@ -67,8 +57,7 @@ public FlinkRowCollector( public void collect(SeaTunnelRow record) { flowControlGate.audit(record); try { - Row row = rowSerialization.convert(record); - readerOutput.collect(row); + readerOutput.collect(record); sourceReadCount.inc(); sourceReadBytes.inc(record.getBytesSize()); sourceReadQPS.markEvent(); @@ -82,7 +71,7 @@ public Object getCheckpointLock() { return this; } - public FlinkRowCollector withReaderOutput(ReaderOutput readerOutput) { + public FlinkRowCollector withReaderOutput(ReaderOutput readerOutput) { this.readerOutput = readerOutput; return this; } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSource.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSource.java index adf54eef4c5..7868e6d3efd 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSource.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSource.java @@ -24,9 +24,7 @@ import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.translation.flink.serialization.FlinkSimpleVersionedSerializer; -import org.apache.seatunnel.translation.flink.utils.TypeConverterUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; @@ -37,7 +35,6 @@ import org.apache.flink.api.connector.source.SplitEnumeratorContext; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.types.Row; import java.io.Serializable; @@ -48,7 +45,8 @@ * @param The generic type of enumerator state */ public class FlinkSource - implements Source, EnumStateT>, ResultTypeQueryable { + implements Source, EnumStateT>, + ResultTypeQueryable { private final SeaTunnelSource source; @@ -68,14 +66,13 @@ public Boundedness getBoundedness() { } @Override - public SourceReader> createReader(SourceReaderContext readerContext) - throws Exception { + public SourceReader> createReader( + SourceReaderContext readerContext) throws Exception { org.apache.seatunnel.api.source.SourceReader.Context context = new FlinkSourceReaderContext(readerContext, source); org.apache.seatunnel.api.source.SourceReader reader = source.createReader(context); - return new FlinkSourceReader<>( - reader, context, envConfig, (SeaTunnelRowType) source.getProducedType()); + return new FlinkSourceReader<>(reader, context, envConfig); } @Override @@ -110,7 +107,7 @@ public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() } @Override - public TypeInformation getProducedType() { - return (TypeInformation) TypeConverterUtils.convert(source.getProducedType()); + public TypeInformation getProducedType() { + return TypeInformation.of(SeaTunnelRow.class); } } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java index 65dc4324779..c2f9cde5005 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java @@ -21,13 +21,11 @@ import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.core.io.InputStatus; -import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,7 +41,7 @@ * @param */ public class FlinkSourceReader - implements SourceReader> { + implements SourceReader> { private final Logger LOGGER = LoggerFactory.getLogger(FlinkSourceReader.class); @@ -58,12 +56,10 @@ public class FlinkSourceReader public FlinkSourceReader( org.apache.seatunnel.api.source.SourceReader sourceReader, org.apache.seatunnel.api.source.SourceReader.Context context, - Config envConfig, - SeaTunnelRowType seaTunnelRowType) { + Config envConfig) { this.sourceReader = sourceReader; this.context = context; - this.flinkRowCollector = - new FlinkRowCollector(seaTunnelRowType, envConfig, context.getMetricsContext()); + this.flinkRowCollector = new FlinkRowCollector(envConfig, context.getMetricsContext()); } @Override @@ -76,7 +72,7 @@ public void start() { } @Override - public InputStatus pollNext(ReaderOutput output) throws Exception { + public InputStatus pollNext(ReaderOutput output) throws Exception { if (!((FlinkSourceReaderContext) context).isSendNoMoreElementEvent()) { sourceReader.pollNext(flinkRowCollector.withReaderOutput(output)); } else { diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/utils/TypeConverterUtils.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/utils/TypeConverterUtils.java deleted file mode 100644 index ebb77da2688..00000000000 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/utils/TypeConverterUtils.java +++ /dev/null @@ -1,210 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.translation.flink.utils; - -import org.apache.seatunnel.api.table.type.ArrayType; -import org.apache.seatunnel.api.table.type.BasicType; -import org.apache.seatunnel.api.table.type.DecimalType; -import org.apache.seatunnel.api.table.type.LocalTimeType; -import org.apache.seatunnel.api.table.type.MapType; -import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; - -import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.LocalTimeTypeInfo; -import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.MapTypeInfo; -import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; -import org.apache.flink.api.java.typeutils.RowTypeInfo; - -import java.math.BigDecimal; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; - -public class TypeConverterUtils { - - private static final Map, BridgedType> BRIDGED_TYPES = new HashMap<>(32); - - static { - // basic types - BRIDGED_TYPES.put( - String.class, - BridgedType.of(BasicType.STRING_TYPE, BasicTypeInfo.STRING_TYPE_INFO)); - BRIDGED_TYPES.put( - Boolean.class, - BridgedType.of(BasicType.BOOLEAN_TYPE, BasicTypeInfo.BOOLEAN_TYPE_INFO)); - BRIDGED_TYPES.put( - Byte.class, BridgedType.of(BasicType.BYTE_TYPE, BasicTypeInfo.BYTE_TYPE_INFO)); - BRIDGED_TYPES.put( - Short.class, BridgedType.of(BasicType.SHORT_TYPE, BasicTypeInfo.SHORT_TYPE_INFO)); - BRIDGED_TYPES.put( - Integer.class, BridgedType.of(BasicType.INT_TYPE, BasicTypeInfo.INT_TYPE_INFO)); - BRIDGED_TYPES.put( - Long.class, BridgedType.of(BasicType.LONG_TYPE, BasicTypeInfo.LONG_TYPE_INFO)); - BRIDGED_TYPES.put( - Float.class, BridgedType.of(BasicType.FLOAT_TYPE, BasicTypeInfo.FLOAT_TYPE_INFO)); - BRIDGED_TYPES.put( - Double.class, - BridgedType.of(BasicType.DOUBLE_TYPE, BasicTypeInfo.DOUBLE_TYPE_INFO)); - BRIDGED_TYPES.put( - Void.class, BridgedType.of(BasicType.VOID_TYPE, BasicTypeInfo.VOID_TYPE_INFO)); - /** - * To solve lost precision and scale of {@link - * org.apache.seatunnel.api.table.type.DecimalType}, use {@link - * org.apache.flink.api.common.typeinfo.BasicTypeInfo#STRING_TYPE_INFO} as the payload of - * {@link org.apache.seatunnel.api.table.type.DecimalType}. - */ - BRIDGED_TYPES.put( - BigDecimal.class, - BridgedType.of(new DecimalType(38, 18), BasicTypeInfo.STRING_TYPE_INFO)); - - // data time types - BRIDGED_TYPES.put( - LocalDate.class, - BridgedType.of(LocalTimeType.LOCAL_DATE_TYPE, LocalTimeTypeInfo.LOCAL_DATE)); - BRIDGED_TYPES.put( - LocalTime.class, - BridgedType.of(LocalTimeType.LOCAL_TIME_TYPE, LocalTimeTypeInfo.LOCAL_TIME)); - BRIDGED_TYPES.put( - LocalDateTime.class, - BridgedType.of( - LocalTimeType.LOCAL_DATE_TIME_TYPE, LocalTimeTypeInfo.LOCAL_DATE_TIME)); - // basic array types - BRIDGED_TYPES.put( - byte[].class, - BridgedType.of( - PrimitiveByteArrayType.INSTANCE, - PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO)); - BRIDGED_TYPES.put( - String[].class, - BridgedType.of( - ArrayType.STRING_ARRAY_TYPE, BasicArrayTypeInfo.STRING_ARRAY_TYPE_INFO)); - BRIDGED_TYPES.put( - Boolean[].class, - BridgedType.of( - ArrayType.BOOLEAN_ARRAY_TYPE, BasicArrayTypeInfo.BOOLEAN_ARRAY_TYPE_INFO)); - BRIDGED_TYPES.put( - Byte[].class, - BridgedType.of(ArrayType.BYTE_ARRAY_TYPE, BasicArrayTypeInfo.BYTE_ARRAY_TYPE_INFO)); - BRIDGED_TYPES.put( - Short[].class, - BridgedType.of( - ArrayType.SHORT_ARRAY_TYPE, BasicArrayTypeInfo.SHORT_ARRAY_TYPE_INFO)); - BRIDGED_TYPES.put( - Integer[].class, - BridgedType.of(ArrayType.INT_ARRAY_TYPE, BasicArrayTypeInfo.INT_ARRAY_TYPE_INFO)); - BRIDGED_TYPES.put( - Long[].class, - BridgedType.of(ArrayType.LONG_ARRAY_TYPE, BasicArrayTypeInfo.LONG_ARRAY_TYPE_INFO)); - BRIDGED_TYPES.put( - Float[].class, - BridgedType.of( - ArrayType.FLOAT_ARRAY_TYPE, BasicArrayTypeInfo.FLOAT_ARRAY_TYPE_INFO)); - BRIDGED_TYPES.put( - Double[].class, - BridgedType.of( - ArrayType.DOUBLE_ARRAY_TYPE, BasicArrayTypeInfo.DOUBLE_ARRAY_TYPE_INFO)); - } - - private TypeConverterUtils() { - throw new UnsupportedOperationException( - "TypeConverterUtils is a utility class and cannot be instantiated"); - } - - public static SeaTunnelDataType convert(TypeInformation dataType) { - BridgedType bridgedType = BRIDGED_TYPES.get(dataType.getTypeClass()); - if (bridgedType != null) { - return bridgedType.getSeaTunnelType(); - } - - if (dataType instanceof MapTypeInfo) { - MapTypeInfo mapTypeInfo = (MapTypeInfo) dataType; - return new MapType<>( - convert(mapTypeInfo.getKeyTypeInfo()), convert(mapTypeInfo.getValueTypeInfo())); - } - if (dataType instanceof RowTypeInfo) { - RowTypeInfo typeInformation = (RowTypeInfo) dataType; - String[] fieldNames = typeInformation.getFieldNames(); - SeaTunnelDataType[] seaTunnelDataTypes = - Arrays.stream(typeInformation.getFieldTypes()) - .map(TypeConverterUtils::convert) - .toArray(SeaTunnelDataType[]::new); - return new SeaTunnelRowType(fieldNames, seaTunnelDataTypes); - } - throw new IllegalArgumentException("Unsupported Flink's data type: " + dataType); - } - - public static TypeInformation convert(SeaTunnelDataType dataType) { - BridgedType bridgedType = BRIDGED_TYPES.get(dataType.getTypeClass()); - if (bridgedType != null) { - return bridgedType.getFlinkType(); - } - - if (dataType instanceof MapType) { - MapType mapType = (MapType) dataType; - return new MapTypeInfo<>( - convert(mapType.getKeyType()), convert(mapType.getValueType())); - } - - if (dataType instanceof ArrayType) { - ArrayType arrayType = (ArrayType) dataType; - return ObjectArrayTypeInfo.getInfoFor( - arrayType.getTypeClass(), convert(arrayType.getElementType())); - } - - if (dataType instanceof SeaTunnelRowType) { - SeaTunnelRowType rowType = (SeaTunnelRowType) dataType; - TypeInformation[] types = - Arrays.stream(rowType.getFieldTypes()) - .map(TypeConverterUtils::convert) - .toArray(TypeInformation[]::new); - return new RowTypeInfo(types, rowType.getFieldNames()); - } - throw new IllegalArgumentException("Unsupported SeaTunnel's data type: " + dataType); - } - - public static class BridgedType { - private final SeaTunnelDataType seaTunnelType; - private final TypeInformation flinkType; - - private BridgedType(SeaTunnelDataType seaTunnelType, TypeInformation flinkType) { - this.seaTunnelType = seaTunnelType; - this.flinkType = flinkType; - } - - public static BridgedType of( - SeaTunnelDataType seaTunnelType, TypeInformation flinkType) { - return new BridgedType(seaTunnelType, flinkType); - } - - public TypeInformation getFlinkType() { - return flinkType; - } - - public SeaTunnelDataType getSeaTunnelType() { - return seaTunnelType; - } - } -} From fa34ac98b42cd2bf52837efa27382cfe24c655cd Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Mon, 12 Aug 2024 13:35:43 +0800 Subject: [PATCH 047/361] [Improve][API] Check catalog table fields name legal before send to downstream (#7358) * [Improve][API] Check catalog table fields name legal before send to downstream * update --- .github/workflows/backend.yml | 38 ++++++++++++ .../table/factory/TableFactoryContext.java | 28 +++++++++ .../factory/TableSinkFactoryContext.java | 8 ++- .../factory/TableTransformFactoryContext.java | 1 + .../api/table/catalog/CatalogTableTest.java | 62 +++++++++++++++++++ 5 files changed, 136 insertions(+), 1 deletion(-) diff --git a/.github/workflows/backend.yml b/.github/workflows/backend.yml index 88a2d59e3f1..81222695a38 100644 --- a/.github/workflows/backend.yml +++ b/.github/workflows/backend.yml @@ -553,6 +553,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run seatunnel zeta integration test if: needs.changes.outputs.api == 'true' run: | @@ -609,6 +611,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run transform-v2 integration test (part-1) if: needs.changes.outputs.api == 'true' run: | @@ -633,6 +637,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run transform-v2 integration test (part-2) if: needs.changes.outputs.api == 'true' run: | @@ -657,6 +663,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run connector-v2 integration test (part-1) if: needs.changes.outputs.api == 'true' run: | @@ -684,6 +692,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run connector-v2 integration test (part-2) if: needs.changes.outputs.api == 'true' run: | @@ -711,6 +721,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run connector-v2 integration test (part-3) if: needs.changes.outputs.api == 'true' run: | @@ -738,6 +750,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run connector-v2 integration test (part-4) if: needs.changes.outputs.api == 'true' run: | @@ -765,6 +779,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run connector-v2 integration test (part-5) if: needs.changes.outputs.api == 'true' run: | @@ -792,6 +808,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run connector-v2 integration test (part-6) if: needs.changes.outputs.api == 'true' run: | @@ -819,6 +837,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run connector-v2 integration test (part-7) if: needs.changes.outputs.api == 'true' run: | @@ -898,6 +918,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run jdbc connectors integration test (part-3) if: needs.changes.outputs.api == 'true' run: | @@ -922,6 +944,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run jdbc connectors integration test (part-4) if: needs.changes.outputs.api == 'true' run: | @@ -946,6 +970,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run jdbc connectors integration test (part-5) if: needs.changes.outputs.api == 'true' run: | @@ -996,6 +1022,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run jdbc connectors integration test (part-7) if: needs.changes.outputs.api == 'true' run: | @@ -1020,6 +1048,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run kudu connector integration test run: | ./mvnw -B -T 1 verify -DskipUT=true -DskipIT=false -D"license.skipAddThirdParty"=true --no-snapshot-updates -pl :connector-kudu-e2e -am -Pci @@ -1043,6 +1073,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run amazonsqs connector integration test run: | ./mvnw -B -T 1 verify -DskipUT=true -DskipIT=false -D"license.skipAddThirdParty"=true --no-snapshot-updates -pl :connector-amazonsqs-e2e -am -Pci @@ -1066,6 +1098,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run kafka connector integration test run: | ./mvnw -B -T 1 verify -DskipUT=true -DskipIT=false -D"license.skipAddThirdParty"=true --no-snapshot-updates -pl :connector-kafka-e2e -am -Pci @@ -1089,6 +1123,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run rocket connector integration test run: | ./mvnw -B -T 1 verify -DskipUT=true -DskipIT=false -D"license.skipAddThirdParty"=true --no-snapshot-updates -pl :connector-rocketmq-e2e -am -Pci @@ -1139,6 +1175,8 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' + - name: free disk space + run: tools/github/free_disk_space.sh - name: run oracle cdc connector integration test run: | ./mvnw -B -T 1 verify -DskipUT=true -DskipIT=false -D"license.skipAddThirdParty"=true --no-snapshot-updates -pl :connector-cdc-oracle-e2e -am -Pci diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableFactoryContext.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableFactoryContext.java index 10436da09b8..5664e48b4e6 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableFactoryContext.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableFactoryContext.java @@ -18,9 +18,16 @@ package org.apache.seatunnel.api.table.factory; import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.common.utils.SeaTunnelException; + +import org.apache.commons.lang3.StringUtils; import lombok.Getter; +import java.util.ArrayList; +import java.util.List; + @Getter public abstract class TableFactoryContext { @@ -31,4 +38,25 @@ public TableFactoryContext(ReadonlyConfig options, ClassLoader classLoader) { this.options = options; this.classLoader = classLoader; } + + protected static void checkCatalogTableIllegal(List catalogTables) { + for (CatalogTable catalogTable : catalogTables) { + List alreadyChecked = new ArrayList<>(); + for (String fieldName : catalogTable.getTableSchema().getFieldNames()) { + if (StringUtils.isBlank(fieldName)) { + throw new SeaTunnelException( + String.format( + "Table %s field name cannot be empty", + catalogTable.getTablePath().getFullName())); + } + if (alreadyChecked.contains(fieldName)) { + throw new SeaTunnelException( + String.format( + "Table %s field %s duplicate", + catalogTable.getTablePath().getFullName(), fieldName)); + } + alreadyChecked.add(fieldName); + } + } + } } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableSinkFactoryContext.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableSinkFactoryContext.java index 9565bad6a03..3e0eb24cd59 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableSinkFactoryContext.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableSinkFactoryContext.java @@ -21,18 +21,24 @@ import org.apache.seatunnel.api.sink.TablePlaceholder; import org.apache.seatunnel.api.table.catalog.CatalogTable; +import com.google.common.annotations.VisibleForTesting; import lombok.Getter; import java.util.Collection; +import java.util.Collections; @Getter public class TableSinkFactoryContext extends TableFactoryContext { private final CatalogTable catalogTable; - protected TableSinkFactoryContext( + @VisibleForTesting + public TableSinkFactoryContext( CatalogTable catalogTable, ReadonlyConfig options, ClassLoader classLoader) { super(options, classLoader); + if (catalogTable != null) { + checkCatalogTableIllegal(Collections.singletonList(catalogTable)); + } this.catalogTable = catalogTable; } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableTransformFactoryContext.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableTransformFactoryContext.java index bf8176c7a8d..8e274a8e5e5 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableTransformFactoryContext.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableTransformFactoryContext.java @@ -32,6 +32,7 @@ public class TableTransformFactoryContext extends TableFactoryContext { public TableTransformFactoryContext( List catalogTables, ReadonlyConfig options, ClassLoader classLoader) { super(options, classLoader); + checkCatalogTableIllegal(catalogTables); this.catalogTables = catalogTables; } } diff --git a/seatunnel-api/src/test/java/org/apache/seatunnel/api/table/catalog/CatalogTableTest.java b/seatunnel-api/src/test/java/org/apache/seatunnel/api/table/catalog/CatalogTableTest.java index d3c7692b606..0ed70456052 100644 --- a/seatunnel-api/src/test/java/org/apache/seatunnel/api/table/catalog/CatalogTableTest.java +++ b/seatunnel-api/src/test/java/org/apache/seatunnel/api/table/catalog/CatalogTableTest.java @@ -18,7 +18,11 @@ package org.apache.seatunnel.api.table.catalog; import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; +import org.apache.seatunnel.api.table.factory.TableTransformFactoryContext; +import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; +import org.apache.seatunnel.common.utils.SeaTunnelException; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -89,4 +93,62 @@ public void testReadCatalogTableWithUnsupportedType() { }); Assertions.assertEquals(result, exception.getParamsValueAs("tableUnsupportedTypes")); } + + @Test + public void testCatalogTableWithIllegalFieldNames() { + CatalogTable catalogTable = + CatalogTable.of( + TableIdentifier.of("catalog", "database", "table"), + TableSchema.builder() + .column( + PhysicalColumn.of( + " ", BasicType.STRING_TYPE, 1L, true, null, "")) + .build(), + Collections.emptyMap(), + Collections.emptyList(), + "comment"); + SeaTunnelException exception = + Assertions.assertThrows( + SeaTunnelException.class, + () -> + new TableTransformFactoryContext( + Collections.singletonList(catalogTable), null, null)); + SeaTunnelException exception2 = + Assertions.assertThrows( + SeaTunnelException.class, + () -> new TableSinkFactoryContext(catalogTable, null, null)); + Assertions.assertEquals( + "Table database.table field name cannot be empty", exception.getMessage()); + Assertions.assertEquals( + "Table database.table field name cannot be empty", exception2.getMessage()); + + CatalogTable catalogTable2 = + CatalogTable.of( + TableIdentifier.of("catalog", "database", "table"), + TableSchema.builder() + .column( + PhysicalColumn.of( + "name1", BasicType.STRING_TYPE, 1L, true, null, "")) + .column( + PhysicalColumn.of( + "name1", BasicType.STRING_TYPE, 1L, true, null, "")) + .build(), + Collections.emptyMap(), + Collections.emptyList(), + "comment"); + SeaTunnelException exception3 = + Assertions.assertThrows( + SeaTunnelException.class, + () -> + new TableTransformFactoryContext( + Collections.singletonList(catalogTable2), null, null)); + SeaTunnelException exception4 = + Assertions.assertThrows( + SeaTunnelException.class, + () -> new TableSinkFactoryContext(catalogTable2, null, null)); + Assertions.assertEquals( + "Table database.table field name1 duplicate", exception3.getMessage()); + Assertions.assertEquals( + "Table database.table field name1 duplicate", exception4.getMessage()); + } } From 2489f6446bae5b971a26e4cd3094f7d7af9d0208 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Mon, 12 Aug 2024 13:36:15 +0800 Subject: [PATCH 048/361] [Improve][Connector] Add multi-table sink option check (#7360) * [Improve][Connector] Add multi-table sink option check * fix --- .../seatunnel/api/sink/SinkCommonOptions.java | 2 +- .../assertion/sink/AssertSinkFactory.java | 6 +++++- .../console/sink/ConsoleSinkFactory.java | 6 +++++- .../connectors/druid/sink/DruidSinkFactory.java | 6 +++++- .../sink/ElasticsearchSinkFactory.java | 4 +++- .../file/local/sink/LocalFileSinkFactory.java | 2 ++ .../file/oss/sink/OssFileSinkFactory.java | 2 ++ .../file/s3/sink/S3FileSinkFactory.java | 2 ++ .../seatunnel/http/sink/HttpSinkFactory.java | 2 ++ .../seatunnel/hudi/sink/HudiSinkFactory.java | 4 +++- .../iceberg/sink/IcebergSinkFactory.java | 4 +++- .../influxdb/sink/InfluxDBSinkFactory.java | 4 +++- .../seatunnel/kudu/sink/KuduSinkFactory.java | 2 ++ .../seatunnel/paimon/sink/PaimonSinkFactory.java | 4 +++- .../seatunnel/redis/sink/RedisSinkFactory.java | 4 +++- .../ConnectorSpecificationCheckTest.java | 16 ++++++++++++++-- 16 files changed, 58 insertions(+), 12 deletions(-) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SinkCommonOptions.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SinkCommonOptions.java index 598193d695f..9c6538ac87c 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SinkCommonOptions.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SinkCommonOptions.java @@ -28,5 +28,5 @@ public class SinkCommonOptions { Options.key("multi_table_sink_replica") .intType() .defaultValue(1) - .withDescription("The replica number of multi table sink"); + .withDescription("The replica number of multi table sink writer"); } diff --git a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSinkFactory.java b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSinkFactory.java index 376863dc184..ae174d9857f 100644 --- a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSinkFactory.java +++ b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSinkFactory.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.assertion.sink; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; @@ -37,7 +38,10 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { - return OptionRule.builder().required(RULES).build(); + return OptionRule.builder() + .required(RULES) + .optional(SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) + .build(); } @Override diff --git a/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkFactory.java b/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkFactory.java index 169a281fc19..fa5c7deae9e 100644 --- a/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkFactory.java +++ b/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkFactory.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.configuration.Options; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; @@ -52,7 +53,10 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { - return OptionRule.builder().optional(LOG_PRINT_DATA, LOG_PRINT_DELAY).build(); + return OptionRule.builder() + .optional( + LOG_PRINT_DATA, LOG_PRINT_DELAY, SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) + .build(); } @Override diff --git a/seatunnel-connectors-v2/connector-druid/src/main/java/org/apache/seatunnel/connectors/druid/sink/DruidSinkFactory.java b/seatunnel-connectors-v2/connector-druid/src/main/java/org/apache/seatunnel/connectors/druid/sink/DruidSinkFactory.java index 0c6824b521e..3199d3d66f4 100644 --- a/seatunnel-connectors-v2/connector-druid/src/main/java/org/apache/seatunnel/connectors/druid/sink/DruidSinkFactory.java +++ b/seatunnel-connectors-v2/connector-druid/src/main/java/org/apache/seatunnel/connectors/druid/sink/DruidSinkFactory.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; @@ -40,7 +41,10 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { - return OptionRule.builder().required(COORDINATOR_URL, DATASOURCE).build(); + return OptionRule.builder() + .required(COORDINATOR_URL, DATASOURCE) + .optional(SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) + .build(); } @Override diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkFactory.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkFactory.java index 56ec1d0ab7b..b290a63c444 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkFactory.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkFactory.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.connector.TableSink; @@ -69,7 +70,8 @@ public OptionRule optionRule() { TLS_KEY_STORE_PATH, TLS_KEY_STORE_PASSWORD, TLS_TRUST_STORE_PATH, - TLS_TRUST_STORE_PASSWORD) + TLS_TRUST_STORE_PASSWORD, + SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) .build(); } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/sink/LocalFileSinkFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/sink/LocalFileSinkFactory.java index e8ee8e436d1..1a9bcc1734f 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/sink/LocalFileSinkFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/sink/LocalFileSinkFactory.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; @@ -48,6 +49,7 @@ public OptionRule optionRule() { .optional(BaseSinkConfig.FILE_FORMAT_TYPE) .optional(BaseSinkConfig.SCHEMA_SAVE_MODE) .optional(BaseSinkConfig.DATA_SAVE_MODE) + .optional(SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) .conditional( BaseSinkConfig.FILE_FORMAT_TYPE, FileFormat.TEXT, diff --git a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/sink/OssFileSinkFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/sink/OssFileSinkFactory.java index 5d6cb649f20..6fd3088ddc9 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/sink/OssFileSinkFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/sink/OssFileSinkFactory.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; @@ -102,6 +103,7 @@ public OptionRule optionRule() { .optional(BaseSinkConfig.DATE_FORMAT) .optional(BaseSinkConfig.DATETIME_FORMAT) .optional(BaseSinkConfig.TIME_FORMAT) + .optional(SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) .build(); } } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/sink/S3FileSinkFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/sink/S3FileSinkFactory.java index 4ac9f45915e..5c231443e99 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/sink/S3FileSinkFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/sink/S3FileSinkFactory.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; @@ -103,6 +104,7 @@ public OptionRule optionRule() { .optional(BaseSinkConfig.DATETIME_FORMAT) .optional(BaseSinkConfig.TIME_FORMAT) .optional(BaseSinkConfig.TMP_PATH) + .optional(SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) .build(); } diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/sink/HttpSinkFactory.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/sink/HttpSinkFactory.java index 539563ecb62..313d26dd3f7 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/sink/HttpSinkFactory.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/sink/HttpSinkFactory.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.http.sink; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; @@ -49,6 +50,7 @@ public OptionRule optionRule() { .optional(HttpConfig.RETRY) .optional(HttpConfig.RETRY_BACKOFF_MULTIPLIER_MS) .optional(HttpConfig.RETRY_BACKOFF_MAX_MS) + .optional(SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) .build(); } } diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiSinkFactory.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiSinkFactory.java index d38785de02d..7697842f826 100644 --- a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiSinkFactory.java +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/sink/HudiSinkFactory.java @@ -19,6 +19,7 @@ package org.apache.seatunnel.connectors.seatunnel.hudi.sink; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; @@ -61,7 +62,8 @@ public OptionRule optionRule() { INSERT_SHUFFLE_PARALLELISM, UPSERT_SHUFFLE_PARALLELISM, MIN_COMMITS_TO_KEEP, - MAX_COMMITS_TO_KEEP) + MAX_COMMITS_TO_KEEP, + SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) .build(); } diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkFactory.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkFactory.java index b32430b3197..212bb6371d3 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkFactory.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkFactory.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.connector.TableSink; @@ -57,7 +58,8 @@ public OptionRule optionRule() { SinkConfig.TABLE_DEFAULT_PARTITION_KEYS, SinkConfig.TABLE_UPSERT_MODE_ENABLED_PROP, SinkConfig.TABLE_SCHEMA_EVOLUTION_ENABLED_PROP, - SinkConfig.TABLES_DEFAULT_COMMIT_BRANCH) + SinkConfig.TABLES_DEFAULT_COMMIT_BRANCH, + SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) .build(); } diff --git a/seatunnel-connectors-v2/connector-influxdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/influxdb/sink/InfluxDBSinkFactory.java b/seatunnel-connectors-v2/connector-influxdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/influxdb/sink/InfluxDBSinkFactory.java index 81a294e95bc..a8c13cdbff6 100644 --- a/seatunnel-connectors-v2/connector-influxdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/influxdb/sink/InfluxDBSinkFactory.java +++ b/seatunnel-connectors-v2/connector-influxdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/influxdb/sink/InfluxDBSinkFactory.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; @@ -65,7 +66,8 @@ public OptionRule optionRule() { KEY_TIME, BATCH_SIZE, MAX_RETRIES, - RETRY_BACKOFF_MULTIPLIER_MS) + RETRY_BACKOFF_MULTIPLIER_MS, + SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) .build(); } diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSinkFactory.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSinkFactory.java index 3917d1cd62a..beff65521d8 100644 --- a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSinkFactory.java +++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSinkFactory.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; @@ -56,6 +57,7 @@ public OptionRule optionRule() { .optional(KuduSinkConfig.IGNORE_DUPLICATE) .optional(KuduSinkConfig.ENABLE_KERBEROS) .optional(KuduSinkConfig.KERBEROS_KRB5_CONF) + .optional(SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) .conditional( KuduSinkConfig.FLUSH_MODE, Arrays.asList(AUTO_FLUSH_BACKGROUND.name(), MANUAL_FLUSH.name()), diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkFactory.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkFactory.java index 83976d84f94..bbc74df3ce9 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkFactory.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkFactory.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.connector.TableSink; @@ -54,7 +55,8 @@ public OptionRule optionRule() { PaimonSinkConfig.DATA_SAVE_MODE, PaimonSinkConfig.PRIMARY_KEYS, PaimonSinkConfig.PARTITION_KEYS, - PaimonSinkConfig.WRITE_PROPS) + PaimonSinkConfig.WRITE_PROPS, + SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) .conditional( PaimonConfig.CATALOG_TYPE, PaimonCatalogEnum.HIVE, PaimonConfig.CATALOG_URI) .build(); diff --git a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/sink/RedisSinkFactory.java b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/sink/RedisSinkFactory.java index c4768c0618b..49c2644d707 100644 --- a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/sink/RedisSinkFactory.java +++ b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/sink/RedisSinkFactory.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.redis.sink; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; @@ -51,7 +52,8 @@ public OptionRule optionRule() { RedisConfig.USER, RedisConfig.KEY_PATTERN, RedisConfig.FORMAT, - RedisConfig.EXPIRE) + RedisConfig.EXPIRE, + SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) .conditional(RedisConfig.MODE, RedisConfig.RedisMode.CLUSTER, RedisConfig.NODES) .build(); } diff --git a/seatunnel-dist/src/test/java/org/apache/seatunnel/api/connector/ConnectorSpecificationCheckTest.java b/seatunnel-dist/src/test/java/org/apache/seatunnel/api/connector/ConnectorSpecificationCheckTest.java index 62a037a6f65..3628a5dce6d 100644 --- a/seatunnel-dist/src/test/java/org/apache/seatunnel/api/connector/ConnectorSpecificationCheckTest.java +++ b/seatunnel-dist/src/test/java/org/apache/seatunnel/api/connector/ConnectorSpecificationCheckTest.java @@ -17,7 +17,9 @@ package org.apache.seatunnel.api.connector; +import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.sink.SeaTunnelSink; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportMultiTableSink; import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; @@ -152,16 +154,26 @@ public void testAllConnectorImplementFactoryWithUpToDateMethod() throws ClassNot log.info( "Check sink connector {} successfully", factory.getClass().getSimpleName()); - checkSupportMultiTableSink(sinkClass); + checkSupportMultiTableSink(factory, sinkClass); } } } - private void checkSupportMultiTableSink(Class sinkClass) { + private void checkSupportMultiTableSink( + TableSinkFactory sinkFactory, Class sinkClass) { if (!SupportMultiTableSink.class.isAssignableFrom(sinkClass)) { return; } + OptionRule sinkOptionRule = sinkFactory.optionRule(); + Assertions.assertTrue( + sinkOptionRule + .getOptionalOptions() + .contains(SinkCommonOptions.MULTI_TABLE_SINK_REPLICA), + "Please add `SinkCommonOptions.MULTI_TABLE_SINK_REPLICA` optional into the `optionRule` method optional of `" + + sinkFactory.getClass().getSimpleName() + + "`"); + // Validate the `createWriter` method return type Optional createWriter = ReflectionUtils.getDeclaredMethod( From 9d161d24ea2fed6ed7392deda6342fd068df3b6a Mon Sep 17 00:00:00 2001 From: lizhenglei <127465317+jackyyyyyssss@users.noreply.github.com> Date: Mon, 12 Aug 2024 14:03:21 +0800 Subject: [PATCH 049/361] [Fix] Fix http e2e case (#7356) * 1 * fix * fix * fix * fix * fix --------- Co-authored-by: lizhenglei <673421862@qq.com> --- .../connector-http-e2e/pom.xml | 7 ++ .../seatunnel/e2e/connector/http/HttpIT.java | 115 +++++++++++++++++- 2 files changed, 119 insertions(+), 3 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/pom.xml index 84b73a5998b..69b776da5f0 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/pom.xml @@ -98,6 +98,13 @@ ${project.version} test + + org.mock-server + mockserver-netty-no-dependencies + 5.14.0 + test + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/src/test/java/org/apache/seatunnel/e2e/connector/http/HttpIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/src/test/java/org/apache/seatunnel/e2e/connector/http/HttpIT.java index f53d8c1d458..af1d7125eae 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/src/test/java/org/apache/seatunnel/e2e/connector/http/HttpIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/src/test/java/org/apache/seatunnel/e2e/connector/http/HttpIT.java @@ -17,6 +17,10 @@ package org.apache.seatunnel.e2e.connector.http; +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.DeserializationFeature; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; + import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.EngineType; @@ -27,6 +31,8 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; +import org.mockserver.client.MockServerClient; +import org.mockserver.model.Format; import org.testcontainers.containers.Container; import org.testcontainers.containers.GenericContainer; import org.testcontainers.containers.output.Slf4jLogConsumer; @@ -36,22 +42,35 @@ import org.testcontainers.utility.DockerLoggerFactory; import org.testcontainers.utility.MountableFile; +import com.google.common.collect.Lists; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.Setter; + import java.io.File; import java.io.IOException; +import java.math.BigDecimal; import java.net.URL; +import java.util.ArrayList; +import java.util.List; import java.util.Optional; +import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.mockserver.model.HttpRequest.request; + public class HttpIT extends TestSuiteBase implements TestResource { private static final String TMP_DIR = "/tmp"; - private static final String successCount = "Total Write Count : 2"; - private static final String IMAGE = "mockserver/mockserver:5.14.0"; private GenericContainer mockserverContainer; + private static final List records = new ArrayList<>(); + + private MockServerClient mockServerClient; + @BeforeAll @Override public void startUp() { @@ -78,7 +97,48 @@ public void startUp() { .withEnv("MOCKSERVER_LOG_LEVEL", "WARN") .withLogConsumer(new Slf4jLogConsumer(DockerLoggerFactory.getLogger(IMAGE))) .waitingFor(new HttpWaitStrategy().forPath("/").forStatusCode(404)); + mockserverContainer.setPortBindings(Lists.newArrayList(String.format("%s:%s", 1080, 1080))); Startables.deepStart(Stream.of(mockserverContainer)).join(); + mockServerClient = new MockServerClient("127.0.0.1", 1080); + fillMockRecords(); + } + + private static void fillMockRecords() { + Record recordFirst = new Record(); + RequestBody requestBodyFirst = new RequestBody(); + JsonBody jsonBodyFirst = new JsonBody(); + jsonBodyFirst.setId(1); + jsonBodyFirst.setVal_bool(true); + jsonBodyFirst.setVal_int8(new Byte("1")); + jsonBodyFirst.setVal_int16((short) 2); + jsonBodyFirst.setVal_int32(3); + jsonBodyFirst.setVal_int64(4); + jsonBodyFirst.setVal_float(4.3F); + jsonBodyFirst.setVal_double(5.3); + jsonBodyFirst.setVal_decimal(BigDecimal.valueOf(6.3)); + jsonBodyFirst.setVal_string("NEW"); + jsonBodyFirst.setVal_unixtime_micros("2020-02-02T02:02:02"); + requestBodyFirst.setJson(jsonBodyFirst); + recordFirst.setBody(requestBodyFirst); + + Record recordSec = new Record(); + RequestBody requestBodySec = new RequestBody(); + JsonBody jsonBodySec = new JsonBody(); + jsonBodySec.setId(2); + jsonBodySec.setVal_bool(true); + jsonBodySec.setVal_int8(new Byte("1")); + jsonBodySec.setVal_int16((short) 2); + jsonBodySec.setVal_int32(3); + jsonBodySec.setVal_int64(4); + jsonBodySec.setVal_float(4.3F); + jsonBodySec.setVal_double(5.3); + jsonBodySec.setVal_decimal(BigDecimal.valueOf(6.3)); + jsonBodySec.setVal_string("NEW"); + jsonBodySec.setVal_unixtime_micros("2020-02-02T02:02:02"); + requestBodySec.setJson(jsonBodySec); + recordSec.setBody(requestBodySec); + records.add(recordFirst); + records.add(recordSec); } @AfterAll @@ -87,6 +147,9 @@ public void tearDown() { if (mockserverContainer != null) { mockserverContainer.stop(); } + if (mockServerClient != null) { + mockServerClient.close(); + } } @TestTemplate @@ -176,7 +239,53 @@ public void testMultiTableHttp(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = container.executeJob("/fake_to_multitable.conf"); Assertions.assertEquals(0, execResult.getExitCode()); - Assertions.assertTrue(execResult.getStdout().contains(successCount)); + ObjectMapper objectMapper = new ObjectMapper(); + objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + String mockResponse = + mockServerClient.retrieveRecordedRequests( + request().withPath("/example/httpMultiTableContentSink").withMethod("POST"), + Format.JSON); + List recordResponse = + objectMapper.readValue(mockResponse, new TypeReference>() {}); + recordResponse = + recordResponse.stream() + .sorted( + (r1, r2) -> + r1.getBody().getJson().getId() + - r2.getBody().getJson().getId()) + .collect(Collectors.toList()); + Assertions.assertIterableEquals(records, recordResponse); + } + + @Getter + @Setter + @EqualsAndHashCode + static class Record { + private RequestBody body; + } + + @Getter + @Setter + @EqualsAndHashCode + static class RequestBody { + private JsonBody json; + } + + @Getter + @Setter + @EqualsAndHashCode + static class JsonBody { + private int id; + private boolean val_bool; + private byte val_int8; + private short val_int16; + private int val_int32; + private long val_int64; + private float val_float; + private double val_double; + private BigDecimal val_decimal; + private String val_string; + private String val_unixtime_micros; } public String getMockServerConfig() { From 7e77384305df100b0967405aa1b04c4d5a7fedce Mon Sep 17 00:00:00 2001 From: Tyrantlucifer Date: Tue, 13 Aug 2024 09:38:53 +0800 Subject: [PATCH 050/361] [Core][Flink] optimize method name (#7372) --- .../api/table/type/SeaTunnelRow.java | 29 +++++-------------- 1 file changed, 7 insertions(+), 22 deletions(-) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java index 11388dbb6a7..5f9f05c9528 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java @@ -28,7 +28,7 @@ public final class SeaTunnelRow implements Serializable { /** Table identifier. */ private String tableId = ""; /** The kind of change that a row describes in a changelog. */ - private RowKind kind = RowKind.INSERT; + private RowKind rowKind = RowKind.INSERT; /** The array to store the actual internal format values. */ private final Object[] fields; @@ -50,17 +50,8 @@ public void setTableId(String tableId) { this.tableId = tableId; } - /** - * The method will be removed in the future, please use {@link #setKind(RowKind)} instanced of - * it. - */ - @Deprecated - public void setRowKind(RowKind kind) { - setKind(kind); - } - - public void setKind(RowKind kind) { - this.kind = kind; + public void setRowKind(RowKind rowKind) { + this.rowKind = rowKind; } public int getArity() { @@ -71,14 +62,8 @@ public String getTableId() { return tableId; } - /** The method will be removed in the future, please use {@link #getKind()} instanced of it. */ - @Deprecated public RowKind getRowKind() { - return getKind(); - } - - public RowKind getKind() { - return this.kind; + return this.rowKind; } public Object[] getFields() { @@ -335,13 +320,13 @@ public boolean equals(Object o) { } SeaTunnelRow that = (SeaTunnelRow) o; return Objects.equals(tableId, that.tableId) - && kind == that.kind + && rowKind == that.rowKind && Arrays.deepEquals(fields, that.fields); } @Override public int hashCode() { - int result = Objects.hash(tableId, kind); + int result = Objects.hash(tableId, rowKind); result = 31 * result + Arrays.deepHashCode(fields); return result; } @@ -352,7 +337,7 @@ public String toString() { + "tableId=" + tableId + ", kind=" - + kind.shortString() + + rowKind.shortString() + ", fields=" + Arrays.toString(fields) + '}'; From 7b19df585f77c4d20b5507b105f5a055583767f4 Mon Sep 17 00:00:00 2001 From: Guangdong Liu <804167098@qq.com> Date: Tue, 13 Aug 2024 10:03:34 +0800 Subject: [PATCH 051/361] [Bugfix][Doris-connector] Fix Json serialization, null value causes data error problem Co-authored-by: gdliu3 --- .../doris/serialize/SeaTunnelRowSerializer.java | 2 +- .../apache/seatunnel/e2e/connector/doris/DorisIT.java | 10 ++++++++-- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java index 0e67257a32e..c984580f8e1 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/SeaTunnelRowSerializer.java @@ -70,7 +70,7 @@ public SeaTunnelRowSerializer( if (JSON.equals(type)) { JsonSerializationSchema jsonSerializationSchema = - new JsonSerializationSchema(this.seaTunnelRowType, NULL_VALUE); + new JsonSerializationSchema(this.seaTunnelRowType); ObjectMapper mapper = jsonSerializationSchema.getMapper(); mapper.configure(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN, true); this.serialize = jsonSerializationSchema; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java index 03ca3ab516d..6b7a3a7f487 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java @@ -535,8 +535,14 @@ protected void batchInsertUniqueTableData() { try (PreparedStatement preparedStatement = conn.prepareStatement(INIT_UNIQUE_TABLE_DATA_SQL)) { for (int i = 0; i < rows.size(); i++) { - for (int index = 0; index < rows.get(i).getFields().length; index++) { - preparedStatement.setObject(index + 1, rows.get(i).getFields()[index]); + if (i % 10 == 0) { + for (int index = 0; index < rows.get(i).getFields().length; index++) { + preparedStatement.setObject(index + 1, null); + } + } else { + for (int index = 0; index < rows.get(i).getFields().length; index++) { + preparedStatement.setObject(index + 1, rows.get(i).getFields()[index]); + } } preparedStatement.addBatch(); } From 6a7df83b3e95affcce0edd66e25f0e236f3d04d7 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Tue, 13 Aug 2024 17:41:08 +0800 Subject: [PATCH 052/361] [Fix][Zeta] Fix task can not end cause by lock metrics failed (#7357) --- .../seatunnel/engine/e2e/JobExecutionIT.java | 61 ++++++++++--------- .../engine/server/TaskExecutionService.java | 10 ++- .../engine/server/dag/physical/SubPlan.java | 9 ++- .../engine/server/master/JobMaster.java | 12 +++- .../server/TaskExecutionServiceTest.java | 13 ---- 5 files changed, 58 insertions(+), 47 deletions(-) diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/JobExecutionIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/JobExecutionIT.java index 0d1a647ded8..86ec21a3755 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/JobExecutionIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/JobExecutionIT.java @@ -22,12 +22,15 @@ import org.apache.seatunnel.engine.client.SeaTunnelClient; import org.apache.seatunnel.engine.client.job.ClientJobExecutionEnvironment; import org.apache.seatunnel.engine.client.job.ClientJobProxy; +import org.apache.seatunnel.engine.common.Constant; import org.apache.seatunnel.engine.common.config.ConfigProvider; import org.apache.seatunnel.engine.common.config.JobConfig; import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; import org.apache.seatunnel.engine.core.job.JobResult; import org.apache.seatunnel.engine.core.job.JobStatus; import org.apache.seatunnel.engine.server.SeaTunnelServerStarter; +import org.apache.seatunnel.engine.server.execution.TaskLocation; +import org.apache.seatunnel.engine.server.metrics.SeaTunnelMetricsContext; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; @@ -36,8 +39,10 @@ import com.hazelcast.client.config.ClientConfig; import com.hazelcast.instance.impl.HazelcastInstanceImpl; +import com.hazelcast.map.IMap; import lombok.extern.slf4j.Slf4j; +import java.util.HashMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -73,11 +78,17 @@ public void testSayHello() { @Test public void testExecuteJob() throws Exception { + runJobFileWithAssertEndStatus( + "batch_fakesource_to_file.conf", "fake_to_file", JobStatus.FINISHED); + } + + private static void runJobFileWithAssertEndStatus( + String confFile, String name, JobStatus finished) + throws ExecutionException, InterruptedException { Common.setDeployMode(DeployMode.CLIENT); - String filePath = TestUtils.getResource("batch_fakesource_to_file.conf"); + String filePath = TestUtils.getResource(confFile); JobConfig jobConfig = new JobConfig(); - jobConfig.setName("fake_to_file"); - + jobConfig.setName(name); ClientConfig clientConfig = ConfigProvider.locateAndGetClientConfig(); clientConfig.setClusterName(TestUtils.getClusterName("JobExecutionIT")); try (SeaTunnelClient engineClient = new SeaTunnelClient(clientConfig)) { @@ -94,11 +105,25 @@ public void testExecuteJob() throws Exception { () -> Assertions.assertTrue( objectCompletableFuture.isDone() - && JobStatus.FINISHED.equals( + && finished.equals( objectCompletableFuture.get()))); } } + @Test + public void testExecuteJobWithLockMetrics() throws Exception { + // lock metrics map + IMap> metricsImap = + hazelcastInstance.getMap(Constant.IMAP_RUNNING_JOB_METRICS); + metricsImap.lock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); + try { + runJobFileWithAssertEndStatus( + "batch_fakesource_to_file.conf", "fake_to_file", JobStatus.FINISHED); + } finally { + metricsImap.unlock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); + } + } + @Test public void cancelJobTest() throws Exception { Common.setDeployMode(DeployMode.CLIENT); @@ -229,29 +254,9 @@ void afterClass() { @Test public void testLastCheckpointErrorJob() throws Exception { - Common.setDeployMode(DeployMode.CLIENT); - String filePath = TestUtils.getResource("batch_last_checkpoint_error.conf"); - JobConfig jobConfig = new JobConfig(); - jobConfig.setName("batch_last_checkpoint_error"); - - ClientConfig clientConfig = ConfigProvider.locateAndGetClientConfig(); - clientConfig.setClusterName(TestUtils.getClusterName("JobExecutionIT")); - try (SeaTunnelClient engineClient = new SeaTunnelClient(clientConfig)) { - ClientJobExecutionEnvironment jobExecutionEnv = - engineClient.createExecutionContext(filePath, jobConfig, SEATUNNEL_CONFIG); - - final ClientJobProxy clientJobProxy = jobExecutionEnv.execute(); - - CompletableFuture objectCompletableFuture = - CompletableFuture.supplyAsync(clientJobProxy::waitForJobComplete); - - await().atMost(600000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> - Assertions.assertTrue( - objectCompletableFuture.isDone() - && JobStatus.FAILED.equals( - objectCompletableFuture.get()))); - } + runJobFileWithAssertEndStatus( + "batch_last_checkpoint_error.conf", + "batch_last_checkpoint_error", + JobStatus.FAILED); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java index 00716f2c90a..a4717c6a81b 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/TaskExecutionService.java @@ -968,10 +968,14 @@ void taskDone(Task task) { cancellationFutures.remove(taskGroupLocation); try { cancelAsyncFunction(taskGroupLocation); - } catch (Throwable e) { - throw new RuntimeException(e); + } catch (Throwable t) { + logger.severe("cancel async function failed", t); + } + try { + updateMetricsContextInImap(); + } catch (Throwable t) { + logger.severe("update metrics context in imap failed", t); } - updateMetricsContextInImap(); if (ex == null) { logger.info( String.format( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/SubPlan.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/SubPlan.java index 5e023f81e8d..6e6667dddaf 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/SubPlan.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/SubPlan.java @@ -308,7 +308,14 @@ private void subPlanDone(PipelineStatus pipelineStatus) { RetryUtils.retryWithException( () -> { jobMaster.savePipelineMetricsToHistory(getPipelineLocation()); - jobMaster.removeMetricsContext(getPipelineLocation(), pipelineStatus); + try { + jobMaster.removeMetricsContext(getPipelineLocation(), pipelineStatus); + } catch (Throwable e) { + log.error( + "Remove metrics context for pipeline {} failed, with exception: {}", + pipelineFullName, + ExceptionUtils.getMessage(e)); + } notifyCheckpointManagerPipelineEnd(pipelineStatus); jobMaster.releasePipelineResource(this); return null; diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java index 888114bec95..f521c05492f 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java @@ -95,6 +95,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import static com.hazelcast.jet.impl.util.ExceptionUtil.withTryCatch; @@ -678,8 +679,13 @@ public void removeMetricsContext( boolean lockedIMap = false; try { - metricsImap.lock(Constant.IMAP_RUNNING_JOB_METRICS_KEY); - lockedIMap = true; + lockedIMap = + metricsImap.tryLock( + Constant.IMAP_RUNNING_JOB_METRICS_KEY, 5, TimeUnit.SECONDS); + if (!lockedIMap) { + LOGGER.severe("lock imap failed in update metrics"); + return; + } HashMap centralMap = metricsImap.get(Constant.IMAP_RUNNING_JOB_METRICS_KEY); @@ -697,6 +703,8 @@ public void removeMetricsContext( collect.forEach(centralMap::remove); metricsImap.put(Constant.IMAP_RUNNING_JOB_METRICS_KEY, centralMap); } + } catch (Exception e) { + LOGGER.warning("failed to remove metrics context", e); } finally { if (lockedIMap) { boolean unLockedIMap = false; diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/TaskExecutionServiceTest.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/TaskExecutionServiceTest.java index 3658f32a13b..75659668ab9 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/TaskExecutionServiceTest.java +++ b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/TaskExecutionServiceTest.java @@ -28,7 +28,6 @@ import org.apache.seatunnel.engine.server.execution.TestTask; import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.Test; @@ -65,8 +64,6 @@ public void before() { } @Test - @Disabled( - "As we have more and more test cases the test the load of the test container will up, the test case may failed") public void testCancel() { TaskExecutionService taskExecutionService = server.getTaskExecutionService(); @@ -92,8 +89,6 @@ public void testCancel() { } @Test - @Disabled( - "As we have more and more test cases the test the load of the test container will up, the test case may failed") public void testCancelBlockTask() throws InterruptedException { TaskExecutionService taskExecutionService = server.getTaskExecutionService(); @@ -118,8 +113,6 @@ public void testCancelBlockTask() throws InterruptedException { } @Test - @Disabled( - "As we have more and more test cases the test the load of the test container will up, the test case may failed") public void testFinish() { TaskExecutionService taskExecutionService = server.getTaskExecutionService(); @@ -150,8 +143,6 @@ public void testFinish() { /** Test task execution time is the same as the timer timeout */ @Test - @Disabled( - "As we have more and more test cases the test the load of the test container will up, the test case may failed") public void testCriticalCallTime() throws InterruptedException { AtomicBoolean stopMark = new AtomicBoolean(false); CopyOnWriteArrayList stopTime = new CopyOnWriteArrayList<>(); @@ -189,8 +180,6 @@ public void testCriticalCallTime() throws InterruptedException { } @Test - @Disabled( - "As we have more and more test cases the test the load of the test container will up, the test case may failed") public void testThrowException() throws InterruptedException { TaskExecutionService taskExecutionService = server.getTaskExecutionService(); @@ -264,8 +253,6 @@ public void testThrowException() throws InterruptedException { } @RepeatedTest(2) - @Disabled( - "As we have more and more test cases the test the load of the test container will up, the test case may failed") public void testDelay() throws InterruptedException { long lowLagSleep = 10; From 527c7c7b5f35e75cfb2eb180b57fe7ee218dfb4c Mon Sep 17 00:00:00 2001 From: GumKey <743344516@qq.com> Date: Tue, 13 Aug 2024 22:16:59 +0800 Subject: [PATCH 053/361] [Feature][Connector-V2] add Aliyun SLS connector #3733 (#7348) --- .../workflows/labeler/label-scope-conf.yml | 5 + config/plugin_config | 1 + docs/en/connector-v2/source/Sls.md | 87 +++++ docs/zh/connector-v2/source/Sls.md | 87 +++++ plugin-mapping.properties | 3 +- seatunnel-connectors-v2/connector-sls/pom.xml | 53 +++ .../seatunnel/sls/config/Config.java | 82 +++++ .../seatunnel/sls/config/StartMode.java | 41 +++ .../serialization/FastLogDeserialization.java | 32 ++ .../FastLogDeserializationContent.java | 102 ++++++ .../FastLogDeserializationSchema.java | 134 +++++++ .../sls/source/ConsumerMetaData.java | 40 +++ .../sls/source/SlsConsumerThread.java | 72 ++++ .../seatunnel/sls/source/SlsSource.java | 89 +++++ .../seatunnel/sls/source/SlsSourceConfig.java | 135 ++++++++ .../sls/source/SlsSourceFactory.java | 68 ++++ .../seatunnel/sls/source/SlsSourceReader.java | 232 +++++++++++++ .../seatunnel/sls/source/SlsSourceSplit.java | 68 ++++ .../sls/source/SlsSourceSplitEnumerator.java | 327 ++++++++++++++++++ .../seatunnel/sls/state/SlsSourceState.java | 39 +++ .../seatunnel/sls/SlsFactoryTest.java | 31 ++ seatunnel-connectors-v2/pom.xml | 1 + seatunnel-dist/pom.xml | 7 + .../connector-sls-e2e/pom.xml | 36 ++ .../seatunnel/e2e/connector/sls/SlsIT.java | 57 +++ .../sls_source_with_schema_to_console.conf | 47 +++ .../sls_source_without_schema_to_console.conf | 39 +++ .../seatunnel-connector-v2-e2e/pom.xml | 1 + 28 files changed, 1915 insertions(+), 1 deletion(-) create mode 100644 docs/en/connector-v2/source/Sls.md create mode 100644 docs/zh/connector-v2/source/Sls.md create mode 100644 seatunnel-connectors-v2/connector-sls/pom.xml create mode 100644 seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/config/Config.java create mode 100644 seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/config/StartMode.java create mode 100644 seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/serialization/FastLogDeserialization.java create mode 100644 seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/serialization/FastLogDeserializationContent.java create mode 100644 seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/serialization/FastLogDeserializationSchema.java create mode 100644 seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/ConsumerMetaData.java create mode 100644 seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsConsumerThread.java create mode 100644 seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSource.java create mode 100644 seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceConfig.java create mode 100644 seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceFactory.java create mode 100644 seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceReader.java create mode 100644 seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceSplit.java create mode 100644 seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceSplitEnumerator.java create mode 100644 seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/state/SlsSourceState.java create mode 100644 seatunnel-connectors-v2/connector-sls/src/test/java/org/apache/seatunnel/connectors/seatunnel/sls/SlsFactoryTest.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/pom.xml create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/src/test/java/org/apache/seatunnel/e2e/connector/sls/SlsIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/src/test/resources/sls_source_with_schema_to_console.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/src/test/resources/sls_source_without_schema_to_console.conf diff --git a/.github/workflows/labeler/label-scope-conf.yml b/.github/workflows/labeler/label-scope-conf.yml index 599ed649396..b0a89dfd354 100644 --- a/.github/workflows/labeler/label-scope-conf.yml +++ b/.github/workflows/labeler/label-scope-conf.yml @@ -275,3 +275,8 @@ format: dependencies: - changed-files: - any-glob-to-any-file: tools/dependencies/** +sls: + - all: + - changed-files: + - any-glob-to-any-file: seatunnel-connectors-v2/connector-sls/** + - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(sls)/**' \ No newline at end of file diff --git a/config/plugin_config b/config/plugin_config index f6549168d6d..e3ac0f1d046 100644 --- a/config/plugin_config +++ b/config/plugin_config @@ -87,4 +87,5 @@ connector-tdengine connector-web3j connector-milvus connector-activemq +connector-sls --end-- \ No newline at end of file diff --git a/docs/en/connector-v2/source/Sls.md b/docs/en/connector-v2/source/Sls.md new file mode 100644 index 00000000000..6468f397ab7 --- /dev/null +++ b/docs/en/connector-v2/source/Sls.md @@ -0,0 +1,87 @@ +# Sls + +> Sls source connector + +## Support Those Engines + +> Spark
+> Flink
+> Seatunnel Zeta
+ +## Key Features + +- [x] [batch](../../concept/connector-v2-features.md) +- [x] [stream](../../concept/connector-v2-features.md) +- [x] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [column projection](../../concept/connector-v2-features.md) +- [x] [parallelism](../../concept/connector-v2-features.md) +- [ ] [support user-defined split](../../concept/connector-v2-features.md) + +## Description + +Source connector for Aliyun Sls. + +## Supported DataSource Info + +In order to use the Sls connector, the following dependencies are required. +They can be downloaded via install-plugin.sh or from the Maven central repository. + +| Datasource | Supported Versions | Maven | +|------------|--------------------|-----------------------------------------------------------------------------------------------------------| +| Sls | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-sls) | + +## Source Options + +| Name | Type | Required | Default | Description | +|-------------------------------------|---------------------------------------------|----------|--------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------| +| project | String | Yes | - | [Aliyun Sls Project](https://help.aliyun.com/zh/sls/user-guide/manage-a-project?spm=a2c4g.11186623.0.0.6f9755ebyfaYSl) | +| logstore | String | Yes | - | [Aliyun Sls Logstore](https://help.aliyun.com/zh/sls/user-guide/manage-a-logstore?spm=a2c4g.11186623.0.0.13137c08nfuiBC) | +| endpoint | String | Yes | - | [Aliyun Access Endpoint](https://help.aliyun.com/zh/sls/developer-reference/api-sls-2020-12-30-endpoint?spm=a2c4g.11186623.0.0.548945a8UyJULa) | +| access_key_id | String | Yes | - | [Aliyun AccessKey ID](https://help.aliyun.com/zh/ram/user-guide/create-an-accesskey-pair?spm=a2c4g.11186623.0.0.4a6e4e554CKhSc#task-2245479) | +| access_key_secret | String | Yes | - | [Aliyun AccessKey Secret](https://help.aliyun.com/zh/ram/user-guide/create-an-accesskey-pair?spm=a2c4g.11186623.0.0.4a6e4e554CKhSc#task-2245479) | +| start_mode | StartMode[earliest],[group_cursor],[latest] | No | group_cursor | The initial consumption pattern of consumers. | +| consumer_group | String | No | SeaTunnel-Consumer-Group | Sls consumer group id, used to distinguish different consumer groups. | +| auto_cursor_reset | CursorMode[begin],[end] | No | end | When there is no cursor in the consumer group, cursor initialization occurs | +| batch_size | Int | No | 1000 | The amount of data pulled from SLS each time | +| partition-discovery.interval-millis | Long | No | -1 | The interval for dynamically discovering topics and partitions. | + +## Task Example + +### Simple + +> This example reads the data of sls's logstore1 and prints it to the client.And if you have not yet installed and deployed SeaTunnel, you need to follow the instructions in Install SeaTunnel to install and deploy SeaTunnel. And if you have not yet installed and deployed SeaTunnel, you need to follow the instructions in [Install SeaTunnel](../../start-v2/locally/deployment.md) to install and deploy SeaTunnel. And then follow the instructions in [Quick Start With SeaTunnel Engine](../../start-v2/locally/quick-start-seatunnel-engine.md) to run this job. + +[Create RAM user and authorization](https://help.aliyun.com/zh/sls/create-a-ram-user-and-authorize-the-ram-user-to-access-log-service?spm=a2c4g.11186623.0.i4),Please ensure thr ram user have sufficient rights to perform, reference [RAM Custom Authorization Example](https://help.aliyun.com/zh/sls/use-custom-policies-to-grant-permissions-to-a-ram-user?spm=a2c4g.11186623.0.0.4a6e4e554CKhSc#reference-s3z-m1l-z2b) + +```hocon +# Defining the runtime environment +env { + parallelism = 2 + job.mode = "STREAMING" + checkpoint.interval = 30000 +} + +source { + Sls { + endpoint = "cn-hangzhou-intranet.log.aliyuncs.com" + project = "project1" + logstore = "logstore1" + access_key_id = "xxxxxxxxxxxxxxxxxxxxxxxx" + access_key_secret = "xxxxxxxxxxxxxxxxxxxxxxxxxxxxxx" + schema = { + fields = { + id = "int" + name = "string" + description = "string" + weight = "string" + } + } + } +} + +sink { + Console { + } +} +``` + diff --git a/docs/zh/connector-v2/source/Sls.md b/docs/zh/connector-v2/source/Sls.md new file mode 100644 index 00000000000..d0e10257258 --- /dev/null +++ b/docs/zh/connector-v2/source/Sls.md @@ -0,0 +1,87 @@ +# Sls + +> Sls source connector + +## 支持的引擎 + +> Spark
+> Flink
+> Seatunnel Zeta
+ +## 主要特性 + +- [x] [batch](../../concept/connector-v2-features.md) +- [x] [stream](../../concept/connector-v2-features.md) +- [x] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [column projection](../../concept/connector-v2-features.md) +- [x] [parallelism](../../concept/connector-v2-features.md) +- [ ] [support user-defined split](../../concept/connector-v2-features.md) + +## 描述 + +从阿里云Sls日志服务中读取数据。 + +## 支持的数据源信息 + +为了使用Sls连接器,需要以下依赖关系。 +它们可以通过install-plugin.sh或Maven中央存储库下载。 + +| 数据源 | 支持的版本 | Maven | +|-----|-----------|-----------------------------------------------------------------------------------------------------------| +| Sls | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-sls) | + +## Source Options + +| Name | Type | Required | Default | Description | +|-------------------------------------|---------------------------------------------|----------|--------------------------|------------------------------------------------------------------------------------------------------------------------------------| +| project | String | Yes | - | [阿里云 Sls 项目](https://help.aliyun.com/zh/sls/user-guide/manage-a-project?spm=a2c4g.11186623.0.0.6f9755ebyfaYSl) | +| logstore | String | Yes | - | [阿里云 Sls 日志库](https://help.aliyun.com/zh/sls/user-guide/manage-a-logstore?spm=a2c4g.11186623.0.0.13137c08nfuiBC) | +| endpoint | String | Yes | - | [阿里云访问服务点](https://help.aliyun.com/zh/sls/developer-reference/api-sls-2020-12-30-endpoint?spm=a2c4g.11186623.0.0.548945a8UyJULa) | +| access_key_id | String | Yes | - | [阿里云访问用户ID](https://help.aliyun.com/zh/ram/user-guide/create-an-accesskey-pair?spm=a2c4g.11186623.0.0.4a6e4e554CKhSc#task-2245479) | +| access_key_secret | String | Yes | - | [阿里云访问用户密码](https://help.aliyun.com/zh/ram/user-guide/create-an-accesskey-pair?spm=a2c4g.11186623.0.0.4a6e4e554CKhSc#task-2245479) | +| start_mode | StartMode[earliest],[group_cursor],[latest] | No | group_cursor | 消费者的初始消费模式 | +| consumer_group | String | No | SeaTunnel-Consumer-Group | Sls消费者组id,用于区分不同的消费者组 | +| auto_cursor_reset | CursorMode[begin],[end] | No | end | 当消费者组中没有记录读取游标时,初始化读取游标 | +| batch_size | Int | No | 1000 | 每次从SLS中读取的数据量 | +| partition-discovery.interval-millis | Long | No | -1 | 动态发现主题和分区的间隔 | + +## 任务示例 + +### 简单示例 + +> 此示例读取sls的logstore1的数据并将其打印到客户端。如果您尚未安装和部署SeaTunnel,则需要按照安装SeaTunnel中的说明安装和部署SeaTunnel。然后按照[快速启动SeaTunnel引擎](../../Start-v2/locale/Quick-Start SeaTunnel Engine.md)中的说明运行此作业。 + +[创建RAM用户及授权](https://help.aliyun.com/zh/sls/create-a-ram-user-and-authorize-the-ram-user-to-access-log-service?spm=a2c4g.11186623.0.i4), 请确认RAM用户有足够的权限来读取及管理数据,参考:[RAM自定义授权示例](https://help.aliyun.com/zh/sls/use-custom-policies-to-grant-permissions-to-a-ram-user?spm=a2c4g.11186623.0.0.4a6e4e554CKhSc#reference-s3z-m1l-z2b) + +```hocon +# Defining the runtime environment +env { + parallelism = 2 + job.mode = "STREAMING" + checkpoint.interval = 30000 +} + +source { + Sls { + endpoint = "cn-hangzhou-intranet.log.aliyuncs.com" + project = "project1" + logstore = "logstore1" + access_key_id = "xxxxxxxxxxxxxxxxxxxxxxxx" + access_key_secret = "xxxxxxxxxxxxxxxxxxxxxxxxxxxxxx" + schema = { + fields = { + id = "int" + name = "string" + description = "string" + weight = "string" + } + } + } +} + +sink { + Console { + } +} +``` + diff --git a/plugin-mapping.properties b/plugin-mapping.properties index 579bf2dac04..a74b9e1223e 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -130,6 +130,7 @@ seatunnel.sink.ObsFile = connector-file-obs seatunnel.source.Milvus = connector-milvus seatunnel.sink.Milvus = connector-milvus seatunnel.sink.ActiveMQ = connector-activemq +seatunnel.source.Sls = connector-sls seatunnel.transform.Sql = seatunnel-transforms-v2 seatunnel.transform.FieldMapper = seatunnel-transforms-v2 @@ -140,4 +141,4 @@ seatunnel.transform.Replace = seatunnel-transforms-v2 seatunnel.transform.Split = seatunnel-transforms-v2 seatunnel.transform.Copy = seatunnel-transforms-v2 seatunnel.transform.DynamicCompile = seatunnel-transforms-v2 -seatunnel.transform.LLM = seatunnel-transforms-v2 \ No newline at end of file +seatunnel.transform.LLM = seatunnel-transforms-v2 diff --git a/seatunnel-connectors-v2/connector-sls/pom.xml b/seatunnel-connectors-v2/connector-sls/pom.xml new file mode 100644 index 00000000000..dd47dd0864b --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/pom.xml @@ -0,0 +1,53 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connectors-v2 + ${revision} + + + connector-sls + SeaTunnel : Connectors V2 : Sls + + + 0.6.109 + + + + + org.apache.seatunnel + connector-common + ${project.version} + + + com.aliyun.openservices + aliyun-log + ${aliyun-log.version} + + + org.apache.seatunnel + seatunnel-format-text + ${project.version} + + + diff --git a/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/config/Config.java b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/config/Config.java new file mode 100644 index 00000000000..46917b8b84a --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/config/Config.java @@ -0,0 +1,82 @@ +/* + * 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 + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.sls.config; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; + +import com.aliyun.openservices.log.common.Consts; + +public class Config { + public static final String CONNECTOR_IDENTITY = "Sls"; + + public static final Option ENDPOINT = + Options.key("endpoint") + .stringType() + .noDefaultValue() + .withDescription("Aliyun Access endpoint"); + public static final Option PROJECT = + Options.key("project") + .stringType() + .noDefaultValue() + .withDescription("Aliyun sls project"); + public static final Option LOGSTORE = + Options.key("logstore") + .stringType() + .noDefaultValue() + .withDescription("Aliyun sls logstore"); + public static final Option ACCESS_KEY_ID = + Options.key("access_key_id") + .stringType() + .noDefaultValue() + .withDescription("Aliyun accessKey id"); + public static final Option ACCESS_KEY_SECRET = + Options.key("access_key_secret") + .stringType() + .noDefaultValue() + .withDescription("Aliyun accessKey secret"); + public static final Option CONSUMER_GROUP = + Options.key("consumer_group") + .stringType() + .defaultValue("SeaTunnel-Consumer-Group") + .withDescription("Aliyun sls consumer group"); + public static final Option BATCH_SIZE = + Options.key("batch_size") + .intType() + .defaultValue(1000) + .withDescription("The amount of data pulled from sls each time"); + + public static final Option START_MODE = + Options.key("start_mode") + .objectType(StartMode.class) + .defaultValue(StartMode.GROUP_CURSOR) + .withDescription("initial consumption pattern of consumers"); + + public static final Option AUTO_CURSOR_RESET = + Options.key("auto_cursor_reset") + .objectType(Consts.CursorMode.class) + .defaultValue(Consts.CursorMode.END) + .withDescription("init consumer cursor"); + + public static final Option KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS = + Options.key("partition-discovery.interval-millis") + .longType() + .defaultValue(-1L) + .withDescription( + "The interval for dynamically discovering topics and partitions."); +} diff --git a/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/config/StartMode.java b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/config/StartMode.java new file mode 100644 index 00000000000..442d3970e51 --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/config/StartMode.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.sls.config; + +public enum StartMode { + EARLIEST("earliest"), + + GROUP_CURSOR("group_cursor"), + + LATEST("latest"); + + private String mode; + + StartMode(String mode) { + this.mode = mode; + } + + public String getMode() { + return mode; + } + + @Override + public String toString() { + return mode; + } +} diff --git a/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/serialization/FastLogDeserialization.java b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/serialization/FastLogDeserialization.java new file mode 100644 index 00000000000..1ae2ce3f181 --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/serialization/FastLogDeserialization.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.sls.serialization; + +import org.apache.seatunnel.api.source.Collector; + +import com.aliyun.openservices.log.common.LogGroupData; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; + +public interface FastLogDeserialization extends Serializable { + + default void deserialize(List logGroupDatas, Collector out) + throws IOException {} +} diff --git a/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/serialization/FastLogDeserializationContent.java b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/serialization/FastLogDeserializationContent.java new file mode 100644 index 00000000000..27bd35bff23 --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/serialization/FastLogDeserializationContent.java @@ -0,0 +1,102 @@ +/* + * 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 + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.sls.serialization; + +import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.type.RowKind; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; + +import com.aliyun.openservices.log.common.FastLog; +import com.aliyun.openservices.log.common.FastLogGroup; +import com.aliyun.openservices.log.common.LogGroupData; + +import java.io.IOException; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.temporal.ChronoField; +import java.util.ArrayList; +import java.util.List; + +public class FastLogDeserializationContent + implements DeserializationSchema, FastLogDeserialization { + + public static final DateTimeFormatter TIME_FORMAT; + private final CatalogTable catalogTable; + + static { + TIME_FORMAT = + (new DateTimeFormatterBuilder()) + .appendPattern("HH:mm:ss") + .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true) + .toFormatter(); + } + + public FastLogDeserializationContent(CatalogTable catalogTable) { + this.catalogTable = catalogTable; + } + + @Override + public SeaTunnelRow deserialize(byte[] bytes) throws IOException { + return null; + } + + @Override + public SeaTunnelDataType getProducedType() { + return null; + } + + public void deserialize(List logGroupDatas, Collector out) + throws IOException { + for (LogGroupData logGroupData : logGroupDatas) { + FastLogGroup logs = logGroupData.GetFastLogGroup(); + for (FastLog log : logs.getLogs()) { + SeaTunnelRow seaTunnelRow = convertFastLogContent(log); + out.collect(seaTunnelRow); + } + } + } + + private SeaTunnelRow convertFastLogContent(FastLog log) { + SeaTunnelRowType rowType = catalogTable.getSeaTunnelRowType(); + List transformedRow = new ArrayList<>(rowType.getTotalFields()); + // json format + StringBuilder jsonStringBuilder = new StringBuilder(); + jsonStringBuilder.append("{"); + log.getContents() + .forEach( + (content) -> + jsonStringBuilder + .append("\"") + .append(content.getKey()) + .append("\":\"") + .append(content.getValue()) + .append("\",")); + jsonStringBuilder.deleteCharAt(jsonStringBuilder.length() - 1); // 删除最后一个逗号 + jsonStringBuilder.append("}"); + // content field + transformedRow.add(jsonStringBuilder.toString()); + SeaTunnelRow seaTunnelRow = new SeaTunnelRow(transformedRow.toArray()); + seaTunnelRow.setRowKind(RowKind.INSERT); + seaTunnelRow.setTableId(catalogTable.getTableId().getTableName()); + return seaTunnelRow; + } +} diff --git a/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/serialization/FastLogDeserializationSchema.java b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/serialization/FastLogDeserializationSchema.java new file mode 100644 index 00000000000..b00f81eb832 --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/serialization/FastLogDeserializationSchema.java @@ -0,0 +1,134 @@ +/* + * 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 + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.sls.serialization; + +import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.type.RowKind; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.format.text.exception.SeaTunnelTextFormatException; + +import com.aliyun.openservices.log.common.FastLog; +import com.aliyun.openservices.log.common.FastLogContent; +import com.aliyun.openservices.log.common.FastLogGroup; +import com.aliyun.openservices.log.common.LogGroupData; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.temporal.ChronoField; +import java.util.ArrayList; +import java.util.List; + +public class FastLogDeserializationSchema + implements DeserializationSchema, FastLogDeserialization { + + public static final DateTimeFormatter TIME_FORMAT; + private final CatalogTable catalogTable; + + static { + TIME_FORMAT = + (new DateTimeFormatterBuilder()) + .appendPattern("HH:mm:ss") + .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true) + .toFormatter(); + } + + public FastLogDeserializationSchema(CatalogTable catalogTable) { + + this.catalogTable = catalogTable; + } + + @Override + public SeaTunnelRow deserialize(byte[] bytes) throws IOException { + return null; + } + + @Override + public SeaTunnelDataType getProducedType() { + return null; + } + + public void deserialize(List logGroupDatas, Collector out) + throws IOException { + for (LogGroupData logGroupData : logGroupDatas) { + FastLogGroup logs = logGroupData.GetFastLogGroup(); + for (FastLog log : logs.getLogs()) { + SeaTunnelRow seaTunnelRow = convertFastLogSchema(log); + out.collect(seaTunnelRow); + } + } + } + + private SeaTunnelRow convertFastLogSchema(FastLog log) { + SeaTunnelRowType rowType = catalogTable.getSeaTunnelRowType(); + List transformedRow = new ArrayList<>(rowType.getTotalFields()); + List logContents = log.getContents(); + for (FastLogContent flc : logContents) { + int keyIndex = rowType.indexOf(flc.getKey(), false); + if (keyIndex > -1) { + Object field = convert(rowType.getFieldType(keyIndex), flc.getValue()); + transformedRow.add(keyIndex, field); + } + } + SeaTunnelRow seaTunnelRow = new SeaTunnelRow(transformedRow.toArray()); + seaTunnelRow.setRowKind(RowKind.INSERT); + seaTunnelRow.setTableId(catalogTable.getTableId().getTableName()); + return seaTunnelRow; + } + + private Object convert(SeaTunnelDataType fieldType, String field) + throws SeaTunnelTextFormatException { + switch (fieldType.getSqlType()) { + case STRING: + return field; + case BOOLEAN: + return Boolean.parseBoolean(field); + case TINYINT: + return Byte.parseByte(field); + case SMALLINT: + return Short.parseShort(field); + case INT: + return Integer.parseInt(field); + case BIGINT: + return Long.parseLong(field); + case FLOAT: + return Float.parseFloat(field); + case DOUBLE: + return Double.parseDouble(field); + case DECIMAL: + return new BigDecimal(field); + case NULL: + return null; + case BYTES: + return field.getBytes(StandardCharsets.UTF_8); + default: + throw new SeaTunnelTextFormatException( + CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + String.format( + "SeaTunnel not support this data type [%s]", + fieldType.getSqlType())); + } + } +} diff --git a/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/ConsumerMetaData.java b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/ConsumerMetaData.java new file mode 100644 index 00000000000..356075aaea0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/ConsumerMetaData.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.sls.source; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.sls.config.StartMode; +import org.apache.seatunnel.connectors.seatunnel.sls.serialization.FastLogDeserialization; + +import com.aliyun.openservices.log.common.Consts; +import lombok.Data; + +import java.io.Serializable; + +@Data +public class ConsumerMetaData implements Serializable { + private String project; + private String logstore; + private String consumerGroup; + private StartMode startMode; + private Consts.CursorMode autoCursorReset; + private int fetchSize; + private FastLogDeserialization deserializationSchema; + private CatalogTable catalogTable; +} diff --git a/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsConsumerThread.java b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsConsumerThread.java new file mode 100644 index 00000000000..7a2b9f65ba4 --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsConsumerThread.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.sls.source; + +import com.aliyun.openservices.log.Client; +import lombok.Getter; + +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; + +public class SlsConsumerThread implements Runnable { + + private final Client client; + + @Getter private final LinkedBlockingQueue> tasks; + + public SlsConsumerThread(SlsSourceConfig slsSourceConfig) { + this.client = this.initClient(slsSourceConfig); + this.tasks = new LinkedBlockingQueue<>(); + } + + public LinkedBlockingQueue> getTasks() { + return tasks; + } + + @Override + public void run() { + try { + while (!Thread.currentThread().isInterrupted()) { + try { + Consumer task = tasks.poll(1, TimeUnit.SECONDS); + if (task != null) { + task.accept(client); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } finally { + try { + if (client != null) { + /** now do nothine, do not need close */ + } + } catch (Throwable t) { + throw new RuntimeException(t); + } + } + } + + private Client initClient(SlsSourceConfig slsSourceConfig) { + return new Client( + slsSourceConfig.getEndpoint(), + slsSourceConfig.getAccessKeyId(), + slsSourceConfig.getAccessKeySecret()); + } +} diff --git a/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSource.java b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSource.java new file mode 100644 index 00000000000..fdd5af871de --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSource.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.sls.source; + +import org.apache.seatunnel.api.common.JobContext; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.source.Boundedness; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.constants.JobMode; +import org.apache.seatunnel.connectors.seatunnel.sls.state.SlsSourceState; + +import com.google.common.collect.Lists; + +import java.util.List; + +public class SlsSource + implements SeaTunnelSource, + SupportParallelism { + + private JobContext jobContext; + + private final SlsSourceConfig slsSourceConfig; + + public SlsSource(ReadonlyConfig readonlyConfig) { + this.slsSourceConfig = new SlsSourceConfig(readonlyConfig); + } + + @Override + public void setJobContext(JobContext jobContext) { + this.jobContext = jobContext; + } + + @Override + public Boundedness getBoundedness() { + return JobMode.BATCH.equals(jobContext.getJobMode()) + ? Boundedness.BOUNDED + : Boundedness.UNBOUNDED; + } + + @Override + public SourceReader createReader(SourceReader.Context readContext) + throws Exception { + return new SlsSourceReader(slsSourceConfig, readContext); + } + + @Override + public SourceSplitEnumerator createEnumerator( + SourceSplitEnumerator.Context enumeratorContext) throws Exception { + return new SlsSourceSplitEnumerator(slsSourceConfig, enumeratorContext); + } + + @Override + public SourceSplitEnumerator restoreEnumerator( + SourceSplitEnumerator.Context enumeratorContext, + SlsSourceState checkpointState) + throws Exception { + return new SlsSourceSplitEnumerator(slsSourceConfig, enumeratorContext, checkpointState); + } + + @Override + public List getProducedCatalogTables() { + return Lists.newArrayList(slsSourceConfig.getCatalogTable()); + } + + @Override + public String getPluginName() { + return org.apache.seatunnel.connectors.seatunnel.sls.config.Config.CONNECTOR_IDENTITY; + } +} diff --git a/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceConfig.java b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceConfig.java new file mode 100644 index 00000000000..90d709491b0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceConfig.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 + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.sls.source; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.catalog.schema.ReadonlyConfigParser; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.sls.serialization.FastLogDeserialization; +import org.apache.seatunnel.connectors.seatunnel.sls.serialization.FastLogDeserializationContent; +import org.apache.seatunnel.connectors.seatunnel.sls.serialization.FastLogDeserializationSchema; +import org.apache.seatunnel.format.text.TextDeserializationSchema; +import org.apache.seatunnel.format.text.constant.TextFormatConstant; + +import lombok.Getter; + +import java.io.Serializable; +import java.util.Collections; +import java.util.Map; +import java.util.Optional; + +import static org.apache.seatunnel.connectors.seatunnel.sls.config.Config.ACCESS_KEY_ID; +import static org.apache.seatunnel.connectors.seatunnel.sls.config.Config.ACCESS_KEY_SECRET; +import static org.apache.seatunnel.connectors.seatunnel.sls.config.Config.AUTO_CURSOR_RESET; +import static org.apache.seatunnel.connectors.seatunnel.sls.config.Config.BATCH_SIZE; +import static org.apache.seatunnel.connectors.seatunnel.sls.config.Config.CONSUMER_GROUP; +import static org.apache.seatunnel.connectors.seatunnel.sls.config.Config.ENDPOINT; +import static org.apache.seatunnel.connectors.seatunnel.sls.config.Config.KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS; +import static org.apache.seatunnel.connectors.seatunnel.sls.config.Config.LOGSTORE; +import static org.apache.seatunnel.connectors.seatunnel.sls.config.Config.PROJECT; +import static org.apache.seatunnel.connectors.seatunnel.sls.config.Config.START_MODE; + +public class SlsSourceConfig implements Serializable { + private static final long serialVersionUID = 1L; + + @Getter private final String endpoint; + @Getter private final String accessKeyId; + @Getter private final String accessKeySecret; + @Getter private final Long discoveryIntervalMillis; + @Getter private final CatalogTable catalogTable; + @Getter private final ConsumerMetaData consumerMetaData; + + public SlsSourceConfig(ReadonlyConfig readonlyConfig) { + this.endpoint = readonlyConfig.get(ENDPOINT); + this.accessKeyId = readonlyConfig.get(ACCESS_KEY_ID); + this.accessKeySecret = readonlyConfig.get(ACCESS_KEY_SECRET); + this.discoveryIntervalMillis = readonlyConfig.get(KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS); + this.catalogTable = createCatalogTable(readonlyConfig); + this.consumerMetaData = createMetaData(readonlyConfig); + } + + /** only single endpoint logstore */ + public ConsumerMetaData createMetaData(ReadonlyConfig readonlyConfig) { + ConsumerMetaData consumerMetaData = new ConsumerMetaData(); + consumerMetaData.setProject(readonlyConfig.get(PROJECT)); + consumerMetaData.setLogstore(readonlyConfig.get(LOGSTORE)); + consumerMetaData.setConsumerGroup(readonlyConfig.get(CONSUMER_GROUP)); + consumerMetaData.setStartMode(readonlyConfig.get(START_MODE)); + consumerMetaData.setFetchSize(readonlyConfig.get(BATCH_SIZE)); + consumerMetaData.setAutoCursorReset(readonlyConfig.get(AUTO_CURSOR_RESET)); + consumerMetaData.setDeserializationSchema(createDeserializationSchema(readonlyConfig)); + consumerMetaData.setCatalogTable(catalogTable); + return consumerMetaData; + } + + private CatalogTable createCatalogTable(ReadonlyConfig readonlyConfig) { + Optional> schemaOptions = + readonlyConfig.getOptional(TableSchemaOptions.SCHEMA); + TablePath tablePath = TablePath.of(readonlyConfig.get(LOGSTORE)); + TableSchema tableSchema; + if (schemaOptions.isPresent()) { + tableSchema = new ReadonlyConfigParser().parse(readonlyConfig); + } else { + // no scheam, all value in content filed + tableSchema = + TableSchema.builder() + .column( + PhysicalColumn.of( + "content", BasicType.STRING_TYPE, 0, false, "{}", null)) + .build(); + } + return CatalogTable.of( + TableIdentifier.of("", tablePath), + tableSchema, + Collections.emptyMap(), + Collections.emptyList(), + null); + } + + private FastLogDeserialization createDeserializationSchema( + ReadonlyConfig readonlyConfig) { + Optional> schemaOptions = + readonlyConfig.getOptional(TableSchemaOptions.SCHEMA); + FastLogDeserialization fastLogDeserialization; + if (schemaOptions.isPresent()) { + fastLogDeserialization = new FastLogDeserializationSchema(catalogTable); + + } else { + fastLogDeserialization = new FastLogDeserializationContent(catalogTable); + } + return fastLogDeserialization; + } + + private DeserializationSchema createDeserializationSchema( + CatalogTable catalogTable) { + SeaTunnelRowType seaTunnelRowType = catalogTable.getSeaTunnelRowType(); + return TextDeserializationSchema.builder() + .seaTunnelRowType(seaTunnelRowType) + .delimiter(TextFormatConstant.PLACEHOLDER) + .build(); + } +} diff --git a/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceFactory.java b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceFactory.java new file mode 100644 index 00000000000..42cda335fd1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceFactory.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.sls.source; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.connector.TableSource; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; +import org.apache.seatunnel.connectors.seatunnel.sls.config.Config; + +import com.google.auto.service.AutoService; + +import java.io.Serializable; + +@AutoService(Factory.class) +public class SlsSourceFactory implements TableSourceFactory { + @Override + public Class getSourceClass() { + return (Class) SlsSource.class; + } + + @Override + public String factoryIdentifier() { + return Config.CONNECTOR_IDENTITY; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required( + Config.ENDPOINT, + Config.PROJECT, + Config.LOGSTORE, + Config.ACCESS_KEY_ID, + Config.ACCESS_KEY_SECRET) + .optional( + Config.BATCH_SIZE, + Config.START_MODE, + Config.KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS, + Config.AUTO_CURSOR_RESET, + Config.CONSUMER_GROUP) + .build(); + } + + @Override + public + TableSource createSource(TableSourceFactoryContext context) { + return () -> (SeaTunnelSource) new SlsSource(context.getOptions()); + } +} diff --git a/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceReader.java b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceReader.java new file mode 100644 index 00000000000..43cb75328a1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceReader.java @@ -0,0 +1,232 @@ +/* + * 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 + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.sls.source; + +import org.apache.seatunnel.api.source.Boundedness; +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.sls.serialization.FastLogDeserialization; + +import com.aliyun.openservices.log.common.LogGroupData; +import com.aliyun.openservices.log.exception.LogException; +import com.aliyun.openservices.log.request.PullLogsRequest; +import com.aliyun.openservices.log.response.PullLogsResponse; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.stream.Collectors; + +@Slf4j +public class SlsSourceReader implements SourceReader { + private static final long THREAD_WAIT_TIME = 500L; + private final SourceReader.Context context; + private volatile boolean running = false; + private final LinkedBlockingQueue pendingShardsQueue; + private final Set sourceSplits; + private final Map consumerThreadMap; + private final SlsSourceConfig slsSourceConfig; + private final ExecutorService executorService; + + private final Map> checkpointOffsetMap; + + SlsSourceReader(SlsSourceConfig slsSourceConfig, Context context) { + this.pendingShardsQueue = new LinkedBlockingQueue(); + this.sourceSplits = new HashSet<>(); + this.consumerThreadMap = new ConcurrentHashMap<>(); + this.slsSourceConfig = slsSourceConfig; + this.context = context; + this.executorService = + Executors.newCachedThreadPool(r -> new Thread(r, "Sls Source Data Consumer")); + this.checkpointOffsetMap = new ConcurrentHashMap<>(); + } + + @Override + public void open() throws Exception {} + + @Override + public void close() throws IOException { + if (executorService != null) { + executorService.shutdownNow(); + } + } + + @Override + public void pollNext(Collector collector) throws Exception { + if (!running) { + Thread.sleep(THREAD_WAIT_TIME); + return; + } + + while (!pendingShardsQueue.isEmpty()) { + sourceSplits.add(pendingShardsQueue.poll()); + } + /** thread for Client */ + sourceSplits.forEach( + sourceSplit -> + consumerThreadMap.computeIfAbsent( + sourceSplit.splitId(), + s -> { + SlsConsumerThread thread = + new SlsConsumerThread(slsSourceConfig); + executorService.submit(thread); + return thread; + })); + List finishedSplits = new CopyOnWriteArrayList<>(); + FastLogDeserialization fastLogDeserialization = + slsSourceConfig.getConsumerMetaData().getDeserializationSchema(); + sourceSplits.forEach( + sourceSplit -> { + CompletableFuture completableFuture = new CompletableFuture<>(); + try { + consumerThreadMap + .get(sourceSplit.splitId()) + .getTasks() + .put( + consumer -> { + try { + PullLogsRequest request = + new PullLogsRequest( + sourceSplit.getProject(), + sourceSplit.getLogStore(), + sourceSplit.getShardId(), + sourceSplit.getFetchSize(), + sourceSplit.getStartCursor()); + PullLogsResponse response = + consumer.pullLogs(request); + List logGroupDatas = + response.getLogGroups(); + fastLogDeserialization.deserialize( + logGroupDatas, collector); + sourceSplit.setStartCursor( + response.getNextCursor()); + completableFuture.complete(true); + } catch (LogException e) { + e.printStackTrace(); + completableFuture.completeExceptionally(e); + throw new RuntimeException(e); + } catch (IOException e) { + e.printStackTrace(); + completableFuture.completeExceptionally(e); + throw new RuntimeException(e); + } + completableFuture.complete(false); + }); + if (completableFuture.get()) { + finishedSplits.add(sourceSplit); + } + } catch (InterruptedException e) { + e.printStackTrace(); + throw new RuntimeException(e); + } catch (ExecutionException e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + }); + + // batch mode only for explore data, so do not update cursor + if (Boundedness.BOUNDED.equals(context.getBoundedness())) { + for (SlsSourceSplit split : finishedSplits) { + split.setFinish(true); + } + if (sourceSplits.stream().allMatch(SlsSourceSplit::isFinish)) { + log.info("sls batch mode finished"); + context.signalNoMoreElement(); + } + } + } + + @Override + public List snapshotState(long checkpointId) throws Exception { + checkpointOffsetMap.put( + checkpointId, + sourceSplits.stream() + .collect(Collectors.toMap(SlsSourceSplit::splitId, SlsSourceSplit::copy))); + return sourceSplits.stream().map(SlsSourceSplit::copy).collect(Collectors.toList()); + } + + // 接受 + @Override + public void addSplits(List splits) { + running = true; + splits.forEach( + s -> { + try { + pendingShardsQueue.put(s); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + } + + @Override + public void handleNoMoreSplits() { + log.info("receive no more splits message, this reader will not add new split."); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + if (!checkpointOffsetMap.containsKey(checkpointId)) { + log.warn("checkpoint {} do not exist or have already been committed.", checkpointId); + } else { + checkpointOffsetMap + .remove(checkpointId) + .forEach( + (sharId, slsSourceSplit) -> { + try { + consumerThreadMap + .get(sharId) + .getTasks() + .put( + client -> { + // now only default onCheckpointCommit + try { + client.UpdateCheckPoint( + slsSourceSplit.getProject(), + slsSourceSplit.getLogStore(), + slsSourceSplit.getConsumer(), + slsSourceSplit.getShardId(), + slsSourceSplit + .getStartCursor()); + } catch (LogException e) { + e.printStackTrace(); + log.error( + "LogException: commit cursor to sls failed", + e); + throw new RuntimeException(e); + } + }); + } catch (InterruptedException e) { + log.error( + "InterruptedException: commit cursor to sls failed", e); + } + }); + } + } +} diff --git a/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceSplit.java b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceSplit.java new file mode 100644 index 00000000000..d5099b9cd1e --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceSplit.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.sls.source; + +import org.apache.seatunnel.api.source.SourceSplit; + +import lombok.Getter; +import lombok.Setter; + +public class SlsSourceSplit implements SourceSplit { + + @Getter private String project; + @Getter private String logStore; + @Getter private String consumer; + @Getter private Integer shardId; + @Getter private String startCursor; + @Getter private Integer fetchSize; + @Setter @Getter private transient volatile boolean finish = false; + + SlsSourceSplit( + String project, + String logStore, + String consumer, + Integer shardId, + String startCursor, + Integer fetchSize) { + this.project = project; + this.logStore = logStore; + this.consumer = consumer; + this.shardId = shardId; + this.startCursor = startCursor; + this.fetchSize = fetchSize; + } + + @Override + public String splitId() { + return String.valueOf(shardId); + } + + public void setStartCursor(String cursor) { + this.startCursor = cursor; + } + + public SlsSourceSplit copy() { + return new SlsSourceSplit( + this.project, + this.logStore, + this.consumer, + this.shardId, + this.startCursor, + this.fetchSize); + } +} diff --git a/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceSplitEnumerator.java new file mode 100644 index 00000000000..f178d441a33 --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/source/SlsSourceSplitEnumerator.java @@ -0,0 +1,327 @@ +/* + * 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 + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.sls.source; + +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.common.config.Common; +import org.apache.seatunnel.connectors.seatunnel.sls.config.StartMode; +import org.apache.seatunnel.connectors.seatunnel.sls.state.SlsSourceState; + +import com.aliyun.openservices.log.Client; +import com.aliyun.openservices.log.common.Consts; +import com.aliyun.openservices.log.common.ConsumerGroup; +import com.aliyun.openservices.log.common.ConsumerGroupShardCheckPoint; +import com.aliyun.openservices.log.exception.LogException; +import com.aliyun.openservices.log.response.ConsumerGroupCheckPointResponse; +import com.aliyun.openservices.log.response.ListConsumerGroupResponse; +import com.aliyun.openservices.log.response.ListShardResponse; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +@Slf4j +public class SlsSourceSplitEnumerator + implements SourceSplitEnumerator { + + private final Client slsCleint; + private final ConsumerMetaData consumerMetaData; + + private final long discoveryIntervalMillis; + + private final Context context; + private final Map pendingSplit; + private final Map assignedSplit; + + private SlsSourceState slsSourceState; + + private ScheduledExecutorService executor; + private ScheduledFuture scheduledFuture; + + public SlsSourceSplitEnumerator( + SlsSourceConfig slsSourceConfig, Context context) { + this.context = context; + this.slsCleint = + new Client( + slsSourceConfig.getEndpoint(), + slsSourceConfig.getAccessKeyId(), + slsSourceConfig.getAccessKeySecret()); + this.assignedSplit = new HashMap<>(); + this.pendingSplit = new HashMap<>(); + this.consumerMetaData = slsSourceConfig.getConsumerMetaData(); + this.discoveryIntervalMillis = slsSourceConfig.getDiscoveryIntervalMillis(); + } + + public SlsSourceSplitEnumerator( + SlsSourceConfig slsSourceConfig, + Context context, + SlsSourceState slsSourceState) { + this.context = context; + this.slsCleint = + new Client( + slsSourceConfig.getEndpoint(), + slsSourceConfig.getAccessKeyId(), + slsSourceConfig.getAccessKeySecret()); + this.assignedSplit = new HashMap<>(); + this.pendingSplit = new HashMap<>(); + this.consumerMetaData = slsSourceConfig.getConsumerMetaData(); + this.discoveryIntervalMillis = slsSourceConfig.getDiscoveryIntervalMillis(); + + /** now only from sls cursor for restore */ + this.slsSourceState = slsSourceState; + if (slsSourceState != null) {} + } + + @Override + public void open() { + if (discoveryIntervalMillis > 0) { + this.executor = + Executors.newScheduledThreadPool( + 1, + runnable -> { + Thread thread = new Thread(runnable); + thread.setDaemon(true); + thread.setName("sls-shard-dynamic-discovery"); + return thread; + }); + this.scheduledFuture = + executor.scheduleWithFixedDelay( + () -> { + try { + discoverySplits(); + } catch (Exception e) { + log.error("Dynamic discovery failure:", e); + } + }, + discoveryIntervalMillis, + discoveryIntervalMillis, + TimeUnit.MILLISECONDS); + } + } + + @Override + public void run() throws Exception { + fetchPendingShardSplit(); + assignSplit(); + } + + @Override + public void close() throws IOException {} + + @Override + public void addSplitsBack(List splits, int subtaskId) { + if (!splits.isEmpty()) { + splits.forEach(split -> pendingSplit.put(split.getShardId(), split)); + } + } + + @Override + public int currentUnassignedSplitSize() { + return 0; + } + + @Override + public void handleSplitRequest(int subtaskId) {} + + @Override + public void registerReader(int subtaskId) { + if (!pendingSplit.isEmpty()) { + assignSplit(); + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception {} + + private void discoverySplits() throws LogException { + fetchPendingShardSplit(); + assignSplit(); + } + + private void fetchPendingShardSplit() throws LogException { + String project = this.consumerMetaData.getProject(); + String logStore = this.consumerMetaData.getLogstore(); + String consumer = this.consumerMetaData.getConsumerGroup(); + StartMode startMode = this.consumerMetaData.getStartMode(); + int fetachSize = this.consumerMetaData.getFetchSize(); + Consts.CursorMode autoCursorReset = this.consumerMetaData.getAutoCursorReset(); + ListShardResponse shards = this.slsCleint.ListShard(project, logStore); + shards.GetShards() + .forEach( + shard -> { + if (!assignedSplit.containsKey(shard.getShardId())) { + if (!pendingSplit.containsKey(shard.getShardId())) { + String cursor = ""; + try { + cursor = + initShardCursor( + project, + logStore, + consumer, + shard.getShardId(), + startMode, + autoCursorReset); + } catch (Exception e) { + throw new RuntimeException(e); + } + if (cursor.equals("")) { + throw new RuntimeException("shard cursor error"); + } + SlsSourceSplit split = + new SlsSourceSplit( + project, + logStore, + consumer, + shard.getShardId(), + cursor, + fetachSize); + pendingSplit.put(shard.getShardId(), split); + } + } + }); + } + + private String initShardCursor( + String project, + String logStore, + String consumer, + int shardIdKey, + StartMode cursorMode, + Consts.CursorMode autoCursorReset) + throws Exception { + switch (cursorMode) { + case EARLIEST: + try { + return this.slsCleint + .GetCursor(project, logStore, shardIdKey, Consts.CursorMode.BEGIN) + .GetCursor(); + } catch (LogException e) { + throw new RuntimeException(e); + } + case LATEST: + try { + return this.slsCleint + .GetCursor(project, logStore, shardIdKey, Consts.CursorMode.END) + .GetCursor(); + } catch (LogException e) { + throw new RuntimeException(e); + } + case GROUP_CURSOR: + try { + boolean groupExists = checkConsumerGroupExists(project, logStore, consumer); + if (!groupExists) { + createConsumerGroup(project, logStore, consumer); + } + ConsumerGroupCheckPointResponse response = + this.slsCleint.GetCheckPoint(project, logStore, consumer, shardIdKey); + List checkpoints = response.getCheckPoints(); + if (checkpoints.size() == 1) { + ConsumerGroupShardCheckPoint checkpoint = checkpoints.get(0); + if (!checkpoint.getCheckPoint().equals("")) { + return checkpoint.getCheckPoint(); + } + } + return this.slsCleint + .GetCursor(project, logStore, shardIdKey, autoCursorReset) + .GetCursor(); + } catch (LogException e) { + if (e.GetErrorCode().equals("ConsumerGroupNotExist")) { + return this.slsCleint + .GetCursor(project, logStore, shardIdKey, autoCursorReset) + .GetCursor(); + } + throw new RuntimeException(e); + } + } + throw new RuntimeException( + project + ":" + logStore + ":" + consumer + ":" + cursorMode + ":" + "fail"); + } + + private synchronized void assignSplit() { + Map> readySplit = new HashMap<>(Common.COLLECTION_SIZE); + // init task from Parallelism + for (int taskID = 0; taskID < context.currentParallelism(); taskID++) { + readySplit.computeIfAbsent(taskID, id -> new ArrayList<>()); + } + // Determine if split has been assigned + pendingSplit.forEach( + (key, value) -> { + if (!assignedSplit.containsKey(key)) { + readySplit + .get( + getSplitOwner( + value.getShardId(), context.currentParallelism())) + .add(value); + } + }); + // assigned split + readySplit.forEach( + (id, split) -> { + context.assignSplit(id, split); + if (discoveryIntervalMillis <= 0) { + context.signalNoMoreSplits(id); + } + }); + // record assigned split + assignedSplit.putAll(pendingSplit); + pendingSplit.clear(); + } + + private static int getSplitOwner(int shardId, int numReaders) { + return shardId % numReaders; + } + + @Override + public SlsSourceState snapshotState(long checkpointId) throws Exception { + return new SlsSourceState(new HashSet<>(assignedSplit.values())); + } + + public boolean checkConsumerGroupExists(String project, String logstore, String consumerGroup) + throws Exception { + ListConsumerGroupResponse response = this.slsCleint.ListConsumerGroup(project, logstore); + if (response != null) { + for (ConsumerGroup item : response.GetConsumerGroups()) { + if (item.getConsumerGroupName().equals(consumerGroup)) { + return true; + } + } + } + return false; + } + + public void createConsumerGroup( + final String project, final String logstore, final String consumerGroupName) + throws LogException { + ConsumerGroup consumerGroup = new ConsumerGroup(consumerGroupName, 100, false); + try { + this.slsCleint.CreateConsumerGroup(project, logstore, consumerGroup); + } catch (LogException ex) { + if ("ConsumerGroupAlreadyExist".equals(ex.GetErrorCode())) {} + + throw ex; + } + } +} diff --git a/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/state/SlsSourceState.java b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/state/SlsSourceState.java new file mode 100644 index 00000000000..bce5ecb1545 --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/src/main/java/org/apache/seatunnel/connectors/seatunnel/sls/state/SlsSourceState.java @@ -0,0 +1,39 @@ +/* + * 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 + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.sls.state; + +import org.apache.seatunnel.connectors.seatunnel.sls.source.SlsSourceSplit; + +import lombok.Data; + +import java.io.Serializable; +import java.util.Set; + +@Data +public class SlsSourceState implements Serializable { + + private Set assignedSplit; + + public SlsSourceState(Set assignedSplit) { + this.assignedSplit = assignedSplit; + } + + public Set getAssignedSplit() { + return this.assignedSplit; + } +} diff --git a/seatunnel-connectors-v2/connector-sls/src/test/java/org/apache/seatunnel/connectors/seatunnel/sls/SlsFactoryTest.java b/seatunnel-connectors-v2/connector-sls/src/test/java/org/apache/seatunnel/connectors/seatunnel/sls/SlsFactoryTest.java new file mode 100644 index 00000000000..1d7c2ab2da7 --- /dev/null +++ b/seatunnel-connectors-v2/connector-sls/src/test/java/org/apache/seatunnel/connectors/seatunnel/sls/SlsFactoryTest.java @@ -0,0 +1,31 @@ +/* + * 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 + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.sls; + +import org.apache.seatunnel.connectors.seatunnel.sls.source.SlsSourceFactory; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class SlsFactoryTest { + + @Test + void optionRule() { + Assertions.assertNotNull((new SlsSourceFactory()).optionRule()); + } +} diff --git a/seatunnel-connectors-v2/pom.xml b/seatunnel-connectors-v2/pom.xml index e0564a5572b..6bd4065d088 100644 --- a/seatunnel-connectors-v2/pom.xml +++ b/seatunnel-connectors-v2/pom.xml @@ -79,6 +79,7 @@ connector-web3j connector-milvus connector-activemq + connector-sls diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index a16d86cad5a..e919293aab8 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -919,6 +919,13 @@ optional provided + + + org.apache.seatunnel + connector-sls + ${project.version} + provided + apache-seatunnel-${project.version} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/pom.xml new file mode 100644 index 00000000000..fdaeffda918 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/pom.xml @@ -0,0 +1,36 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connector-v2-e2e + ${revision} + + + connector-sls-e2e + SeaTunnel : E2E : Connector V2 : Sls + + + + org.apache.seatunnel + connector-sls + ${project.version} + test + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/src/test/java/org/apache/seatunnel/e2e/connector/sls/SlsIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/src/test/java/org/apache/seatunnel/e2e/connector/sls/SlsIT.java new file mode 100644 index 00000000000..07d368ec8cd --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/src/test/java/org/apache/seatunnel/e2e/connector/sls/SlsIT.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.e2e.connector.sls; + +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.TestContainer; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; + +@Slf4j +@Disabled("Disabled because it needs user's personal sls account to run this test") +public class SlsIT extends TestSuiteBase implements TestResource { + + @BeforeEach + @Override + public void startUp() throws Exception {} + + @AfterEach + @Override + public void tearDown() throws Exception {} + + @TestTemplate + public void testSlsStreamingSource(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult execResult1 = + container.executeJob("/sls_source_with_schema_to_console.conf"); + Assertions.assertEquals(0, execResult1.getExitCode(), execResult1.getStderr()); + Container.ExecResult execResult2 = + container.executeJob("/sls_source_without_schema_to_console.conf"); + Assertions.assertEquals(0, execResult2.getExitCode(), execResult2.getStderr()); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/src/test/resources/sls_source_with_schema_to_console.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/src/test/resources/sls_source_with_schema_to_console.conf new file mode 100644 index 00000000000..0e9ca29d4ae --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/src/test/resources/sls_source_with_schema_to_console.conf @@ -0,0 +1,47 @@ +# +# 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. +# + +env { + parallelism = 1 + job.mode = "BATCH" + checkpoint.interval = 30000 +} + +source { + Sls { + endpoint = "xxxxxx" + project = "xxxxxx" + logstore = "xxxxxx" + access_key_id = "xxxxxx" + access_key_secret = "xxxxxxx" + schema = { + fields = { + id = "int" + name = "string" + description = "string" + weight = "string" + } + } + } +} + + + +sink { + Console { + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/src/test/resources/sls_source_without_schema_to_console.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/src/test/resources/sls_source_without_schema_to_console.conf new file mode 100644 index 00000000000..0d979b01579 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-sls-e2e/src/test/resources/sls_source_without_schema_to_console.conf @@ -0,0 +1,39 @@ +# +# 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. +# + +env { + parallelism = 1 + job.mode = "BATCH" + checkpoint.interval = 30000 +} + +source { + Sls { + endpoint = "xxxxxx" + project = "xxxxxx" + logstore = "xxxxxx" + access_key_id = "xxxxxx" + access_key_secret = "xxxxxxx" + } +} + + + +sink { + Console { + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index 2db67f88147..ed36310474a 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -76,6 +76,7 @@ connector-hudi-e2e connector-milvus-e2e connector-activemq-e2e + connector-sls-e2e From 464da8fb9bc4a01609f548b42fd15f2ba5ae7dae Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Tue, 13 Aug 2024 22:20:09 +0800 Subject: [PATCH 054/361] [Fix][Connector-V2] Fix doris primary key order and fields order are inconsistent (#7377) --- .../doris/util/DorisCatalogUtil.java | 8 ++-- .../doris/catalog/DorisCreateTableTest.java | 41 ++++++++++++++++++- 2 files changed, 44 insertions(+), 5 deletions(-) diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/DorisCatalogUtil.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/DorisCatalogUtil.java index 5025caed21c..dce94c0d0d6 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/DorisCatalogUtil.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/DorisCatalogUtil.java @@ -117,10 +117,10 @@ public static String getCreateTableStatement( String primaryKey = ""; if (tableSchema.getPrimaryKey() != null) { - primaryKey = - tableSchema.getPrimaryKey().getColumnNames().stream() - .map(r -> "`" + r + "`") - .collect(Collectors.joining(",")); + List fields = Arrays.asList(catalogTable.getTableSchema().getFieldNames()); + List keys = tableSchema.getPrimaryKey().getColumnNames(); + keys.sort(Comparator.comparingInt(fields::indexOf)); + primaryKey = keys.stream().map(r -> "`" + r + "`").collect(Collectors.joining(",")); } String uniqueKey = ""; if (!tableSchema.getConstraintKeys().isEmpty()) { diff --git a/seatunnel-connectors-v2/connector-doris/src/test/java/org/apache/seatunnel/connectors/doris/catalog/DorisCreateTableTest.java b/seatunnel-connectors-v2/connector-doris/src/test/java/org/apache/seatunnel/connectors/doris/catalog/DorisCreateTableTest.java index 09a5b6a3293..8dc508db7ba 100644 --- a/seatunnel-connectors-v2/connector-doris/src/test/java/org/apache/seatunnel/connectors/doris/catalog/DorisCreateTableTest.java +++ b/seatunnel-connectors-v2/connector-doris/src/test/java/org/apache/seatunnel/connectors/doris/catalog/DorisCreateTableTest.java @@ -33,6 +33,7 @@ import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.connectors.doris.config.DorisOptions; import org.apache.seatunnel.connectors.doris.datatype.DorisTypeConverterV1; +import org.apache.seatunnel.connectors.doris.datatype.DorisTypeConverterV2; import org.apache.seatunnel.connectors.doris.util.DorisCatalogUtil; import org.apache.commons.lang3.StringUtils; @@ -389,7 +390,45 @@ public void testWithThreePrimaryKeys() { + "`comment` VARCHAR(500) NULL ,\n" + "`description` STRING NULL \n" + " )\n" - + " partitioned by `id`,`age`,`name`;", + + " partitioned by `id`,`name`,`age`;", + result); + } + + @Test + public void testWithResortedMultiPrimaryKey() { + List columns = new ArrayList<>(); + + columns.add(PhysicalColumn.of("id", BasicType.LONG_TYPE, (Long) null, true, null, "")); + columns.add(PhysicalColumn.of("name", BasicType.STRING_TYPE, (Long) null, true, null, "")); + columns.add(PhysicalColumn.of("age", BasicType.INT_TYPE, (Long) null, true, null, "")); + + String result = + DorisCatalogUtil.getCreateTableStatement( + DorisOptions.SAVE_MODE_CREATE_TEMPLATE.defaultValue(), + TablePath.of("test1", "test2"), + CatalogTable.of( + TableIdentifier.of("test", "test1", "test2"), + TableSchema.builder() + .primaryKey(PrimaryKey.of("", Arrays.asList("age", "id"))) + .columns(columns) + .build(), + Collections.emptyMap(), + Collections.emptyList(), + ""), + DorisTypeConverterV2.INSTANCE); + Assertions.assertEquals( + "CREATE TABLE IF NOT EXISTS `test1`.`test2` (\n" + + "`id` BIGINT NULL ,`age` INT NULL ,\n" + + "`name` STRING NULL \n" + + ") ENGINE=OLAP\n" + + " UNIQUE KEY (`id`,`age`)\n" + + "DISTRIBUTED BY HASH (`id`,`age`)\n" + + " PROPERTIES (\n" + + "\"replication_allocation\" = \"tag.location.default: 1\",\n" + + "\"in_memory\" = \"false\",\n" + + "\"storage_format\" = \"V2\",\n" + + "\"disable_auto_compaction\" = \"false\"\n" + + ")", result); } } From 40c5f313eb1b56f9c6d81c805eb1b9a91d87cb1d Mon Sep 17 00:00:00 2001 From: hailin0 Date: Wed, 14 Aug 2024 13:34:12 +0800 Subject: [PATCH 055/361] [Hotfix][MySQL-CDC] Fix ArrayIndexOutOfBoundsException in mysql binlog read (#7381) --- .../binlog/io/BufferedSocketInputStream.java | 77 +++++++++++++++++++ .../io/BufferedSocketInputStreamTest.java | 51 ++++++++++++ 2 files changed, 128 insertions(+) create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/com/github/shyiko/mysql/binlog/io/BufferedSocketInputStream.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/test/java/com/github/shyiko/mysql/binlog/io/BufferedSocketInputStreamTest.java diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/com/github/shyiko/mysql/binlog/io/BufferedSocketInputStream.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/com/github/shyiko/mysql/binlog/io/BufferedSocketInputStream.java new file mode 100644 index 00000000000..78cf2865319 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/com/github/shyiko/mysql/binlog/io/BufferedSocketInputStream.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.github.shyiko.mysql.binlog.io; + +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; + +/** + * Copied from https://github.com/osheroff/mysql-binlog-connector-java project to fix + * https://github.com/apache/seatunnel/issues/7380 + * + *

reference: - https://github.com/osheroff/mysql-binlog-connector-java/issues/66 - + * https://github.com/apache/flink-cdc/issues/460 + */ +public class BufferedSocketInputStream extends FilterInputStream { + + private byte[] buffer; + private int offset; + private int limit; + + public BufferedSocketInputStream(InputStream in) { + this(in, 512 * 1024); + } + + public BufferedSocketInputStream(InputStream in, int bufferSize) { + super(in); + this.buffer = new byte[bufferSize]; + } + + @Override + public int available() throws IOException { + return limit == -1 ? in.available() : limit - offset + in.available(); + } + + @Override + public int read() throws IOException { + if (offset < limit) { + return buffer[offset++] & 0xff; + } + offset = 0; + limit = in.read(buffer, 0, buffer.length); + return limit != -1 ? buffer[offset++] & 0xff : -1; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + if (offset >= limit) { + if (len >= buffer.length) { + return in.read(b, off, len); + } + offset = 0; + limit = in.read(buffer, 0, buffer.length); + if (limit == -1) { + return limit; + } + } + int bytesRemainingInBuffer = Math.min(len, limit - offset); + System.arraycopy(buffer, offset, b, off, bytesRemainingInBuffer); + offset += bytesRemainingInBuffer; + return bytesRemainingInBuffer; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/test/java/com/github/shyiko/mysql/binlog/io/BufferedSocketInputStreamTest.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/test/java/com/github/shyiko/mysql/binlog/io/BufferedSocketInputStreamTest.java new file mode 100644 index 00000000000..cc3fad37a53 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/test/java/com/github/shyiko/mysql/binlog/io/BufferedSocketInputStreamTest.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.github.shyiko.mysql.binlog.io; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class BufferedSocketInputStreamTest { + + @Test + public void testReadFromBufferedSocketInputStream() throws Exception { + BufferedSocketInputStream in = + new BufferedSocketInputStream( + new ByteArrayInputStream( + new byte[] {'A', 'B', 'C', 'D', 'E', 'F', 'G', 'H'})); + byte[] buf = new byte[3]; + assertEquals(3, in.read(buf, 0, buf.length)); + Arrays.equals(new byte[] {'A', 'B', 'C'}, buf); + assertEquals(5, in.available()); + + assertEquals(3, in.read(buf, 0, buf.length)); + Arrays.equals(new byte[] {'D', 'E', 'F'}, buf); + assertEquals(2, in.available()); + + assertEquals(2, in.read(buf, 0, buf.length)); + Arrays.equals(new byte[] {'G', 'H'}, buf); + assertEquals(0, in.available()); + + // reach the end of stream normally + assertEquals(-1, in.read(buf, 0, buf.length)); + assertEquals(0, in.available()); + } +} From ff107ac6d252ba4b50b47b5d014b1400c9679915 Mon Sep 17 00:00:00 2001 From: Carl-Zhou-CN <1058249259@qq.com> Date: Wed, 14 Aug 2024 14:13:41 +0800 Subject: [PATCH 056/361] [Feature][Spark] Support multiple tables read and write (#7283) --- .../core/starter/execution/PluginUtil.java | 98 +- .../spark/execution/SinkExecuteProcessor.java | 51 +- .../spark/execution/SinkExecuteProcessor.java | 52 +- .../execution/SourceExecuteProcessor.java | 7 +- .../SparkAbstractPluginExecuteProcessor.java | 3 +- .../spark/execution/SparkExecution.java | 1 + .../execution/TransformExecuteProcessor.java | 17 +- .../ClickhouseSinkCDCChangelogIT.java | 6 - .../e2e/connector/druid/DruidIT.java | 4 +- .../elasticsearch/ElasticsearchIT.java | 4 +- .../connector/fake/FakeWithMultiTableTT.java | 2 +- .../local/LocalFileWithMultipleTableIT.java | 7 +- .../seatunnel/e2e/connector/http/HttpIT.java | 8 +- .../e2e/connector/influxdb/InfluxdbIT.java | 4 +- .../jdbc/JdbcMysqlMultipleTablesIT.java | 2 +- .../seatunnel/e2e/connector/kudu/KuduIT.java | 8 +- .../e2e/connector/redis/RedisIT.java | 4 +- .../translation/spark/sink/SparkSink.java | 10 +- .../spark/sink/SparkSinkInjector.java | 8 +- .../sink/writer/SparkDataSourceWriter.java | 8 +- .../spark/sink/writer/SparkDataWriter.java | 25 +- .../sink/writer/SparkDataWriterFactory.java | 9 +- .../spark/sink/writer/SparkStreamWriter.java | 4 +- .../spark/source/SeaTunnelSourceSupport.java | 33 +- .../partition/batch/BatchPartition.java | 11 +- .../partition/micro/MicroBatchPartition.java | 13 +- .../reader/batch/BatchSourceReader.java | 21 +- .../CoordinatedBatchPartitionReader.java | 9 +- .../batch/ParallelBatchPartitionReader.java | 11 +- .../CoordinatedMicroBatchPartitionReader.java | 10 +- .../reader/micro/MicroBatchSourceReader.java | 15 +- .../ParallelMicroBatchPartitionReader.java | 6 +- .../spark/sink/SeaTunnelBatchWrite.java | 8 +- .../spark/sink/SeaTunnelSinkTable.java | 10 +- .../spark/sink/SparkSinkInjector.java | 10 +- .../sink/write/SeaTunnelSparkDataWriter.java | 25 +- .../SeaTunnelSparkDataWriterFactory.java | 9 +- .../spark/sink/write/SeaTunnelWrite.java | 10 +- .../sink/write/SeaTunnelWriteBuilder.java | 8 +- .../spark/source/SeaTunnelSourceTable.java | 22 +- .../CoordinatedBatchPartitionReader.java | 9 +- .../batch/ParallelBatchPartitionReader.java | 10 +- .../partition/batch/SeaTunnelBatch.java | 10 +- .../SeaTunnelBatchPartitionReaderFactory.java | 11 +- .../CoordinatedMicroBatchPartitionReader.java | 10 +- .../ParallelMicroBatchPartitionReader.java | 6 +- .../partition/micro/SeaTunnelMicroBatch.java | 14 +- ...unnelMicroBatchPartitionReaderFactory.java | 13 +- .../spark/source/scan/SeaTunnelScan.java | 19 +- .../source/scan/SeaTunnelScanBuilder.java | 10 +- .../translation/spark/sink/SparkSinkTest.java | 19 +- .../spark/execution/ColumnWithIndex.java | 53 + .../spark/execution/DatasetTableInfo.java | 17 +- .../spark/execution/IndexQueue.java | 54 + .../spark/execution/MultiTableManager.java | 160 +++ .../InternalMultiRowCollector.java | 58 ++ .../serialization/InternalRowCollector.java | 19 +- .../serialization/InternalRowConverter.java | 56 +- .../serialization/SeaTunnelRowConverter.java | 33 +- .../spark/utils/TypeConverterUtils.java | 17 +- .../execution/MultiTableManagerTest.java | 950 ++++++++++++++++++ 61 files changed, 1840 insertions(+), 281 deletions(-) create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/ColumnWithIndex.java rename {seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter => seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation}/spark/execution/DatasetTableInfo.java (74%) create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/IndexQueue.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/MultiTableManager.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalMultiRowCollector.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/test/java/org/apache/seatunnel/translation/spark/execution/MultiTableManagerTest.java diff --git a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/PluginUtil.java b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/PluginUtil.java index c47ea0b1215..e8a682062e5 100644 --- a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/PluginUtil.java +++ b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/execution/PluginUtil.java @@ -23,17 +23,21 @@ import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.ConfigValidator; +import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.connector.TableSource; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.FactoryException; +import org.apache.seatunnel.api.table.factory.FactoryUtil; +import org.apache.seatunnel.api.table.factory.TableSinkFactory; +import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.common.constants.JobMode; -import org.apache.seatunnel.common.utils.SeaTunnelException; +import org.apache.seatunnel.core.starter.enums.PluginType; import org.apache.seatunnel.plugin.discovery.PluginIdentifier; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelFactoryDiscovery; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSinkPluginDiscovery; @@ -43,7 +47,9 @@ import com.google.common.collect.Lists; import java.net.URL; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Optional; import static org.apache.seatunnel.api.common.CommonOptions.PLUGIN_NAME; @@ -97,10 +103,11 @@ public static SourceTableInfo createSource( CatalogTableUtil.convertDataTypeToCatalogTables(seaTunnelDataType, tableId); } - if (catalogTables.size() != 1) { - throw new SeaTunnelException( - String.format("Unsupported table number: %d on flink", catalogTables.size())); - } + // if (catalogTables.size() != 1) { + // throw new SeaTunnelException( + // String.format("Unsupported table number: %d on flink", + // catalogTables.size())); + // } return new SourceTableInfo(source, catalogTables); } @@ -162,6 +169,87 @@ public static Optional createSinkFactory( } } + public static SeaTunnelSink createSink( + Optional factory, + Config sinkConfig, + SeaTunnelSinkPluginDiscovery sinkPluginDiscovery, + JobContext jobContext, + List catalogTables, + ClassLoader classLoader) { + boolean fallBack = !factory.isPresent() || isFallback(factory.get()); + if (fallBack) { + SeaTunnelSink sink = + fallbackCreateSink( + sinkPluginDiscovery, + PluginIdentifier.of( + ENGINE_TYPE, + PluginType.SINK.getType(), + sinkConfig.getString(PLUGIN_NAME.key())), + sinkConfig); + sink.setJobContext(jobContext); + sink.setTypeInfo(catalogTables.get(0).getSeaTunnelRowType()); + return sink; + } else { + if (catalogTables.size() > 1) { + Map sinks = new HashMap<>(); + ReadonlyConfig readonlyConfig = ReadonlyConfig.fromConfig(sinkConfig); + catalogTables.forEach( + catalogTable -> { + TableSinkFactoryContext context = + TableSinkFactoryContext.replacePlaceholderAndCreate( + catalogTable, + ReadonlyConfig.fromConfig(sinkConfig), + classLoader, + ((TableSinkFactory) factory.get()) + .excludeTablePlaceholderReplaceKeys()); + ConfigValidator.of(context.getOptions()) + .validate(factory.get().optionRule()); + SeaTunnelSink action = + ((TableSinkFactory) factory.get()) + .createSink(context) + .createSink(); + action.setJobContext(jobContext); + sinks.put(catalogTable.getTablePath().toString(), action); + }); + return FactoryUtil.createMultiTableSink(sinks, readonlyConfig, classLoader); + } + TableSinkFactoryContext context = + TableSinkFactoryContext.replacePlaceholderAndCreate( + catalogTables.get(0), + ReadonlyConfig.fromConfig(sinkConfig), + classLoader, + ((TableSinkFactory) factory.get()) + .excludeTablePlaceholderReplaceKeys()); + ConfigValidator.of(context.getOptions()).validate(factory.get().optionRule()); + SeaTunnelSink sink = + ((TableSinkFactory) factory.get()).createSink(context).createSink(); + sink.setJobContext(jobContext); + return sink; + } + } + + public static boolean isFallback(Factory factory) { + try { + ((TableSinkFactory) factory).createSink(null); + } catch (Exception e) { + if (e instanceof UnsupportedOperationException + && "The Factory has not been implemented and the deprecated Plugin will be used." + .equals(e.getMessage())) { + return true; + } + } + return false; + } + + public static SeaTunnelSink fallbackCreateSink( + SeaTunnelSinkPluginDiscovery sinkPluginDiscovery, + PluginIdentifier pluginIdentifier, + Config pluginConfig) { + SeaTunnelSink source = sinkPluginDiscovery.createPluginInstance(pluginIdentifier); + source.prepare(pluginConfig); + return source; + } + public static void ensureJobModeMatch(JobContext jobContext, SeaTunnelSource source) { if (jobContext.getJobMode() == JobMode.BATCH && source.getBoundedness() diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java index 7751286b227..0f3fc750967 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java @@ -21,17 +21,13 @@ import org.apache.seatunnel.api.common.CommonOptions; import org.apache.seatunnel.api.common.JobContext; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.configuration.util.ConfigValidator; import org.apache.seatunnel.api.sink.SaveModeExecuteWrapper; import org.apache.seatunnel.api.sink.SaveModeHandler; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.sink.SupportSaveMode; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; -import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.core.starter.enums.PluginType; import org.apache.seatunnel.core.starter.exception.TaskExecuteException; @@ -39,6 +35,7 @@ import org.apache.seatunnel.plugin.discovery.PluginIdentifier; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelFactoryDiscovery; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSinkPluginDiscovery; +import org.apache.seatunnel.translation.spark.execution.DatasetTableInfo; import org.apache.seatunnel.translation.spark.sink.SparkSinkInjector; import org.apache.spark.sql.Dataset; @@ -50,7 +47,6 @@ import java.util.Optional; import java.util.stream.Collectors; -import static org.apache.seatunnel.api.common.CommonOptions.PLUGIN_NAME; import static org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode.HANDLE_SAVE_MODE_FAILED; public class SinkExecuteProcessor @@ -97,8 +93,6 @@ public List execute(List upstreamDataStreams DatasetTableInfo datasetTableInfo = fromSourceTable(sinkConfig, sparkRuntimeEnvironment, upstreamDataStreams) .orElse(input); - SeaTunnelDataType inputType = - datasetTableInfo.getCatalogTable().getSeaTunnelRowType(); Dataset dataset = datasetTableInfo.getDataset(); int parallelism; @@ -114,31 +108,14 @@ public List execute(List upstreamDataStreams } dataset.sparkSession().read().option(CommonOptions.PARALLELISM.key(), parallelism); Optional factory = plugins.get(i); - boolean fallBack = !factory.isPresent() || isFallback(factory.get()); - SeaTunnelSink sink; - if (fallBack) { - sink = - fallbackCreateSink( - sinkPluginDiscovery, - PluginIdentifier.of( - ENGINE_TYPE, - PLUGIN_TYPE, - sinkConfig.getString(PLUGIN_NAME.key())), - sinkConfig); - sink.setJobContext(jobContext); - sink.setTypeInfo((SeaTunnelRowType) inputType); - } else { - TableSinkFactoryContext context = - TableSinkFactoryContext.replacePlaceholderAndCreate( - datasetTableInfo.getCatalogTable(), - ReadonlyConfig.fromConfig(sinkConfig), - classLoader, - ((TableSinkFactory) factory.get()) - .excludeTablePlaceholderReplaceKeys()); - ConfigValidator.of(context.getOptions()).validate(factory.get().optionRule()); - sink = ((TableSinkFactory) factory.get()).createSink(context).createSink(); - sink.setJobContext(jobContext); - } + SeaTunnelSink sink = + PluginUtil.createSink( + factory, + sinkConfig, + sinkPluginDiscovery, + jobContext, + datasetTableInfo.getCatalogTables(), + classLoader); // TODO modify checkpoint location if (SupportSaveMode.class.isAssignableFrom(sink.getClass())) { SupportSaveMode saveModeSink = (SupportSaveMode) sink; @@ -153,11 +130,9 @@ public List execute(List upstreamDataStreams } String applicationId = sparkRuntimeEnvironment.getSparkSession().sparkContext().applicationId(); - SparkSinkInjector.inject( - dataset.write(), - sink, - datasetTableInfo.getCatalogTable(), - applicationId) + CatalogTable[] catalogTables = + datasetTableInfo.getCatalogTables().toArray(new CatalogTable[0]); + SparkSinkInjector.inject(dataset.write(), sink, catalogTables, applicationId) .option("checkpointLocation", "/tmp") .save(); } diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java index 46b3233b00e..29ff90cfb8a 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java @@ -21,17 +21,13 @@ import org.apache.seatunnel.api.common.CommonOptions; import org.apache.seatunnel.api.common.JobContext; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.configuration.util.ConfigValidator; import org.apache.seatunnel.api.sink.SaveModeExecuteWrapper; import org.apache.seatunnel.api.sink.SaveModeHandler; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.sink.SupportSaveMode; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; -import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.core.starter.enums.PluginType; import org.apache.seatunnel.core.starter.exception.TaskExecuteException; @@ -39,6 +35,7 @@ import org.apache.seatunnel.plugin.discovery.PluginIdentifier; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelFactoryDiscovery; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSinkPluginDiscovery; +import org.apache.seatunnel.translation.spark.execution.DatasetTableInfo; import org.apache.seatunnel.translation.spark.sink.SparkSinkInjector; import org.apache.spark.sql.Dataset; @@ -51,7 +48,6 @@ import java.util.Optional; import java.util.stream.Collectors; -import static org.apache.seatunnel.api.common.CommonOptions.PLUGIN_NAME; import static org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode.HANDLE_SAVE_MODE_FAILED; public class SinkExecuteProcessor @@ -98,10 +94,7 @@ public List execute(List upstreamDataStreams DatasetTableInfo datasetTableInfo = fromSourceTable(sinkConfig, sparkRuntimeEnvironment, upstreamDataStreams) .orElse(input); - SeaTunnelDataType inputType = - datasetTableInfo.getCatalogTable().getSeaTunnelRowType(); Dataset dataset = datasetTableInfo.getDataset(); - int parallelism; if (sinkConfig.hasPath(CommonOptions.PARALLELISM.key())) { parallelism = sinkConfig.getInt(CommonOptions.PARALLELISM.key()); @@ -115,31 +108,14 @@ public List execute(List upstreamDataStreams } dataset.sparkSession().read().option(CommonOptions.PARALLELISM.key(), parallelism); Optional factory = plugins.get(i); - boolean fallBack = !factory.isPresent() || isFallback(factory.get()); - SeaTunnelSink sink; - if (fallBack) { - sink = - fallbackCreateSink( - sinkPluginDiscovery, - PluginIdentifier.of( - ENGINE_TYPE, - PLUGIN_TYPE, - sinkConfig.getString(PLUGIN_NAME.key())), - sinkConfig); - sink.setJobContext(jobContext); - sink.setTypeInfo((SeaTunnelRowType) inputType); - } else { - TableSinkFactoryContext context = - TableSinkFactoryContext.replacePlaceholderAndCreate( - datasetTableInfo.getCatalogTable(), - ReadonlyConfig.fromConfig(sinkConfig), - classLoader, - ((TableSinkFactory) factory.get()) - .excludeTablePlaceholderReplaceKeys()); - ConfigValidator.of(context.getOptions()).validate(factory.get().optionRule()); - sink = ((TableSinkFactory) factory.get()).createSink(context).createSink(); - sink.setJobContext(jobContext); - } + SeaTunnelSink sink = + PluginUtil.createSink( + factory, + sinkConfig, + sinkPluginDiscovery, + jobContext, + datasetTableInfo.getCatalogTables(), + classLoader); // TODO modify checkpoint location if (SupportSaveMode.class.isAssignableFrom(sink.getClass())) { SupportSaveMode saveModeSink = (SupportSaveMode) sink; @@ -154,11 +130,9 @@ public List execute(List upstreamDataStreams } String applicationId = sparkRuntimeEnvironment.getStreamingContext().sparkContext().applicationId(); - SparkSinkInjector.inject( - dataset.write(), - sink, - datasetTableInfo.getCatalogTable(), - applicationId) + CatalogTable[] catalogTables = + datasetTableInfo.getCatalogTables().toArray(new CatalogTable[0]); + SparkSinkInjector.inject(dataset.write(), sink, catalogTables, applicationId) .option("checkpointLocation", "/tmp") .mode(SaveMode.Append) .save(); diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java index cf81ab60a0f..d4a4038e485 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java @@ -31,11 +31,10 @@ import org.apache.seatunnel.plugin.discovery.PluginIdentifier; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelFactoryDiscovery; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSourcePluginDiscovery; -import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; +import org.apache.seatunnel.translation.spark.execution.DatasetTableInfo; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.types.StructType; import com.google.common.collect.Lists; @@ -87,7 +86,6 @@ public List execute(List upstreamDataStreams CommonOptions.PARALLELISM.key(), CommonOptions.PARALLELISM.defaultValue()); } - StructType schema = (StructType) TypeConverterUtils.convert(source.getProducedType()); Dataset dataset = sparkRuntimeEnvironment .getSparkSession() @@ -98,12 +96,11 @@ public List execute(List upstreamDataStreams Constants.SOURCE_SERIALIZATION, SerializationUtils.objectToString(source)) .options(envOption) - .schema(schema) .load(); sources.add( new DatasetTableInfo( dataset, - sourceTableInfo.getCatalogTables().get(0), + sourceTableInfo.getCatalogTables(), pluginConfig.hasPath(RESULT_TABLE_NAME.key()) ? pluginConfig.getString(RESULT_TABLE_NAME.key()) : null)); diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java index a431f23dd16..f1c138285ad 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java @@ -22,6 +22,7 @@ import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.core.starter.execution.PluginExecuteProcessor; +import org.apache.seatunnel.translation.spark.execution.DatasetTableInfo; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -84,7 +85,7 @@ protected Optional fromSourceTable( return Optional.of( new DatasetTableInfo( sparkRuntimeEnvironment.getSparkSession().read().table(sourceTableName), - datasetTableInfo.getCatalogTable(), + datasetTableInfo.getCatalogTables(), sourceTableName)); } diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java index 48aa9abb2a3..55c145bc3b6 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java @@ -26,6 +26,7 @@ import org.apache.seatunnel.core.starter.execution.PluginExecuteProcessor; import org.apache.seatunnel.core.starter.execution.RuntimeEnvironment; import org.apache.seatunnel.core.starter.execution.TaskExecution; +import org.apache.seatunnel.translation.spark.execution.DatasetTableInfo; import lombok.extern.slf4j.Slf4j; diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java index fc4a9e00d0d..43ef1afceeb 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java @@ -31,6 +31,7 @@ import org.apache.seatunnel.core.starter.execution.PluginUtil; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelFactoryDiscovery; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelTransformPluginDiscovery; +import org.apache.seatunnel.translation.spark.execution.DatasetTableInfo; import org.apache.seatunnel.translation.spark.serialization.SeaTunnelRowConverter; import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; @@ -110,7 +111,7 @@ public List execute(List upstreamDataStreams TableTransformFactory factory = plugins.get(i); TableTransformFactoryContext context = new TableTransformFactoryContext( - Collections.singletonList(dataset.getCatalogTable()), + dataset.getCatalogTables(), ReadonlyConfig.fromConfig(pluginConfig), classLoader); ConfigValidator.of(context.getOptions()).validate(factory.optionRule()); @@ -121,7 +122,7 @@ public List execute(List upstreamDataStreams upstreamDataStreams.add( new DatasetTableInfo( inputDataset, - transform.getProducedCatalogTable(), + Collections.singletonList(transform.getProducedCatalogTable()), pluginConfig.hasPath(RESULT_TABLE_NAME.key()) ? pluginConfig.getString(RESULT_TABLE_NAME.key()) : null)); @@ -138,10 +139,11 @@ public List execute(List upstreamDataStreams private Dataset sparkTransform(SeaTunnelTransform transform, DatasetTableInfo tableInfo) { Dataset stream = tableInfo.getDataset(); - SeaTunnelDataType inputDataType = tableInfo.getCatalogTable().getSeaTunnelRowType(); + SeaTunnelDataType inputDataType = + tableInfo.getCatalogTables().get(0).getSeaTunnelRowType(); SeaTunnelDataType outputDataTYpe = transform.getProducedCatalogTable().getSeaTunnelRowType(); - StructType outputSchema = (StructType) TypeConverterUtils.convert(outputDataTYpe); + StructType outputSchema = (StructType) TypeConverterUtils.parcel(outputDataTYpe); SeaTunnelRowConverter inputRowConverter = new SeaTunnelRowConverter(inputDataType); SeaTunnelRowConverter outputRowConverter = new SeaTunnelRowConverter(outputDataTYpe); ExpressionEncoder encoder = RowEncoder.apply(outputSchema); @@ -187,15 +189,12 @@ public boolean hasNext() { public Row next() { try { Row row = sourceIterator.next(); - SeaTunnelRow seaTunnelRow = - inputRowConverter.reconvert( - new SeaTunnelRow(((GenericRowWithSchema) row).values())); + SeaTunnelRow seaTunnelRow = inputRowConverter.unpack((GenericRowWithSchema) row); seaTunnelRow = (SeaTunnelRow) transform.map(seaTunnelRow); if (seaTunnelRow == null) { return null; } - seaTunnelRow = outputRowConverter.convert(seaTunnelRow); - return new GenericRowWithSchema(seaTunnelRow.getFields(), structType); + return outputRowConverter.parcel(seaTunnelRow); } catch (Exception e) { throw new TaskExecuteException("Row convert failed, caused: " + e.getMessage(), e); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-clickhouse-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/ClickhouseSinkCDCChangelogIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-clickhouse-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/ClickhouseSinkCDCChangelogIT.java index 4c2b9cedb18..3d46ac8c553 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-clickhouse-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/ClickhouseSinkCDCChangelogIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-clickhouse-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/ClickhouseSinkCDCChangelogIT.java @@ -19,9 +19,7 @@ import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; -import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; -import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.awaitility.Awaitility; import org.junit.jupiter.api.AfterAll; @@ -50,10 +48,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -@DisabledOnContainer( - value = {}, - type = {EngineType.SPARK}, - disabledReason = "Spark engine will lose the row kind of record") @Slf4j public class ClickhouseSinkCDCChangelogIT extends TestSuiteBase implements TestResource { private static final String CLICKHOUSE_DOCKER_IMAGE = "clickhouse/clickhouse-server:23.3.13.6"; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-druid-e2e/src/test/java/org/apache/seatunnel/e2e/connector/druid/DruidIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-druid-e2e/src/test/java/org/apache/seatunnel/e2e/connector/druid/DruidIT.java index 21bf3c5d5cd..21463e4917d 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-druid-e2e/src/test/java/org/apache/seatunnel/e2e/connector/druid/DruidIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-druid-e2e/src/test/java/org/apache/seatunnel/e2e/connector/druid/DruidIT.java @@ -119,8 +119,8 @@ public void testDruidSink(TestContainer container) throws Exception { @DisabledOnContainer( value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK/FLINK do not support multiple table read") + type = {EngineType.FLINK}, + disabledReason = "Currently FLINK do not support multiple table read") @TestTemplate public void testDruidMultiSink(TestContainer container) throws Exception { Container.ExecResult execResult = diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/java/org/apache/seatunnel/e2e/connector/elasticsearch/ElasticsearchIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/java/org/apache/seatunnel/e2e/connector/elasticsearch/ElasticsearchIT.java index 623dd9d2210..375a6cc0491 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/java/org/apache/seatunnel/e2e/connector/elasticsearch/ElasticsearchIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/java/org/apache/seatunnel/e2e/connector/elasticsearch/ElasticsearchIT.java @@ -186,8 +186,8 @@ public void testElasticsearch(TestContainer container) @DisabledOnContainer( value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK/FLINK do not support multiple table read") + type = {EngineType.FLINK}, + disabledReason = "Currently FLINK do not support multiple table read") @TestTemplate public void testElasticsearchWithMultiSink(TestContainer container) throws IOException, InterruptedException { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithMultiTableTT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithMultiTableTT.java index 2825f9d7dac..798a2a89210 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithMultiTableTT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fake-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fake/FakeWithMultiTableTT.java @@ -30,7 +30,7 @@ @DisabledOnContainer( value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, + type = {EngineType.FLINK}, disabledReason = "Currently SPARK and FLINK do not support multi-table") public class FakeWithMultiTableTT extends TestSuiteBase { @TestTemplate diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/local/LocalFileWithMultipleTableIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/local/LocalFileWithMultipleTableIT.java index 35f29e635fa..77e635c2f74 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/local/LocalFileWithMultipleTableIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/local/LocalFileWithMultipleTableIT.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.container.TestContainerId; import org.apache.seatunnel.e2e.common.container.TestHelper; import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; @@ -31,9 +32,9 @@ import java.io.IOException; @DisabledOnContainer( - value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support multi table") + value = {TestContainerId.SPARK_2_4}, + type = {EngineType.FLINK}, + disabledReason = "Currently FLINK do not support multi table") public class LocalFileWithMultipleTableIT extends TestSuiteBase { /** Copy data files to container */ diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/src/test/java/org/apache/seatunnel/e2e/connector/http/HttpIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/src/test/java/org/apache/seatunnel/e2e/connector/http/HttpIT.java index af1d7125eae..ab8acd1f865 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/src/test/java/org/apache/seatunnel/e2e/connector/http/HttpIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/src/test/java/org/apache/seatunnel/e2e/connector/http/HttpIT.java @@ -32,6 +32,7 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; import org.mockserver.client.MockServerClient; +import org.mockserver.model.ClearType; import org.mockserver.model.Format; import org.testcontainers.containers.Container; import org.testcontainers.containers.GenericContainer; @@ -232,8 +233,8 @@ public void testSourceToAssertSink(TestContainer container) @DisabledOnContainer( value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK/FLINK do not support multiple table read") + type = {EngineType.FLINK}, + disabledReason = "Currently FLINK do not support multiple table read") @TestTemplate public void testMultiTableHttp(TestContainer container) throws IOException, InterruptedException { @@ -245,6 +246,9 @@ public void testMultiTableHttp(TestContainer container) mockServerClient.retrieveRecordedRequests( request().withPath("/example/httpMultiTableContentSink").withMethod("POST"), Format.JSON); + mockServerClient.clear( + request().withPath("/example/httpMultiTableContentSink").withMethod("POST"), + ClearType.LOG); List recordResponse = objectMapper.readValue(mockResponse, new TypeReference>() {}); recordResponse = diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-influxdb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/influxdb/InfluxdbIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-influxdb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/influxdb/InfluxdbIT.java index c139afc5e87..74310f78689 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-influxdb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/influxdb/InfluxdbIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-influxdb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/influxdb/InfluxdbIT.java @@ -249,8 +249,8 @@ public void testInfluxdbWithTz(TestContainer container) @TestTemplate @DisabledOnContainer( value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK/FLINK do not support multiple table read") + type = {EngineType.FLINK}, + disabledReason = "Currently FLINK do not support multiple table read") public void testInfluxdbMultipleWrite(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlMultipleTablesIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlMultipleTablesIT.java index 9cb23f2c517..494d2738791 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlMultipleTablesIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlMultipleTablesIT.java @@ -171,7 +171,7 @@ public void testMysqlJdbcSingleTableE2e(TestContainer container) @DisabledOnContainer( value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, + type = {EngineType.FLINK}, disabledReason = "Currently SPARK and FLINK do not support multiple tables") @TestTemplate public void testMysqlJdbcMultipleTableE2e(TestContainer container) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kudu-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kudu/KuduIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kudu-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kudu/KuduIT.java index dc7ab433868..e8e61d39314 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kudu-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kudu/KuduIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kudu-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kudu/KuduIT.java @@ -387,8 +387,8 @@ public void testCdcKudu(TestContainer container) throws IOException, Interrupted @DisabledOnContainer( value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK/FLINK do not support multiple table read") + type = {EngineType.FLINK}, + disabledReason = "Currently FLINK do not support multiple table read") @TestTemplate public void testKuduMultipleRead(TestContainer container) throws IOException, InterruptedException { @@ -405,8 +405,8 @@ public void testKuduMultipleRead(TestContainer container) @DisabledOnContainer( value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK/FLINK do not support multiple table read") + type = {EngineType.FLINK}, + disabledReason = "Currently FLINK do not support multiple table read") @TestTemplate public void testKuduMultipleWrite(TestContainer container) throws IOException, InterruptedException { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-redis-e2e/src/test/java/org/apache/seatunnel/e2e/connector/redis/RedisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-redis-e2e/src/test/java/org/apache/seatunnel/e2e/connector/redis/RedisIT.java index 0b9ed1bedbd..280ca94faf1 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-redis-e2e/src/test/java/org/apache/seatunnel/e2e/connector/redis/RedisIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-redis-e2e/src/test/java/org/apache/seatunnel/e2e/connector/redis/RedisIT.java @@ -339,8 +339,8 @@ public void testScanZsetTypeWriteRedis(TestContainer container) @TestTemplate @DisabledOnContainer( value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK/FLINK do not support multiple table read") + type = {EngineType.FLINK}, + disabledReason = "Currently FLINK do not support multiple table read") public void testMultipletableRedisSink(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/SparkSink.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/SparkSink.java index 96438d3ac73..0315f275ff9 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/SparkSink.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/SparkSink.java @@ -43,7 +43,7 @@ public class SparkSink private volatile SeaTunnelSink sink; - private volatile CatalogTable catalogTable; + private volatile CatalogTable[] catalogTables; private volatile String jobId; @@ -58,8 +58,8 @@ private void init(DataSourceOptions options) { "can not find sink " + "class string in DataSourceOptions"))); } - if (catalogTable == null) { - this.catalogTable = + if (catalogTables == null) { + this.catalogTables = SerializationUtils.stringToObject( options.get(SparkSinkInjector.SINK_CATALOG_TABLE) .orElseThrow( @@ -79,7 +79,7 @@ public StreamWriter createStreamWriter( init(options); try { - return new SparkStreamWriter<>(sink, catalogTable, jobId); + return new SparkStreamWriter<>(sink, catalogTables, jobId); } catch (IOException e) { throw new RuntimeException("find error when createStreamWriter", e); } @@ -91,7 +91,7 @@ public Optional createWriter( init(options); try { - return Optional.of(new SparkDataSourceWriter<>(sink, catalogTable, jobId)); + return Optional.of(new SparkDataSourceWriter<>(sink, catalogTables, jobId)); } catch (IOException e) { throw new RuntimeException("find error when createStreamWriter", e); } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/SparkSinkInjector.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/SparkSinkInjector.java index 44aec8f70a2..740188339c5 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/SparkSinkInjector.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/SparkSinkInjector.java @@ -38,23 +38,23 @@ public class SparkSinkInjector { public static DataStreamWriter inject( DataStreamWriter dataset, SeaTunnelSink sink, - CatalogTable catalogTable, + CatalogTable[] catalogTables, String applicationId) { return dataset.format(SPARK_SINK_CLASS_NAME) .outputMode(OutputMode.Append()) .option(Constants.SINK_SERIALIZATION, SerializationUtils.objectToString(sink)) - .option(SINK_CATALOG_TABLE, SerializationUtils.objectToString(catalogTable)) + .option(SINK_CATALOG_TABLE, SerializationUtils.objectToString(catalogTables)) .option(JOB_ID, applicationId); } public static DataFrameWriter inject( DataFrameWriter dataset, SeaTunnelSink sink, - CatalogTable catalogTable, + CatalogTable[] catalogTables, String applicationId) { return dataset.format(SPARK_SINK_CLASS_NAME) .option(Constants.SINK_SERIALIZATION, SerializationUtils.objectToString(sink)) - .option(SINK_CATALOG_TABLE, SerializationUtils.objectToString(catalogTable)) + .option(SINK_CATALOG_TABLE, SerializationUtils.objectToString(catalogTables)) .option(JOB_ID, applicationId); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataSourceWriter.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataSourceWriter.java index b02ae32fbc0..29755c6db44 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataSourceWriter.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataSourceWriter.java @@ -47,18 +47,18 @@ public class SparkDataSourceWriter @Nullable protected final SinkAggregatedCommitter sinkAggregatedCommitter; - protected final CatalogTable catalogTable; + protected final CatalogTable[] catalogTables; protected final String jobId; private MultiTableResourceManager resourceManager; public SparkDataSourceWriter( SeaTunnelSink sink, - CatalogTable catalogTable, + CatalogTable[] catalogTables, String jobId) throws IOException { this.sink = sink; - this.catalogTable = catalogTable; + this.catalogTables = catalogTables; this.jobId = jobId; this.sinkAggregatedCommitter = sink.createAggregatedCommitter().orElse(null); if (sinkAggregatedCommitter != null) { @@ -78,7 +78,7 @@ public SparkDataSourceWriter( @Override public DataWriterFactory createWriterFactory() { - return new SparkDataWriterFactory<>(sink, catalogTable, jobId); + return new SparkDataWriterFactory<>(sink, catalogTables, jobId); } @Override diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriter.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriter.java index 95d4c840c17..434b1ef9799 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriter.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriter.java @@ -21,10 +21,8 @@ import org.apache.seatunnel.api.sink.SinkCommitter; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportResourceShare; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.translation.serialization.RowConverter; -import org.apache.seatunnel.translation.spark.serialization.InternalRowConverter; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.writer.DataWriter; @@ -41,32 +39,33 @@ @Slf4j public class SparkDataWriter implements DataWriter { - private final SinkWriter sinkWriter; + protected final SinkWriter sinkWriter; - @Nullable private final SinkCommitter sinkCommitter; - private final RowConverter rowConverter; - private CommitInfoT latestCommitInfoT; - private long epochId; - private MultiTableResourceManager resourceManager; + @Nullable protected final SinkCommitter sinkCommitter; + protected CommitInfoT latestCommitInfoT; + protected long epochId; + protected volatile MultiTableResourceManager resourceManager; + + private final MultiTableManager multiTableManager; SparkDataWriter( SinkWriter sinkWriter, @Nullable SinkCommitter sinkCommitter, - SeaTunnelDataType dataType, + MultiTableManager multiTableManager, long epochId) { this.sinkWriter = sinkWriter; this.sinkCommitter = sinkCommitter; - this.rowConverter = new InternalRowConverter(dataType); this.epochId = epochId == 0 ? 1 : epochId; + this.multiTableManager = multiTableManager; initResourceManger(); } @Override public void write(InternalRow record) throws IOException { - sinkWriter.write(rowConverter.reconvert(record)); + sinkWriter.write(multiTableManager.reconvert(record)); } - private void initResourceManger() { + protected void initResourceManger() { if (sinkWriter instanceof SupportResourceShare) { resourceManager = ((SupportResourceShare) sinkWriter).initMultiTableResourceManager(1, 1); diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriterFactory.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriterFactory.java index a9c6cd445a1..3a646f3aca2 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriterFactory.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriterFactory.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.writer.DataWriter; @@ -33,15 +34,15 @@ public class SparkDataWriterFactory implements DataWriterFactory { private final SeaTunnelSink sink; - private final CatalogTable catalogTable; + private final CatalogTable[] catalogTables; private final String jobId; SparkDataWriterFactory( SeaTunnelSink sink, - CatalogTable catalogTable, + CatalogTable[] catalogTables, String jobId) { this.sink = sink; - this.catalogTable = catalogTable; + this.catalogTables = catalogTables; this.jobId = jobId; } @@ -62,6 +63,6 @@ public DataWriter createDataWriter(int partitionId, long taskId, lo throw new RuntimeException("Failed to create SinkCommitter.", e); } return new SparkDataWriter<>( - writer, committer, catalogTable.getSeaTunnelRowType(), epochId); + writer, committer, new MultiTableManager(catalogTables), epochId); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkStreamWriter.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkStreamWriter.java index 4d297454f8b..1c8bbb4b875 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkStreamWriter.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkStreamWriter.java @@ -34,10 +34,10 @@ public class SparkStreamWriter public SparkStreamWriter( SeaTunnelSink sink, - CatalogTable catalogTable, + CatalogTable[] catalogTables, String jobId) throws IOException { - super(sink, catalogTable, jobId); + super(sink, catalogTables, jobId); } @Override diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/SeaTunnelSourceSupport.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/SeaTunnelSourceSupport.java index be97f39752a..ad6ee7a723c 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/SeaTunnelSourceSupport.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/SeaTunnelSourceSupport.java @@ -20,9 +20,13 @@ import org.apache.seatunnel.api.common.CommonOptions; import org.apache.seatunnel.api.env.EnvCommonOptions; import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.Constants; import org.apache.seatunnel.common.utils.SerializationUtils; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.source.reader.batch.BatchSourceReader; import org.apache.seatunnel.translation.spark.source.reader.micro.MicroBatchSourceReader; @@ -42,6 +46,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.List; import java.util.Map; import java.util.Optional; @@ -67,7 +72,19 @@ public DataSourceReader createReader(DataSourceOptions options) { int parallelism = options.getInt(CommonOptions.PARALLELISM.key(), 1); Map envOptions = options.asMap(); String applicationId = SparkSession.getActiveSession().get().sparkContext().applicationId(); - return new BatchSourceReader(seaTunnelSource, applicationId, parallelism, envOptions); + List catalogTables; + try { + catalogTables = seaTunnelSource.getProducedCatalogTables(); + } catch (UnsupportedOperationException e) { + // TODO remove it when all connector use `getProducedCatalogTables` + SeaTunnelDataType seaTunnelDataType = seaTunnelSource.getProducedType(); + catalogTables = + CatalogTableUtil.convertDataTypeToCatalogTables(seaTunnelDataType, "default"); + } + MultiTableManager multiTableManager = + new MultiTableManager(catalogTables.toArray(new CatalogTable[0])); + return new BatchSourceReader( + seaTunnelSource, applicationId, parallelism, envOptions, multiTableManager); } @Override @@ -91,6 +108,17 @@ public MicroBatchReader createMicroBatchReader( String hdfsUser = options.get(Constants.HDFS_USER).orElse(""); Integer checkpointId = options.getInt(Constants.CHECKPOINT_ID, 1); Map envOptions = options.asMap(); + List catalogTables; + try { + catalogTables = seaTunnelSource.getProducedCatalogTables(); + } catch (UnsupportedOperationException e) { + // TODO remove it when all connector use `getProducedCatalogTables` + SeaTunnelDataType seaTunnelDataType = seaTunnelSource.getProducedType(); + catalogTables = + CatalogTableUtil.convertDataTypeToCatalogTables(seaTunnelDataType, "default"); + } + MultiTableManager multiTableManager = + new MultiTableManager(catalogTables.toArray(new CatalogTable[0])); return new MicroBatchSourceReader( seaTunnelSource, parallelism, @@ -100,7 +128,8 @@ public MicroBatchReader createMicroBatchReader( checkpointPath, hdfsRoot, hdfsUser, - envOptions); + envOptions, + multiTableManager); } private SeaTunnelSource getSeaTunnelSource(DataSourceOptions options) { diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/BatchPartition.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/BatchPartition.java index 965706024c2..090a8a1cf78 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/BatchPartition.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/BatchPartition.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SupportCoordinate; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.source.reader.SeaTunnelInputPartitionReader; import org.apache.seatunnel.translation.spark.source.reader.batch.CoordinatedBatchPartitionReader; import org.apache.seatunnel.translation.spark.source.reader.batch.ParallelBatchPartitionReader; @@ -37,17 +38,21 @@ public class BatchPartition implements InputPartition { protected final Integer subtaskId; private Map envOptions; + private final MultiTableManager multiTableManager; + public BatchPartition( SeaTunnelSource source, Integer parallelism, String jobId, Integer subtaskId, - Map envOptions) { + Map envOptions, + MultiTableManager multiTableManager) { this.source = source; this.parallelism = parallelism; this.jobId = jobId; this.subtaskId = subtaskId; this.envOptions = envOptions; + this.multiTableManager = multiTableManager; } @Override @@ -56,11 +61,11 @@ public InputPartitionReader createPartitionReader() { if (source instanceof SupportCoordinate) { partitionReader = new CoordinatedBatchPartitionReader( - source, parallelism, jobId, subtaskId, envOptions); + source, parallelism, jobId, subtaskId, envOptions, multiTableManager); } else { partitionReader = new ParallelBatchPartitionReader( - source, parallelism, jobId, subtaskId, envOptions); + source, parallelism, jobId, subtaskId, envOptions, multiTableManager); } return new SeaTunnelInputPartitionReader(partitionReader); } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/MicroBatchPartition.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/MicroBatchPartition.java index 8f281603575..264d17eeeb7 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/MicroBatchPartition.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/MicroBatchPartition.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SupportCoordinate; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.source.reader.SeaTunnelInputPartitionReader; import org.apache.seatunnel.translation.spark.source.reader.batch.ParallelBatchPartitionReader; import org.apache.seatunnel.translation.spark.source.reader.micro.CoordinatedMicroBatchPartitionReader; @@ -43,6 +44,8 @@ public class MicroBatchPartition implements InputPartition { protected final String hdfsUser; private Map envOptions; + protected final MultiTableManager multiTableManager; + public MicroBatchPartition( SeaTunnelSource source, Integer parallelism, @@ -53,7 +56,8 @@ public MicroBatchPartition( String checkpointPath, String hdfsRoot, String hdfsUser, - Map envOptions) { + Map envOptions, + MultiTableManager multiTableManager) { this.source = source; this.parallelism = parallelism; this.jobId = jobId; @@ -64,6 +68,7 @@ public MicroBatchPartition( this.hdfsRoot = hdfsRoot; this.hdfsUser = hdfsUser; this.envOptions = envOptions; + this.multiTableManager = multiTableManager; } @Override @@ -81,7 +86,8 @@ public InputPartitionReader createPartitionReader() { checkpointPath, hdfsRoot, hdfsUser, - envOptions); + envOptions, + multiTableManager); } else { partitionReader = new ParallelMicroBatchPartitionReader( @@ -94,7 +100,8 @@ public InputPartitionReader createPartitionReader() { checkpointPath, hdfsRoot, hdfsUser, - envOptions); + envOptions, + multiTableManager); } return new SeaTunnelInputPartitionReader(partitionReader); } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/batch/BatchSourceReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/batch/BatchSourceReader.java index 4f5528622d4..4bef013a60f 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/batch/BatchSourceReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/batch/BatchSourceReader.java @@ -20,8 +20,8 @@ import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SupportCoordinate; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.source.partition.batch.BatchPartition; -import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.reader.DataSourceReader; @@ -38,21 +38,24 @@ public class BatchSourceReader implements DataSourceReader { protected final String jobId; protected final Integer parallelism; private Map envOptions; + private final MultiTableManager multiTableManager; public BatchSourceReader( SeaTunnelSource source, String jobId, Integer parallelism, - Map envOptions) { + Map envOptions, + MultiTableManager multiTableManager) { this.source = source; this.jobId = jobId; this.parallelism = parallelism; this.envOptions = envOptions; + this.multiTableManager = multiTableManager; } @Override public StructType readSchema() { - return (StructType) TypeConverterUtils.convert(source.getProducedType()); + return multiTableManager.getTableSchema(); } @Override @@ -60,12 +63,20 @@ public List> planInputPartitions() { List> virtualPartitions; if (source instanceof SupportCoordinate) { virtualPartitions = new ArrayList<>(1); - virtualPartitions.add(new BatchPartition(source, parallelism, jobId, 0, envOptions)); + virtualPartitions.add( + new BatchPartition( + source, parallelism, jobId, 0, envOptions, multiTableManager)); } else { virtualPartitions = new ArrayList<>(parallelism); for (int subtaskId = 0; subtaskId < parallelism; subtaskId++) { virtualPartitions.add( - new BatchPartition(source, parallelism, jobId, subtaskId, envOptions)); + new BatchPartition( + source, + parallelism, + jobId, + subtaskId, + envOptions, + multiTableManager)); } } return virtualPartitions; diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/batch/CoordinatedBatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/batch/CoordinatedBatchPartitionReader.java index bf7dd221425..47d55767ff4 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/batch/CoordinatedBatchPartitionReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/batch/CoordinatedBatchPartitionReader.java @@ -24,6 +24,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.translation.source.BaseSourceFunction; import org.apache.seatunnel.translation.source.CoordinatedSource; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.serialization.InternalRowCollector; import java.io.Serializable; @@ -41,14 +42,14 @@ public CoordinatedBatchPartitionReader( Integer parallelism, String jobId, Integer subtaskId, - Map envOptions) { - super(source, parallelism, jobId, subtaskId, envOptions); + Map envOptions, + MultiTableManager multiTableManager) { + super(source, parallelism, jobId, subtaskId, envOptions, multiTableManager); this.collectorMap = new HashMap<>(parallelism); for (int i = 0; i < parallelism; i++) { collectorMap.put( i, - new InternalRowCollector( - handover, new Object(), source.getProducedType(), envOptions)); + multiTableManager.getInternalRowCollector(handover, new Object(), envOptions)); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/batch/ParallelBatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/batch/ParallelBatchPartitionReader.java index 71ed2f7c224..20240638702 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/batch/ParallelBatchPartitionReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/batch/ParallelBatchPartitionReader.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.common.Handover; import org.apache.seatunnel.translation.source.BaseSourceFunction; import org.apache.seatunnel.translation.source.ParallelSource; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.serialization.InternalRowCollector; import org.apache.seatunnel.translation.util.ThreadPoolExecutorFactory; @@ -58,12 +59,15 @@ public class ParallelBatchPartitionReader { protected volatile InternalRowCollector internalRowCollector; private Map envOptions; + protected final MultiTableManager multiTableManager; + public ParallelBatchPartitionReader( SeaTunnelSource source, Integer parallelism, String jobId, Integer subtaskId, - Map envOptions) { + Map envOptions, + MultiTableManager multiTableManager) { this.source = source; this.parallelism = parallelism; this.jobId = jobId; @@ -73,6 +77,7 @@ public ParallelBatchPartitionReader( 1, getEnumeratorThreadName()); this.handover = new Handover<>(); this.envOptions = envOptions; + this.multiTableManager = multiTableManager; } protected String getEnumeratorThreadName() { @@ -105,8 +110,8 @@ protected void prepare() { } this.internalRowCollector = - new InternalRowCollector( - handover, checkpointLock, source.getProducedType(), envOptions); + multiTableManager.getInternalRowCollector(handover, checkpointLock, envOptions); + executorService.execute( () -> { try { diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/CoordinatedMicroBatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/CoordinatedMicroBatchPartitionReader.java index 11e57cc4374..9023ba3651f 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/CoordinatedMicroBatchPartitionReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/CoordinatedMicroBatchPartitionReader.java @@ -24,6 +24,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.translation.source.BaseSourceFunction; import org.apache.seatunnel.translation.source.CoordinatedSource; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.serialization.InternalRowCollector; import org.apache.seatunnel.translation.spark.source.state.ReaderState; @@ -47,7 +48,8 @@ public CoordinatedMicroBatchPartitionReader( String checkpointPath, String hdfsRoot, String hdfsUser, - Map envOptions) { + Map envOptions, + MultiTableManager multiTableManager) { super( source, parallelism, @@ -58,13 +60,13 @@ public CoordinatedMicroBatchPartitionReader( checkpointPath, hdfsRoot, hdfsUser, - envOptions); + envOptions, + multiTableManager); this.collectorMap = new HashMap<>(parallelism); for (int i = 0; i < parallelism; i++) { collectorMap.put( i, - new InternalRowCollector( - handover, new Object(), source.getProducedType(), envOptions)); + multiTableManager.getInternalRowCollector(handover, new Object(), envOptions)); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/MicroBatchSourceReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/MicroBatchSourceReader.java index d49e5bd6a62..f28c8c1b794 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/MicroBatchSourceReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/MicroBatchSourceReader.java @@ -21,9 +21,9 @@ import org.apache.seatunnel.api.source.SupportCoordinate; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.utils.SerializationUtils; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.source.partition.micro.MicroBatchPartition; import org.apache.seatunnel.translation.spark.source.state.MicroBatchState; -import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.reader.InputPartition; @@ -50,6 +50,7 @@ public class MicroBatchSourceReader implements MicroBatchReader { protected MicroBatchState startOffset; protected MicroBatchState endOffset; private Map envOptions; + private final MultiTableManager multiTableManager; public MicroBatchSourceReader( SeaTunnelSource source, @@ -60,7 +61,8 @@ public MicroBatchSourceReader( String checkpointPath, String hdfsRoot, String hdfsUser, - Map envOptions) { + Map envOptions, + MultiTableManager multiTableManager) { this.source = source; this.parallelism = parallelism; this.jobId = jobId; @@ -70,6 +72,7 @@ public MicroBatchSourceReader( this.hdfsRoot = hdfsRoot; this.hdfsUser = hdfsUser; this.envOptions = envOptions; + this.multiTableManager = multiTableManager; } @Override @@ -108,7 +111,7 @@ public void stop() { @Override public StructType readSchema() { - return (StructType) TypeConverterUtils.convert(source.getProducedType()); + return multiTableManager.getTableSchema(); } @Override @@ -127,7 +130,8 @@ public List> planInputPartitions() { checkpointPath, hdfsRoot, hdfsUser, - envOptions)); + envOptions, + multiTableManager)); } else { virtualPartitions = new ArrayList<>(parallelism); for (int subtaskId = 0; subtaskId < parallelism; subtaskId++) { @@ -142,7 +146,8 @@ public List> planInputPartitions() { checkpointPath, hdfsRoot, hdfsUser, - envOptions)); + envOptions, + multiTableManager)); } } checkpointId++; diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/ParallelMicroBatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/ParallelMicroBatchPartitionReader.java index d3457246f7c..b820a2caf70 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/ParallelMicroBatchPartitionReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/ParallelMicroBatchPartitionReader.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.utils.SerializationUtils; import org.apache.seatunnel.translation.source.BaseSourceFunction; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.source.reader.batch.ParallelBatchPartitionReader; import org.apache.seatunnel.translation.spark.source.state.ReaderState; import org.apache.seatunnel.translation.util.ThreadPoolExecutorFactory; @@ -65,8 +66,9 @@ public ParallelMicroBatchPartitionReader( String checkpointPath, String hdfsRoot, String hdfsUser, - Map envOptions) { - super(source, parallelism, jobId, subtaskId, envOptions); + Map envOptions, + MultiTableManager multiTableManager) { + super(source, parallelism, jobId, subtaskId, envOptions, multiTableManager); this.checkpointId = checkpointId; this.checkpointInterval = checkpointInterval; this.checkpointPath = checkpointPath; diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SeaTunnelBatchWrite.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SeaTunnelBatchWrite.java index a650854e2ac..f1c343ba56f 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SeaTunnelBatchWrite.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SeaTunnelBatchWrite.java @@ -49,17 +49,17 @@ public class SeaTunnelBatchWrite private MultiTableResourceManager resourceManager; - private final CatalogTable catalogTable; + private final CatalogTable[] catalogTables; private final String jobId; public SeaTunnelBatchWrite( SeaTunnelSink sink, - CatalogTable catalogTable, + CatalogTable[] catalogTables, String jobId) throws IOException { this.sink = sink; - this.catalogTable = catalogTable; + this.catalogTables = catalogTables; this.jobId = jobId; this.aggregatedCommitter = sink.createAggregatedCommitter().orElse(null); if (aggregatedCommitter != null) { @@ -78,7 +78,7 @@ public SeaTunnelBatchWrite( @Override public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { - return new SeaTunnelSparkDataWriterFactory<>(sink, catalogTable, jobId); + return new SeaTunnelSparkDataWriterFactory<>(sink, catalogTables, jobId); } @Override diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SeaTunnelSinkTable.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SeaTunnelSinkTable.java index 1fb8109eb5d..f3652be673f 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SeaTunnelSinkTable.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SeaTunnelSinkTable.java @@ -22,8 +22,8 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.Constants; import org.apache.seatunnel.common.utils.SerializationUtils; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.sink.write.SeaTunnelWriteBuilder; -import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; import org.apache.commons.lang3.StringUtils; import org.apache.spark.sql.connector.catalog.SupportsWrite; @@ -46,7 +46,7 @@ public class SeaTunnelSinkTable implements Table, SupportsWrite { private final SeaTunnelSink sink; - private final CatalogTable catalogTable; + private final CatalogTable[] catalogTables; private final String jobId; public SeaTunnelSinkTable(Map properties) { @@ -62,13 +62,13 @@ public SeaTunnelSinkTable(Map properties) { throw new IllegalArgumentException( SparkSinkInjector.SINK_CATALOG_TABLE + " must be specified"); } - this.catalogTable = SerializationUtils.stringToObject(sinkCatalogTableSerialization); + this.catalogTables = SerializationUtils.stringToObject(sinkCatalogTableSerialization); this.jobId = properties.getOrDefault(SparkSinkInjector.JOB_ID, null); } @Override public WriteBuilder newWriteBuilder(LogicalWriteInfo info) { - return new SeaTunnelWriteBuilder<>(sink, catalogTable, jobId); + return new SeaTunnelWriteBuilder<>(sink, catalogTables, jobId); } @Override @@ -78,7 +78,7 @@ public String name() { @Override public StructType schema() { - return (StructType) TypeConverterUtils.convert(catalogTable.getSeaTunnelRowType()); + return new MultiTableManager(catalogTables).getTableSchema(); } @Override diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SparkSinkInjector.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SparkSinkInjector.java index d6d148c3fed..b975c0913b2 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SparkSinkInjector.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SparkSinkInjector.java @@ -38,23 +38,25 @@ public class SparkSinkInjector { public static DataStreamWriter inject( DataStreamWriter dataset, SeaTunnelSink sink, - CatalogTable catalogTable, + CatalogTable[] catalogTables, String applicationId) { return dataset.format(SINK_NAME) .outputMode(OutputMode.Append()) .option(Constants.SINK_SERIALIZATION, SerializationUtils.objectToString(sink)) - .option(SINK_CATALOG_TABLE, SerializationUtils.objectToString(catalogTable)) + // TODO this should require fetching the catalog table in sink + .option(SINK_CATALOG_TABLE, SerializationUtils.objectToString(catalogTables)) .option(JOB_ID, applicationId); } public static DataFrameWriter inject( DataFrameWriter dataset, SeaTunnelSink sink, - CatalogTable catalogTable, + CatalogTable[] catalogTables, String applicationId) { return dataset.format(SINK_NAME) .option(Constants.SINK_SERIALIZATION, SerializationUtils.objectToString(sink)) - .option(SINK_CATALOG_TABLE, SerializationUtils.objectToString(catalogTable)) + // TODO this should require fetching the catalog table in sink + .option(SINK_CATALOG_TABLE, SerializationUtils.objectToString(catalogTables)) .option(JOB_ID, applicationId); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java index 0ce7896bdd2..59f931e38f1 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java @@ -21,10 +21,8 @@ import org.apache.seatunnel.api.sink.SinkCommitter; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportResourceShare; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.translation.serialization.RowConverter; -import org.apache.seatunnel.translation.spark.serialization.InternalRowConverter; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.write.DataWriter; @@ -41,32 +39,33 @@ @Slf4j public class SeaTunnelSparkDataWriter implements DataWriter { - private final SinkWriter sinkWriter; + protected final SinkWriter sinkWriter; - @Nullable private final SinkCommitter sinkCommitter; - private final RowConverter rowConverter; - private CommitInfoT latestCommitInfoT; - private long epochId; - private volatile MultiTableResourceManager resourceManager; + @Nullable protected final SinkCommitter sinkCommitter; + protected CommitInfoT latestCommitInfoT; + protected long epochId; + protected volatile MultiTableResourceManager resourceManager; + + private final MultiTableManager multiTableManager; public SeaTunnelSparkDataWriter( SinkWriter sinkWriter, @Nullable SinkCommitter sinkCommitter, - SeaTunnelDataType dataType, + MultiTableManager multiTableManager, long epochId) { this.sinkWriter = sinkWriter; this.sinkCommitter = sinkCommitter; - this.rowConverter = new InternalRowConverter(dataType); + this.multiTableManager = multiTableManager; this.epochId = epochId == 0 ? 1 : epochId; initResourceManger(); } @Override public void write(InternalRow record) throws IOException { - sinkWriter.write(rowConverter.reconvert(record)); + sinkWriter.write(multiTableManager.reconvert(record)); } - private void initResourceManger() { + protected void initResourceManger() { if (sinkWriter instanceof SupportResourceShare) { resourceManager = ((SupportResourceShare) sinkWriter).initMultiTableResourceManager(1, 1); diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java index 5046c2deccc..b83787cac1e 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.write.DataWriter; @@ -35,15 +36,15 @@ public class SeaTunnelSparkDataWriterFactory implements DataWriterFactory, StreamingDataWriterFactory { private final SeaTunnelSink sink; - private final CatalogTable catalogTable; + private final CatalogTable[] catalogTables; private final String jobId; public SeaTunnelSparkDataWriterFactory( SeaTunnelSink sink, - CatalogTable catalogTable, + CatalogTable[] catalogTables, String jobId) { this.sink = sink; - this.catalogTable = catalogTable; + this.catalogTables = catalogTables; this.jobId = jobId; } @@ -63,7 +64,7 @@ public DataWriter createWriter(int partitionId, long taskId) { throw new RuntimeException("Failed to create SinkCommitter.", e); } return new SeaTunnelSparkDataWriter<>( - writer, committer, catalogTable.getSeaTunnelRowType(), 0); + writer, committer, new MultiTableManager(catalogTables), 0); } @Override diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWrite.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWrite.java index 21ae52f1281..2bd4388cc22 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWrite.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWrite.java @@ -31,22 +31,22 @@ public class SeaTunnelWrite implements Write { private final SeaTunnelSink sink; - private final CatalogTable catalogTable; + private final CatalogTable[] catalogTables; private final String jobId; public SeaTunnelWrite( SeaTunnelSink sink, - CatalogTable catalogTable, + CatalogTable[] catalogTables, String jobId) { this.sink = sink; - this.catalogTable = catalogTable; + this.catalogTables = catalogTables; this.jobId = jobId; } @Override public BatchWrite toBatch() { try { - return new SeaTunnelBatchWrite<>(sink, catalogTable, jobId); + return new SeaTunnelBatchWrite<>(sink, catalogTables, jobId); } catch (IOException e) { throw new RuntimeException("SeaTunnel Spark sink create batch failed", e); } @@ -55,7 +55,7 @@ public BatchWrite toBatch() { @Override public StreamingWrite toStreaming() { try { - return new SeaTunnelBatchWrite<>(sink, catalogTable, jobId); + return new SeaTunnelBatchWrite<>(sink, catalogTables, jobId); } catch (IOException e) { throw new RuntimeException("SeaTunnel Spark sink create batch failed", e); } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWriteBuilder.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWriteBuilder.java index d1ff3cfca1a..896982c3516 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWriteBuilder.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWriteBuilder.java @@ -28,20 +28,20 @@ public class SeaTunnelWriteBuilder implements WriteBuilder { private final SeaTunnelSink sink; - private final CatalogTable catalogTable; + private final CatalogTable[] catalogTables; private final String jobId; public SeaTunnelWriteBuilder( SeaTunnelSink sink, - CatalogTable catalogTable, + CatalogTable[] catalogTables, String jobId) { this.sink = sink; - this.catalogTable = catalogTable; + this.catalogTables = catalogTables; this.jobId = jobId; } @Override public Write build() { - return new SeaTunnelWrite<>(sink, catalogTable, jobId); + return new SeaTunnelWrite<>(sink, catalogTables, jobId); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/SeaTunnelSourceTable.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/SeaTunnelSourceTable.java index fb5eb6d6a9e..ef707095e63 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/SeaTunnelSourceTable.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/SeaTunnelSourceTable.java @@ -19,11 +19,14 @@ import org.apache.seatunnel.api.common.CommonOptions; import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.Constants; import org.apache.seatunnel.common.utils.SerializationUtils; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.source.scan.SeaTunnelScanBuilder; -import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; import org.apache.commons.lang3.StringUtils; import org.apache.spark.sql.SparkSession; @@ -37,6 +40,7 @@ import com.google.common.collect.Sets; +import java.util.List; import java.util.Map; import java.util.Set; @@ -48,6 +52,8 @@ public class SeaTunnelSourceTable implements Table, SupportsRead { private final SeaTunnelSource source; + private final MultiTableManager multiTableManager; + public SeaTunnelSourceTable(Map properties) { this.properties = properties; String sourceSerialization = properties.getOrDefault(Constants.SOURCE_SERIALIZATION, ""); @@ -55,6 +61,16 @@ public SeaTunnelSourceTable(Map properties) { throw new IllegalArgumentException("source.serialization must be specified"); } this.source = SerializationUtils.stringToObject(sourceSerialization); + List catalogTables; + try { + catalogTables = source.getProducedCatalogTables(); + } catch (UnsupportedOperationException e) { + // TODO remove it when all connector use `getProducedCatalogTables` + SeaTunnelDataType seaTunnelDataType = source.getProducedType(); + catalogTables = + CatalogTableUtil.convertDataTypeToCatalogTables(seaTunnelDataType, "default"); + } + multiTableManager = new MultiTableManager(catalogTables.toArray(new CatalogTable[0])); } /** @@ -69,7 +85,7 @@ public ScanBuilder newScanBuilder(CaseInsensitiveStringMap caseInsensitiveString Integer.parseInt(properties.getOrDefault(CommonOptions.PARALLELISM.key(), "1")); String applicationId = SparkSession.getActiveSession().get().sparkContext().applicationId(); return new SeaTunnelScanBuilder( - source, parallelism, applicationId, caseInsensitiveStringMap); + source, parallelism, applicationId, caseInsensitiveStringMap, multiTableManager); } /** A name to identify this table */ @@ -81,7 +97,7 @@ public String name() { /** Returns the schema of this table */ @Override public StructType schema() { - return (StructType) TypeConverterUtils.convert(source.getProducedType()); + return multiTableManager.getTableSchema(); } /** Returns the set of capabilities for this table */ diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/CoordinatedBatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/CoordinatedBatchPartitionReader.java index 746cf348386..69dc64c234b 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/CoordinatedBatchPartitionReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/CoordinatedBatchPartitionReader.java @@ -24,6 +24,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.translation.source.BaseSourceFunction; import org.apache.seatunnel.translation.source.CoordinatedSource; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.serialization.InternalRowCollector; import java.io.Serializable; @@ -41,14 +42,14 @@ public CoordinatedBatchPartitionReader( Integer parallelism, String jobId, Integer subtaskId, - Map envOptions) { - super(source, parallelism, jobId, subtaskId, envOptions); + Map envOptions, + MultiTableManager multiTableManager) { + super(source, parallelism, jobId, subtaskId, envOptions, multiTableManager); this.collectorMap = new HashMap<>(parallelism); for (int i = 0; i < parallelism; i++) { collectorMap.put( i, - new InternalRowCollector( - handover, new Object(), source.getProducedType(), envOptions)); + multiTableManager.getInternalRowCollector(handover, new Object(), envOptions)); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/ParallelBatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/ParallelBatchPartitionReader.java index 8bbbcc71c17..27ab9f42d2c 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/ParallelBatchPartitionReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/ParallelBatchPartitionReader.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.common.Handover; import org.apache.seatunnel.translation.source.BaseSourceFunction; import org.apache.seatunnel.translation.source.ParallelSource; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.serialization.InternalRowCollector; import org.apache.seatunnel.translation.util.ThreadPoolExecutorFactory; @@ -58,12 +59,15 @@ public class ParallelBatchPartitionReader { protected volatile InternalRowCollector internalRowCollector; private final Map envOptions; + private final MultiTableManager multiTableManager; + public ParallelBatchPartitionReader( SeaTunnelSource source, Integer parallelism, String jobId, Integer subtaskId, - Map envOptions) { + Map envOptions, + MultiTableManager multiTableManager) { this.source = source; this.parallelism = parallelism; this.jobId = jobId; @@ -73,6 +77,7 @@ public ParallelBatchPartitionReader( 1, getEnumeratorThreadName()); this.handover = new Handover<>(); this.envOptions = envOptions; + this.multiTableManager = multiTableManager; } protected String getEnumeratorThreadName() { @@ -105,8 +110,7 @@ protected void prepare() { } this.internalRowCollector = - new InternalRowCollector( - handover, checkpointLock, source.getProducedType(), envOptions); + multiTableManager.getInternalRowCollector(handover, checkpointLock, envOptions); executorService.execute( () -> { try { diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/SeaTunnelBatch.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/SeaTunnelBatch.java index 491555f32b2..30c46440783 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/SeaTunnelBatch.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/SeaTunnelBatch.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SupportCoordinate; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.spark.sql.connector.read.Batch; import org.apache.spark.sql.connector.read.InputPartition; @@ -36,15 +37,19 @@ public class SeaTunnelBatch implements Batch { private final String jobId; private final Map envOptions; + private final MultiTableManager multiTableManager; + public SeaTunnelBatch( SeaTunnelSource source, int parallelism, String jobId, - Map envOptions) { + Map envOptions, + MultiTableManager multiTableManager) { this.source = source; this.parallelism = parallelism; this.jobId = jobId; this.envOptions = envOptions; + this.multiTableManager = multiTableManager; } @Override @@ -64,6 +69,7 @@ public InputPartition[] planInputPartitions() { @Override public PartitionReaderFactory createReaderFactory() { - return new SeaTunnelBatchPartitionReaderFactory(source, parallelism, jobId, envOptions); + return new SeaTunnelBatchPartitionReaderFactory( + source, parallelism, jobId, envOptions, multiTableManager); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/SeaTunnelBatchPartitionReaderFactory.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/SeaTunnelBatchPartitionReaderFactory.java index f15ae051ec2..5781262ef56 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/SeaTunnelBatchPartitionReaderFactory.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/SeaTunnelBatchPartitionReaderFactory.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SupportCoordinate; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.read.InputPartition; @@ -36,15 +37,19 @@ public class SeaTunnelBatchPartitionReaderFactory implements PartitionReaderFact private final String jobId; private final Map envOptions; + private final MultiTableManager multiTableManager; + public SeaTunnelBatchPartitionReaderFactory( SeaTunnelSource source, int parallelism, String jobId, - Map envOptions) { + Map envOptions, + MultiTableManager multiTableManager) { this.source = source; this.parallelism = parallelism; this.jobId = jobId; this.envOptions = envOptions; + this.multiTableManager = multiTableManager; } @Override @@ -55,11 +60,11 @@ public PartitionReader createReader(InputPartition partition) { if (source instanceof SupportCoordinate) { partitionReader = new CoordinatedBatchPartitionReader( - source, parallelism, jobId, partitionId, envOptions); + source, parallelism, jobId, partitionId, envOptions, multiTableManager); } else { partitionReader = new ParallelBatchPartitionReader( - source, parallelism, jobId, partitionId, envOptions); + source, parallelism, jobId, partitionId, envOptions, multiTableManager); } return new SeaTunnelBatchPartitionReader(partitionReader); } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/CoordinatedMicroBatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/CoordinatedMicroBatchPartitionReader.java index 2df1b089264..ccf71692dcc 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/CoordinatedMicroBatchPartitionReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/CoordinatedMicroBatchPartitionReader.java @@ -24,6 +24,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.translation.source.BaseSourceFunction; import org.apache.seatunnel.translation.source.CoordinatedSource; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.serialization.InternalRowCollector; import java.io.Serializable; @@ -46,7 +47,8 @@ public CoordinatedMicroBatchPartitionReader( String checkpointPath, String hdfsRoot, String hdfsUser, - Map envOptions) { + Map envOptions, + MultiTableManager multiTableManager) { super( source, parallelism, @@ -57,13 +59,13 @@ public CoordinatedMicroBatchPartitionReader( checkpointPath, hdfsRoot, hdfsUser, - envOptions); + envOptions, + multiTableManager); this.collectorMap = new HashMap<>(parallelism); for (int i = 0; i < parallelism; i++) { collectorMap.put( i, - new InternalRowCollector( - handover, new Object(), source.getProducedType(), envOptions)); + multiTableManager.getInternalRowCollector(handover, new Object(), envOptions)); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/ParallelMicroBatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/ParallelMicroBatchPartitionReader.java index c62185d0995..bfa7aa6079d 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/ParallelMicroBatchPartitionReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/ParallelMicroBatchPartitionReader.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.utils.SerializationUtils; import org.apache.seatunnel.translation.source.BaseSourceFunction; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.source.partition.batch.ParallelBatchPartitionReader; import org.apache.seatunnel.translation.util.ThreadPoolExecutorFactory; @@ -64,8 +65,9 @@ public ParallelMicroBatchPartitionReader( String checkpointPath, String hdfsRoot, String hdfsUser, - Map envOptions) { - super(source, parallelism, jobId, subtaskId, envOptions); + Map envOptions, + MultiTableManager multiTableManager) { + super(source, parallelism, jobId, subtaskId, envOptions, multiTableManager); this.checkpointId = checkpointId; this.checkpointInterval = checkpointInterval; this.checkpointPath = checkpointPath; diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatch.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatch.java index 0d9f6d4cdbf..11eec4d7fc5 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatch.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatch.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.Constants; import org.apache.seatunnel.common.utils.JsonUtils; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -56,17 +57,21 @@ public class SeaTunnelMicroBatch implements MicroBatchStream { private Offset currentOffset = initialOffset; + private final MultiTableManager multiTableManager; + public SeaTunnelMicroBatch( SeaTunnelSource source, int parallelism, String jobId, String checkpointLocation, - CaseInsensitiveStringMap caseInsensitiveStringMap) { + CaseInsensitiveStringMap caseInsensitiveStringMap, + MultiTableManager multiTableManager) { this.source = source; this.parallelism = parallelism; this.jobId = jobId; this.checkpointLocation = checkpointLocation; this.caseInsensitiveStringMap = caseInsensitiveStringMap; + this.multiTableManager = multiTableManager; } @Override @@ -119,7 +124,12 @@ public InputPartition[] planInputPartitions(Offset start, Offset end) { @Override public PartitionReaderFactory createReaderFactory() { return new SeaTunnelMicroBatchPartitionReaderFactory( - source, parallelism, jobId, checkpointLocation, caseInsensitiveStringMap); + source, + parallelism, + jobId, + checkpointLocation, + caseInsensitiveStringMap, + multiTableManager); } @Override diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatchPartitionReaderFactory.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatchPartitionReaderFactory.java index 4742916894d..a92a7dfb3bf 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatchPartitionReaderFactory.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatchPartitionReaderFactory.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SupportCoordinate; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.source.partition.batch.ParallelBatchPartitionReader; import org.apache.spark.sql.catalyst.InternalRow; @@ -41,17 +42,21 @@ public class SeaTunnelMicroBatchPartitionReaderFactory implements PartitionReade private final CaseInsensitiveStringMap caseInsensitiveStringMap; + private final MultiTableManager multiTableManager; + public SeaTunnelMicroBatchPartitionReaderFactory( SeaTunnelSource source, int parallelism, String jobId, String checkpointLocation, - CaseInsensitiveStringMap caseInsensitiveStringMap) { + CaseInsensitiveStringMap caseInsensitiveStringMap, + MultiTableManager multiTableManager) { this.source = source; this.parallelism = parallelism; this.jobId = jobId; this.checkpointLocation = checkpointLocation; this.caseInsensitiveStringMap = caseInsensitiveStringMap; + this.multiTableManager = multiTableManager; } @Override @@ -77,7 +82,8 @@ public PartitionReader createReader(InputPartition partition) { checkpointLocation, hdfsRoot, hdfsUser, - envOptions); + envOptions, + multiTableManager); } else { partitionReader = new ParallelMicroBatchPartitionReader( @@ -90,7 +96,8 @@ public PartitionReader createReader(InputPartition partition) { checkpointLocation, hdfsRoot, hdfsUser, - envOptions); + envOptions, + multiTableManager); } return new SeaTunnelMicroBatchPartitionReader(partitionReader); } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScan.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScan.java index aa196994087..c2085d91270 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScan.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScan.java @@ -19,9 +19,9 @@ import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.source.partition.batch.SeaTunnelBatch; import org.apache.seatunnel.translation.spark.source.partition.micro.SeaTunnelMicroBatch; -import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; import org.apache.spark.sql.connector.read.Batch; import org.apache.spark.sql.connector.read.Scan; @@ -40,31 +40,40 @@ public class SeaTunnelScan implements Scan { private final CaseInsensitiveStringMap caseInsensitiveStringMap; + private final MultiTableManager multiTableManager; + public SeaTunnelScan( SeaTunnelSource source, int parallelism, String jobId, - CaseInsensitiveStringMap caseInsensitiveStringMap) { + CaseInsensitiveStringMap caseInsensitiveStringMap, + MultiTableManager multiTableManager) { this.source = source; this.parallelism = parallelism; this.jobId = jobId; this.caseInsensitiveStringMap = caseInsensitiveStringMap; + this.multiTableManager = multiTableManager; } @Override public StructType readSchema() { - return (StructType) TypeConverterUtils.convert(source.getProducedType()); + return multiTableManager.getTableSchema(); } @Override public Batch toBatch() { Map envOptions = caseInsensitiveStringMap.asCaseSensitiveMap(); - return new SeaTunnelBatch(source, parallelism, jobId, envOptions); + return new SeaTunnelBatch(source, parallelism, jobId, envOptions, multiTableManager); } @Override public MicroBatchStream toMicroBatchStream(String checkpointLocation) { return new SeaTunnelMicroBatch( - source, parallelism, jobId, checkpointLocation, caseInsensitiveStringMap); + source, + parallelism, + jobId, + checkpointLocation, + caseInsensitiveStringMap, + multiTableManager); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScanBuilder.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScanBuilder.java index efff7956c15..932f83dac9e 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScanBuilder.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScanBuilder.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.spark.sql.connector.read.Scan; import org.apache.spark.sql.connector.read.ScanBuilder; @@ -33,20 +34,25 @@ public class SeaTunnelScanBuilder implements ScanBuilder { private final CaseInsensitiveStringMap caseInsensitiveStringMap; + private final MultiTableManager multiTableManager; + public SeaTunnelScanBuilder( SeaTunnelSource source, int parallelism, String jobId, - CaseInsensitiveStringMap caseInsensitiveStringMap) { + CaseInsensitiveStringMap caseInsensitiveStringMap, + MultiTableManager multiTableManager) { this.source = source; this.parallelism = parallelism; this.jobId = jobId; this.caseInsensitiveStringMap = caseInsensitiveStringMap; + this.multiTableManager = multiTableManager; } /** Returns the {@link SeaTunnelScan} */ @Override public Scan build() { - return new SeaTunnelScan(source, parallelism, jobId, caseInsensitiveStringMap); + return new SeaTunnelScan( + source, parallelism, jobId, caseInsensitiveStringMap, multiTableManager); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/test/java/org/apache/seatunnel/translation/spark/sink/SparkSinkTest.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/test/java/org/apache/seatunnel/translation/spark/sink/SparkSinkTest.java index b426f175480..73084fb76b3 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/test/java/org/apache/seatunnel/translation/spark/sink/SparkSinkTest.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/test/java/org/apache/seatunnel/translation/spark/sink/SparkSinkTest.java @@ -17,11 +17,13 @@ package org.apache.seatunnel.translation.spark.sink; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.LocalTimeType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -131,6 +133,8 @@ public void testSparkSinkWriteDataWithCopy() { GenericRow row1WithRow = new GenericRow( new Object[] { + (byte) 1, + "test.test.test", 42, "string1", true, @@ -196,6 +200,8 @@ public void testSparkSinkWriteDataWithCopy() { GenericRow row2WithRow = new GenericRow( new Object[] { + (byte) 1, + "test.test.test", 12, "string2", false, @@ -261,6 +267,8 @@ public void testSparkSinkWriteDataWithCopy() { GenericRow row3WithRow = new GenericRow( new Object[] { + (byte) 1, + "test.test.test", 233, "string3", true, @@ -397,15 +405,18 @@ public void testSparkSinkWriteDataWithCopy() { BasicType.STRING_TYPE, BasicType.STRING_TYPE) }) }); - + structType.add("row", structType); + StructType parcelStructType = (StructType) TypeConverterUtils.parcel(rowType); Dataset dataset = spark.createDataFrame( - Arrays.asList(row1WithRow, row2WithRow, row3WithRow), - structType.add("row", structType)); + Arrays.asList(row1WithRow, row2WithRow, row3WithRow), parcelStructType); SparkSinkInjector.inject( dataset.write(), new SeaTunnelSinkWithBuffer(), - CatalogTableUtil.getCatalogTable("test", "test", "test", "test", rowType), + new CatalogTable[] { + CatalogTableUtil.getCatalogTable( + "test", "test", "test", "test", rowType) + }, spark.sparkContext().applicationId()) .option("checkpointLocation", "/tmp") .mode(SaveMode.Append) diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/ColumnWithIndex.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/ColumnWithIndex.java new file mode 100644 index 00000000000..43628a6dffd --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/ColumnWithIndex.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.translation.spark.execution; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; + +import java.io.Serializable; +import java.util.Arrays; + +public class ColumnWithIndex implements Serializable { + + private int[] index; + private CatalogTable catalogTable; + + public ColumnWithIndex(int[] index, CatalogTable catalogTable) { + this.index = index; + this.catalogTable = catalogTable; + } + + public int[] getIndex() { + return index; + } + + public CatalogTable getCatalogTable() { + return catalogTable; + } + + @Override + public String toString() { + return "ColumnWithIndex{" + + "table=" + + catalogTable.getTablePath() + + ", index=" + + Arrays.toString(index) + + ", schema=" + + catalogTable.getSeaTunnelRowType() + + '}'; + } +} diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/DatasetTableInfo.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/DatasetTableInfo.java similarity index 74% rename from seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/DatasetTableInfo.java rename to seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/DatasetTableInfo.java index 3af8a5ad6de..cd2fa938d55 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/DatasetTableInfo.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/DatasetTableInfo.java @@ -15,23 +15,28 @@ * limitations under the License. */ -package org.apache.seatunnel.core.starter.spark.execution; +package org.apache.seatunnel.translation.spark.execution; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import lombok.AllArgsConstructor; import lombok.Data; +import java.util.List; + @Data -@AllArgsConstructor public class DatasetTableInfo { private Dataset dataset; - - private CatalogTable catalogTable; - + private List catalogTables; private String tableName; + + public DatasetTableInfo( + Dataset dataset, List catalogTables, String tableName) { + this.dataset = dataset; + this.catalogTables = catalogTables; + this.tableName = tableName; + } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/IndexQueue.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/IndexQueue.java new file mode 100644 index 00000000000..521b0854df7 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/IndexQueue.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.translation.spark.execution; + +import java.util.List; +import java.util.ListIterator; + +public class IndexQueue { + private List list; + private ListIterator listIterator; + + public IndexQueue(List list) { + this.list = list; + this.listIterator = list.listIterator(); + } + + public boolean hasNext() { + return listIterator.hasNext(); + } + + public T next() { + return listIterator.next(); + } + + public void add(T t) { + listIterator.add(t); + } + + public void append(T t) { + list.add(t); + } + + public void set(T t) { + listIterator.set(t); + } + + public void reset() { + listIterator = list.listIterator(); + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/MultiTableManager.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/MultiTableManager.java new file mode 100644 index 00000000000..ab470ce3dd4 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/MultiTableManager.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.translation.spark.execution; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.Handover; +import org.apache.seatunnel.translation.spark.serialization.InternalMultiRowCollector; +import org.apache.seatunnel.translation.spark.serialization.InternalRowCollector; +import org.apache.seatunnel.translation.spark.serialization.InternalRowConverter; +import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructType; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +@Slf4j +public class MultiTableManager implements Serializable { + + private Map rowSerializationMap; + + private InternalRowConverter rowSerialization; + private CatalogTable mergeCatalogTable; + private boolean isMultiTable = false; + + public MultiTableManager(CatalogTable[] catalogTables) { + List columnWithIndexes = mergeSchema(catalogTables); + if (catalogTables.length > 1) { + isMultiTable = true; + rowSerializationMap = + columnWithIndexes.stream() + .collect( + Collectors.toMap( + columnWithIndex -> + columnWithIndex + .getCatalogTable() + .getTablePath() + .toString(), + columnWithIndex -> + new InternalRowConverter( + mergeCatalogTable.getSeaTunnelRowType(), + columnWithIndex.getIndex()))); + } else { + rowSerialization = new InternalRowConverter(catalogTables[0].getSeaTunnelRowType()); + } + log.info("Multi-table enabled:{}", isMultiTable); + log.info( + "merged table {}, schema {}", + mergeCatalogTable.getTablePath(), + mergeCatalogTable.getSeaTunnelRowType()); + for (ColumnWithIndex columnWithIndex : columnWithIndexes) { + log.info("MultiTableManager columnWithIndex:{}", columnWithIndex); + } + } + + public SeaTunnelRow reconvert(InternalRow record) throws IOException { + if (isMultiTable) { + String tableId = record.getString(1); + return rowSerializationMap.get(tableId).reconvert(record); + } + return rowSerialization.reconvert(record); + } + + public StructType getTableSchema() { + return (StructType) TypeConverterUtils.parcel(mergeCatalogTable.getSeaTunnelRowType()); + } + + public List mergeSchema(CatalogTable[] catalogTables) { + List columnWithIndexes = new ArrayList<>(); + if (catalogTables.length == 1) { + CatalogTable catalogTable = catalogTables[0]; + columnWithIndexes.add( + new ColumnWithIndex( + IntStream.rangeClosed( + 0, catalogTable.getSeaTunnelRowType().getTotalFields()) + .toArray(), + catalogTable)); + mergeCatalogTable = catalogTable; + return columnWithIndexes; + } + List fieldNames = new ArrayList<>(); + List> fieldTypes = new ArrayList<>(); + int indexSize = -1; + HashMap, IndexQueue> map = new HashMap<>(); + for (int i = 0; i < catalogTables.length; i++) { + CatalogTable catalogTable = catalogTables[i]; + SeaTunnelRowType seaTunnelRowType = catalogTable.getSeaTunnelRowType(); + SeaTunnelDataType[] seaTunnelDataTypes = seaTunnelRowType.getFieldTypes(); + int[] indexes = new int[seaTunnelDataTypes.length]; + for (int j = 0; j < seaTunnelDataTypes.length; j++) { + IndexQueue indexQueue = + map.computeIfAbsent( + seaTunnelDataTypes[j], k -> new IndexQueue<>(new ArrayList<>())); + if (indexQueue.hasNext()) { + indexes[j] = indexQueue.next(); + } else { + indexSize++; + indexes[j] = indexSize; + indexQueue.add(indexSize); + fieldNames.add(editColumnName(indexSize)); + fieldTypes.add(seaTunnelDataTypes[j]); + } + } + map.forEach((k, v) -> v.reset()); + columnWithIndexes.add(new ColumnWithIndex(indexes, catalogTable)); + } + SeaTunnelRowType rowType = + new SeaTunnelRowType( + fieldNames.toArray(new String[0]), + fieldTypes.toArray(new SeaTunnelDataType[0])); + mergeCatalogTable = + CatalogTableUtil.getCatalogTable( + "spark", "default", "default", "merge_table", rowType); + return columnWithIndexes; + } + + public static String editColumnName(int index) { + return "column" + index; + } + + public InternalRowCollector getInternalRowCollector( + Handover handover, + Object checkpointLock, + Map envOptionsInfo) { + if (isMultiTable) { + return new InternalMultiRowCollector( + handover, checkpointLock, rowSerializationMap, envOptionsInfo); + } else { + return new InternalRowCollector( + handover, checkpointLock, rowSerialization, envOptionsInfo); + } + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalMultiRowCollector.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalMultiRowCollector.java new file mode 100644 index 00000000000..5c5573b3e36 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalMultiRowCollector.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.translation.spark.serialization; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.Handover; + +import org.apache.spark.sql.catalyst.InternalRow; + +import java.util.Map; + +public class InternalMultiRowCollector extends InternalRowCollector { + private final Map rowSerializationMap; + + public InternalMultiRowCollector( + Handover handover, + Object checkpointLock, + Map rowSerializationMap, + Map envOptionsInfo) { + super(handover, checkpointLock, null, envOptionsInfo); + this.rowSerializationMap = rowSerializationMap; + } + + @Override + public void collect(SeaTunnelRow record) { + try { + synchronized (checkpointLock) { + InternalRowConverter rowSerialization = + rowSerializationMap.get(record.getTableId()); + flowControlGate.audit(record); + handover.produce(rowSerialization.convert(record)); + } + collectTotalCount.incrementAndGet(); + emptyThisPollNext = false; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public Map getRowSerializationMap() { + return rowSerializationMap; + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalRowCollector.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalRowCollector.java index 0bbcd3a6fe0..d13b31dc853 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalRowCollector.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalRowCollector.java @@ -18,7 +18,6 @@ package org.apache.seatunnel.translation.spark.serialization; import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.Handover; import org.apache.seatunnel.core.starter.flowcontrol.FlowControlGate; @@ -30,22 +29,22 @@ import java.util.concurrent.atomic.AtomicLong; public class InternalRowCollector implements Collector { - private final Handover handover; - private final Object checkpointLock; + protected final Handover handover; + protected final Object checkpointLock; private final InternalRowConverter rowSerialization; - private final AtomicLong collectTotalCount; + protected final AtomicLong collectTotalCount; private Map envOptions; - private FlowControlGate flowControlGate; - private volatile boolean emptyThisPollNext; + protected FlowControlGate flowControlGate; + protected volatile boolean emptyThisPollNext; public InternalRowCollector( Handover handover, Object checkpointLock, - SeaTunnelDataType dataType, + InternalRowConverter rowSerialization, Map envOptionsInfo) { this.handover = handover; this.checkpointLock = checkpointLock; - this.rowSerialization = new InternalRowConverter(dataType); + this.rowSerialization = rowSerialization; this.collectTotalCount = new AtomicLong(0); this.envOptions = (Map) envOptionsInfo; this.flowControlGate = FlowControlGate.create(FlowControlStrategy.fromMap(envOptions)); @@ -83,4 +82,8 @@ public boolean isEmptyThisPollNext() { public void resetEmptyThisPollNext() { this.emptyThisPollNext = true; } + + public InternalRowConverter getRowSerialization() { + return rowSerialization; + } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalRowConverter.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalRowConverter.java index 79269ecf8e3..8dbca1975a7 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalRowConverter.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalRowConverter.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.MapType; +import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -63,17 +64,24 @@ import java.util.HashMap; import java.util.LinkedHashMap; import java.util.Map; +import java.util.stream.IntStream; public final class InternalRowConverter extends RowConverter { + private final int[] indexes; public InternalRowConverter(SeaTunnelDataType dataType) { super(dataType); + indexes = IntStream.range(0, ((SeaTunnelRowType) dataType).getTotalFields()).toArray(); + } + + public InternalRowConverter(SeaTunnelDataType dataType, int[] indexes) { + super(dataType); + this.indexes = indexes; } @Override public InternalRow convert(SeaTunnelRow seaTunnelRow) throws IOException { - validate(seaTunnelRow); - return (InternalRow) convert(seaTunnelRow, dataType); + return parcel(seaTunnelRow, (SeaTunnelRowType) dataType); } private static Object convert(Object field, SeaTunnelDataType dataType) { @@ -136,6 +144,30 @@ private static InternalRow convert(SeaTunnelRow seaTunnelRow, SeaTunnelRowType r return new SpecificInternalRow(values); } + private InternalRow parcel(SeaTunnelRow seaTunnelRow, SeaTunnelRowType rowType) { + // 0 -> row kind, 1 -> table id + int arity = rowType.getTotalFields(); + MutableValue[] values = new MutableValue[arity + 2]; + for (int i = 0; i < indexes.length; i++) { + values[indexes[i] + 2] = createMutableValue(rowType.getFieldType(indexes[i])); + Object fieldValue = convert(seaTunnelRow.getField(i), rowType.getFieldType(indexes[i])); + if (fieldValue != null) { + values[indexes[i] + 2].update(fieldValue); + } + } + values[0] = new MutableByte(); + values[0].update(seaTunnelRow.getRowKind().toByteValue()); + values[1] = new MutableAny(); + values[1].update(UTF8String.fromString(seaTunnelRow.getTableId())); + // Fill any remaining null values with MutableAny + for (int i = 0; i < values.length; i++) { + if (values[i] == null) { + values[i] = new MutableAny(); + } + } + return new SpecificInternalRow(values); + } + private static ArrayBasedMapData convertMap(Map mapData, MapType mapType) { if (mapData == null || mapData.size() == 0) { return ArrayBasedMapData.apply(new Object[] {}, new Object[] {}); @@ -213,9 +245,27 @@ private static MutableValue createMutableValue(SeaTunnelDataType dataType) { } } + public SeaTunnelRow unpack(InternalRow engineRow, SeaTunnelRowType rowType) throws IOException { + RowKind rowKind = RowKind.fromByteValue(engineRow.getByte(0)); + String tableId = engineRow.getString(1); + Object[] fields = new Object[indexes.length]; + for (int i = 0; i < indexes.length; i++) { + fields[i] = + reconvert( + engineRow.get( + indexes[i] + 2, + TypeConverterUtils.convert(rowType.getFieldType(indexes[i]))), + rowType.getFieldType(indexes[i])); + } + SeaTunnelRow seaTunnelRow = new SeaTunnelRow(fields); + seaTunnelRow.setRowKind(rowKind); + seaTunnelRow.setTableId(tableId); + return seaTunnelRow; + } + @Override public SeaTunnelRow reconvert(InternalRow engineRow) throws IOException { - return (SeaTunnelRow) reconvert(engineRow, dataType); + return unpack(engineRow, (SeaTunnelRowType) dataType); } private static Object reconvert(Object field, SeaTunnelDataType dataType) { diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/SeaTunnelRowConverter.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/SeaTunnelRowConverter.java index db23d4912a9..33d1b49fb1c 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/SeaTunnelRowConverter.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/SeaTunnelRowConverter.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.MapType; +import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -56,11 +57,25 @@ public SeaTunnelRow convert(SeaTunnelRow seaTunnelRow) throws IOException { validate(seaTunnelRow); GenericRowWithSchema rowWithSchema = (GenericRowWithSchema) convert(seaTunnelRow, dataType); SeaTunnelRow newRow = new SeaTunnelRow(rowWithSchema.values()); - newRow.setRowKind(seaTunnelRow.getRowKind()); - newRow.setTableId(seaTunnelRow.getTableId()); return newRow; } + public GenericRowWithSchema parcel(SeaTunnelRow seaTunnelRow) { + SeaTunnelRowType rowType = (SeaTunnelRowType) dataType; + int arity = rowType.getTotalFields(); + Object[] fields = new Object[arity + 2]; + fields[0] = seaTunnelRow.getRowKind().toByteValue(); + fields[1] = seaTunnelRow.getTableId(); + StructType schema = (StructType) TypeConverterUtils.parcel(rowType); + for (int i = 0; i < arity; i++) { + Object fieldValue = convert(seaTunnelRow.getField(i), rowType.getFieldType(i)); + if (fieldValue != null) { + fields[i + 2] = fieldValue; + } + } + return new GenericRowWithSchema(fields, schema); + } + private Object convert(Object field, SeaTunnelDataType dataType) { if (field == null) { return null; @@ -156,6 +171,20 @@ public SeaTunnelRow reconvert(SeaTunnelRow engineRow) throws IOException { return (SeaTunnelRow) reconvert(engineRow, dataType); } + public SeaTunnelRow unpack(GenericRowWithSchema engineRow) throws IOException { + SeaTunnelRowType rowType = (SeaTunnelRowType) dataType; + RowKind rowKind = RowKind.fromByteValue(engineRow.getByte(0)); + String tableId = engineRow.getString(1); + Object[] fields = new Object[rowType.getTotalFields()]; + for (int i = 0; i < fields.length; i++) { + fields[i] = reconvert(engineRow.get(i + 2), rowType.getFieldType(i)); + } + SeaTunnelRow seaTunnelRow = new SeaTunnelRow(fields); + seaTunnelRow.setRowKind(rowKind); + seaTunnelRow.setTableId(tableId); + return seaTunnelRow; + } + private Object reconvert(Object field, SeaTunnelDataType dataType) { if (field == null) { return null; diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/utils/TypeConverterUtils.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/utils/TypeConverterUtils.java index a4dbcd0c729..627cde28340 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/utils/TypeConverterUtils.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/utils/TypeConverterUtils.java @@ -43,7 +43,9 @@ public class TypeConverterUtils { private static final Map> TO_SEA_TUNNEL_TYPES = new HashMap<>(16); - public static final String ROW_KIND_FIELD = "op"; + public static final String ROW_KIND_FIELD = "seatunnel_row_kind"; + public static final String ROW = "row"; + public static final String TABLE_ID = "seatunnel_table_id"; public static final String LOGICAL_TIME_TYPE_FLAG = "logical_time_type"; static { @@ -129,6 +131,19 @@ private static StructType convert(SeaTunnelRowType rowType) { return new StructType(fields); } + public static DataType parcel(SeaTunnelDataType dataType) { + // 0 -> row kind, 1 -> table id + SeaTunnelRowType seaTunnelRowType = (SeaTunnelRowType) dataType; + StructField[] fields = new StructField[2 + seaTunnelRowType.getTotalFields()]; + fields[0] = new StructField(ROW_KIND_FIELD, DataTypes.ByteType, true, Metadata.empty()); + fields[1] = new StructField(TABLE_ID, DataTypes.StringType, true, Metadata.empty()); + StructType structType = (StructType) convert(dataType); + for (int i = 0; i < seaTunnelRowType.getTotalFields(); i++) { + fields[i + 2] = structType.fields()[i]; + } + return new StructType(fields); + } + public static SeaTunnelDataType convert(DataType sparkType) { checkNotNull(sparkType, "The Spark's data type is required."); SeaTunnelDataType dataType = TO_SEA_TUNNEL_TYPES.get(sparkType); diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/test/java/org/apache/seatunnel/translation/spark/execution/MultiTableManagerTest.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/test/java/org/apache/seatunnel/translation/spark/execution/MultiTableManagerTest.java new file mode 100644 index 00000000000..ad5fdfbea53 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/test/java/org/apache/seatunnel/translation/spark/execution/MultiTableManagerTest.java @@ -0,0 +1,950 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.translation.spark.execution; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.RowKind; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.translation.spark.serialization.InternalMultiRowCollector; +import org.apache.seatunnel.translation.spark.serialization.InternalRowCollector; +import org.apache.seatunnel.translation.spark.serialization.InternalRowConverter; +import org.apache.seatunnel.translation.spark.utils.InstantConverterUtils; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.MutableAny; +import org.apache.spark.sql.catalyst.expressions.MutableBoolean; +import org.apache.spark.sql.catalyst.expressions.MutableByte; +import org.apache.spark.sql.catalyst.expressions.MutableDouble; +import org.apache.spark.sql.catalyst.expressions.MutableFloat; +import org.apache.spark.sql.catalyst.expressions.MutableInt; +import org.apache.spark.sql.catalyst.expressions.MutableLong; +import org.apache.spark.sql.catalyst.expressions.MutableShort; +import org.apache.spark.sql.catalyst.expressions.MutableValue; +import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow; +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.seatunnel.translation.spark.utils.TypeConverterUtils.ROW_KIND_FIELD; +import static org.apache.seatunnel.translation.spark.utils.TypeConverterUtils.TABLE_ID; +import static org.apache.spark.sql.types.DataTypes.BooleanType; +import static org.apache.spark.sql.types.DataTypes.ByteType; +import static org.apache.spark.sql.types.DataTypes.DateType; +import static org.apache.spark.sql.types.DataTypes.DoubleType; +import static org.apache.spark.sql.types.DataTypes.FloatType; +import static org.apache.spark.sql.types.DataTypes.IntegerType; +import static org.apache.spark.sql.types.DataTypes.LongType; +import static org.apache.spark.sql.types.DataTypes.NullType; +import static org.apache.spark.sql.types.DataTypes.ShortType; +import static org.apache.spark.sql.types.DataTypes.StringType; +import static org.apache.spark.sql.types.DataTypes.TimestampType; + +public class MultiTableManagerTest { + + private SeaTunnelRowType rowType1; + private CatalogTable catalogTable1; + private SeaTunnelRowType rowType2; + private CatalogTable catalogTable2; + private SeaTunnelRowType rowType3; + private CatalogTable catalogTable3; + + private StructType structType1; + private StructType structType2; + private StructType structType3; + + private SeaTunnelRow seaTunnelRow1; + private SeaTunnelRow seaTunnelRow3; + + private SpecificInternalRow specificInternalRow1; + private SpecificInternalRow specificInternalRow2; + private SpecificInternalRow specificInternalRow3; + + @Test + public void testMergeSchema() { + initSchema(); + MultiTableManager multiTableManager1 = + new MultiTableManager(new CatalogTable[] {catalogTable1, catalogTable2}); + StructType tableSchema1 = multiTableManager1.getTableSchema(); + Assertions.assertEquals(structType1, tableSchema1); + + MultiTableManager multiTableManager2 = + new MultiTableManager(new CatalogTable[] {catalogTable2, catalogTable1}); + StructType tableSchema2 = multiTableManager2.getTableSchema(); + Assertions.assertEquals(structType1, tableSchema2); + + MultiTableManager multiTableManager3 = + new MultiTableManager(new CatalogTable[] {catalogTable2, catalogTable3}); + StructType tableSchema3 = multiTableManager3.getTableSchema(); + Assertions.assertEquals(structType2, tableSchema3); + + MultiTableManager multiTableManager4 = + new MultiTableManager( + new CatalogTable[] {catalogTable1, catalogTable2, catalogTable3}); + StructType tableSchema4 = multiTableManager4.getTableSchema(); + Assertions.assertEquals(structType2, tableSchema4); + + MultiTableManager multiTableManager5 = + new MultiTableManager(new CatalogTable[] {catalogTable1}); + StructType tableSchema5 = multiTableManager5.getTableSchema(); + Assertions.assertEquals(structType3, tableSchema5); + } + + @Test + public void testWriteConverter() throws IOException { + initSchema(); + initData(); + MultiTableManager multiTableManager = + new MultiTableManager(new CatalogTable[] {catalogTable1}); + SeaTunnelRow seaTunnelRow = multiTableManager.reconvert(specificInternalRow1); + for (int i = 0; i < seaTunnelRow.getFields().length; i++) { + Object[] values = seaTunnelRow.getFields(); + Object[] actual = seaTunnelRow1.getFields(); + for (int v = 0; v < values.length; v++) { + if (values[v] instanceof Object[]) { + Assertions.assertArrayEquals((Object[]) values[v], (Object[]) actual[v]); + } else { + Assertions.assertEquals(values[v], actual[v]); + } + } + } + } + + @Test + public void testMultiWriteConverter() throws IOException { + initSchema(); + initData(); + MultiTableManager multiTableManager = + new MultiTableManager( + new CatalogTable[] {catalogTable1, catalogTable2, catalogTable3}); + SeaTunnelRow seaTunnelRow = multiTableManager.reconvert(specificInternalRow1); + for (int i = 0; i < seaTunnelRow.getFields().length; i++) { + Object[] values = seaTunnelRow.getFields(); + Object[] actual = seaTunnelRow1.getFields(); + for (int v = 0; v < values.length; v++) { + if (values[v] instanceof Object[]) { + Assertions.assertArrayEquals((Object[]) values[v], (Object[]) actual[v]); + } else { + Assertions.assertEquals(values[v], actual[v]); + } + } + } + } + + @Test + public void testMultiReaderConverter() throws IOException { + initSchema(); + initData(); + MultiTableManager multiTableManager = + new MultiTableManager( + new CatalogTable[] {catalogTable1, catalogTable2, catalogTable3}); + InternalMultiRowCollector internalMultiRowCollector = + (InternalMultiRowCollector) + multiTableManager.getInternalRowCollector(null, null, null); + Map rowSerializationMap = + internalMultiRowCollector.getRowSerializationMap(); + InternalRow internalRow = + rowSerializationMap.get(seaTunnelRow1.getTableId()).convert(seaTunnelRow1); + for (int v = 0; v < specificInternalRow2.numFields(); v++) { + if (specificInternalRow2.genericGet(v) instanceof ArrayBasedMapData) { + Assertions.assertEquals( + specificInternalRow2.getMap(v).keyArray(), + internalRow.getMap(v).keyArray()); + Assertions.assertEquals( + specificInternalRow2.getMap(v).valueArray(), + internalRow.getMap(v).valueArray()); + } else if (specificInternalRow2.genericGet(v) instanceof SpecificInternalRow) { + SpecificInternalRow expected = + (SpecificInternalRow) specificInternalRow2.genericGet(v); + SpecificInternalRow actual = + (SpecificInternalRow) ((SpecificInternalRow) internalRow).genericGet(v); + for (int o = 0; v < expected.numFields(); v++) { + if (expected.genericGet(o) instanceof ArrayBasedMapData) { + Assertions.assertEquals( + expected.getMap(o).keyArray(), actual.getMap(o).keyArray()); + Assertions.assertEquals( + expected.getMap(o).valueArray(), actual.getMap(o).valueArray()); + } else { + Assertions.assertEquals( + expected.genericGet(v), + ((SpecificInternalRow) actual).genericGet(v)); + } + } + } else { + Assertions.assertEquals( + specificInternalRow2.genericGet(v), + ((SpecificInternalRow) internalRow).genericGet(v)); + } + } + InternalRow internalRow3 = + rowSerializationMap.get(seaTunnelRow3.getTableId()).convert(seaTunnelRow3); + Assertions.assertEquals(specificInternalRow3, internalRow3); + for (int v = 0; v < specificInternalRow3.numFields(); v++) { + Assertions.assertEquals( + specificInternalRow3.genericGet(v), + ((SpecificInternalRow) internalRow3).genericGet(v)); + } + } + + @Test + public void testReaderConverter() throws IOException { + initSchema(); + initData(); + MultiTableManager multiTableManager = + new MultiTableManager(new CatalogTable[] {catalogTable1}); + InternalRowCollector internalRowCollector = + multiTableManager.getInternalRowCollector(null, null, null); + InternalRowConverter rowSerialization = internalRowCollector.getRowSerialization(); + InternalRow internalRow = rowSerialization.convert(seaTunnelRow1); + for (int v = 0; v < specificInternalRow1.numFields(); v++) { + if (specificInternalRow1.genericGet(v) instanceof ArrayBasedMapData) { + Assertions.assertEquals( + specificInternalRow1.getMap(v).keyArray(), + internalRow.getMap(v).keyArray()); + Assertions.assertEquals( + specificInternalRow1.getMap(v).valueArray(), + internalRow.getMap(v).valueArray()); + } else if (specificInternalRow1.genericGet(v) instanceof SpecificInternalRow) { + SpecificInternalRow expected = + (SpecificInternalRow) specificInternalRow1.genericGet(v); + SpecificInternalRow actual = + (SpecificInternalRow) ((SpecificInternalRow) internalRow).genericGet(v); + for (int o = 0; v < expected.numFields(); v++) { + if (expected.genericGet(o) instanceof ArrayBasedMapData) { + Assertions.assertEquals( + expected.getMap(o).keyArray(), actual.getMap(o).keyArray()); + Assertions.assertEquals( + expected.getMap(o).valueArray(), actual.getMap(o).valueArray()); + } else { + Assertions.assertEquals( + expected.genericGet(v), + ((SpecificInternalRow) actual).genericGet(v)); + } + } + } else { + Assertions.assertEquals( + specificInternalRow1.genericGet(v), + ((SpecificInternalRow) internalRow).genericGet(v)); + } + } + } + + public void initSchema() { + this.rowType1 = + new SeaTunnelRowType( + new String[] { + "int", + "string", + "boolean", + "float", + "double", + "byte", + "short", + "long", + "decimal", + "date", + "timestamp", + "null", + "array_string", + "array_boolean", + "array_byte", + "array_short", + "array_int", + "array_long", + "array_float", + "array_double", + "map", + "row" + }, + new SeaTunnelDataType[] { + BasicType.INT_TYPE, + BasicType.STRING_TYPE, + BasicType.BOOLEAN_TYPE, + BasicType.FLOAT_TYPE, + BasicType.DOUBLE_TYPE, + BasicType.BYTE_TYPE, + BasicType.SHORT_TYPE, + BasicType.LONG_TYPE, + new org.apache.seatunnel.api.table.type.DecimalType(10, 2), + LocalTimeType.LOCAL_DATE_TYPE, + LocalTimeType.LOCAL_DATE_TIME_TYPE, + BasicType.VOID_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.STRING_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.BOOLEAN_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.BYTE_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.SHORT_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.INT_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.LONG_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.FLOAT_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.DOUBLE_ARRAY_TYPE, + new org.apache.seatunnel.api.table.type.MapType<>( + BasicType.STRING_TYPE, BasicType.STRING_TYPE), + new SeaTunnelRowType( + new String[] { + "int", + "string", + "boolean", + "float", + "double", + "byte", + "short", + "long", + "decimal", + "date", + "timestamp", + "null", + "array_string", + "array_boolean", + "array_byte", + "array_short", + "array_int", + "array_long", + "array_float", + "array_double", + "map" + }, + new SeaTunnelDataType[] { + BasicType.INT_TYPE, + BasicType.STRING_TYPE, + BasicType.BOOLEAN_TYPE, + BasicType.FLOAT_TYPE, + BasicType.DOUBLE_TYPE, + BasicType.BYTE_TYPE, + BasicType.SHORT_TYPE, + BasicType.LONG_TYPE, + new org.apache.seatunnel.api.table.type.DecimalType(10, 2), + LocalTimeType.LOCAL_DATE_TYPE, + LocalTimeType.LOCAL_DATE_TIME_TYPE, + BasicType.VOID_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .STRING_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .BOOLEAN_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .BYTE_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .SHORT_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .INT_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .LONG_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .FLOAT_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .DOUBLE_ARRAY_TYPE, + new org.apache.seatunnel.api.table.type.MapType<>( + BasicType.STRING_TYPE, BasicType.STRING_TYPE) + }) + }); + + this.rowType2 = + new SeaTunnelRowType( + new String[] { + "int", + "string", + "boolean", + "float", + "double", + "byte", + "short", + "long", + "decimal", + "date", + "timestamp", + "null", + "array_string", + "array_boolean", + "array_byte", + "array_short", + "array_int", + "array_long", + "array_float", + "array_double", + "map", + "row" + }, + new SeaTunnelDataType[] { + BasicType.INT_TYPE, + BasicType.STRING_TYPE, + BasicType.BOOLEAN_TYPE, + BasicType.FLOAT_TYPE, + BasicType.DOUBLE_TYPE, + BasicType.BYTE_TYPE, + BasicType.SHORT_TYPE, + BasicType.LONG_TYPE, + new org.apache.seatunnel.api.table.type.DecimalType(10, 2), + LocalTimeType.LOCAL_DATE_TYPE, + LocalTimeType.LOCAL_DATE_TIME_TYPE, + BasicType.VOID_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.STRING_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.BOOLEAN_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.BYTE_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.SHORT_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.INT_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.LONG_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.FLOAT_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType.DOUBLE_ARRAY_TYPE, + new org.apache.seatunnel.api.table.type.MapType<>( + BasicType.STRING_TYPE, BasicType.STRING_TYPE), + new SeaTunnelRowType( + new String[] { + "int", + "string", + "boolean", + "float", + "double", + "byte", + "short", + "long", + "decimal", + "date", + "timestamp", + "null", + "array_string", + "array_boolean", + "array_byte", + "array_short", + "array_int", + "array_long", + "array_float", + "array_double", + "map" + }, + new SeaTunnelDataType[] { + BasicType.INT_TYPE, + BasicType.STRING_TYPE, + BasicType.BOOLEAN_TYPE, + BasicType.FLOAT_TYPE, + BasicType.DOUBLE_TYPE, + BasicType.BYTE_TYPE, + BasicType.SHORT_TYPE, + BasicType.LONG_TYPE, + new org.apache.seatunnel.api.table.type.DecimalType(10, 2), + LocalTimeType.LOCAL_DATE_TYPE, + LocalTimeType.LOCAL_DATE_TIME_TYPE, + BasicType.VOID_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .STRING_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .BOOLEAN_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .BYTE_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .SHORT_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .INT_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .LONG_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .FLOAT_ARRAY_TYPE, + org.apache.seatunnel.api.table.type.ArrayType + .DOUBLE_ARRAY_TYPE, + new org.apache.seatunnel.api.table.type.MapType<>( + BasicType.STRING_TYPE, BasicType.STRING_TYPE) + }) + }); + + this.rowType3 = + new SeaTunnelRowType( + new String[] { + "int", + "string", + "float1", + "float2", + "boolean1", + "boolean2", + "double", + "byte1", + "byte2", + "long", + "short", + "decimal", + "timestamp", + "date", + "null" + }, + new SeaTunnelDataType[] { + BasicType.INT_TYPE, + BasicType.STRING_TYPE, + BasicType.FLOAT_TYPE, + BasicType.FLOAT_TYPE, + BasicType.BOOLEAN_TYPE, + BasicType.BOOLEAN_TYPE, + BasicType.DOUBLE_TYPE, + BasicType.BYTE_TYPE, + BasicType.BYTE_TYPE, + BasicType.LONG_TYPE, + BasicType.SHORT_TYPE, + new org.apache.seatunnel.api.table.type.DecimalType(10, 2), + LocalTimeType.LOCAL_DATE_TIME_TYPE, + LocalTimeType.LOCAL_DATE_TYPE, + BasicType.VOID_TYPE, + }); + + catalogTable1 = CatalogTableUtil.getCatalogTable("test", "test", "test", "test1", rowType1); + + catalogTable2 = CatalogTableUtil.getCatalogTable("test", "test", "test", "test2", rowType2); + + catalogTable3 = CatalogTableUtil.getCatalogTable("test", "test", "test", "test3", rowType3); + + StructType structType = + new StructType() + .add("int", IntegerType) + .add("string", StringType) + .add("boolean", BooleanType) + .add("float", FloatType) + .add("double", DoubleType) + .add("byte", ByteType) + .add("short", ShortType) + .add("long", LongType) + .add("decimal", new DecimalType(10, 2)) + .add("date", DateType) + .add("timestamp", TimestampType) + .add("null", NullType) + .add("array_string", new ArrayType(StringType, true)) + .add("array_boolean", new ArrayType(BooleanType, true)) + .add("array_byte", new ArrayType(ByteType, true)) + .add("array_short", new ArrayType(ShortType, true)) + .add("array_int", new ArrayType(IntegerType, true)) + .add("array_long", new ArrayType(LongType, true)) + .add("array_float", new ArrayType(FloatType, true)) + .add("array_double", new ArrayType(DoubleType, true)) + .add("map", new MapType(StringType, StringType, true)); + + structType1 = + new StructType() + .add(ROW_KIND_FIELD, DataTypes.ByteType) + .add(TABLE_ID, DataTypes.StringType) + .add("column0", IntegerType) + .add("column1", StringType) + .add("column2", BooleanType) + .add("column3", FloatType) + .add("column4", DoubleType) + .add("column5", ByteType) + .add("column6", ShortType) + .add("column7", LongType) + .add("column8", new DecimalType(10, 2)) + .add("column9", DateType) + .add("column10", TimestampType) + .add("column11", NullType) + .add("column12", new ArrayType(StringType, true)) + .add("column13", new ArrayType(BooleanType, true)) + .add("column14", new ArrayType(ByteType, true)) + .add("column15", new ArrayType(ShortType, true)) + .add("column16", new ArrayType(IntegerType, true)) + .add("column17", new ArrayType(LongType, true)) + .add("column18", new ArrayType(FloatType, true)) + .add("column19", new ArrayType(DoubleType, true)) + .add("column20", new MapType(StringType, StringType, true)) + .add("column21", structType); + + structType2 = + new StructType() + .add(ROW_KIND_FIELD, DataTypes.ByteType) + .add(TABLE_ID, DataTypes.StringType) + .add("column0", IntegerType) + .add("column1", StringType) + .add("column2", BooleanType) + .add("column3", FloatType) + .add("column4", DoubleType) + .add("column5", ByteType) + .add("column6", ShortType) + .add("column7", LongType) + .add("column8", new DecimalType(10, 2)) + .add("column9", DateType) + .add("column10", TimestampType) + .add("column11", NullType) + .add("column12", new ArrayType(StringType, true)) + .add("column13", new ArrayType(BooleanType, true)) + .add("column14", new ArrayType(ByteType, true)) + .add("column15", new ArrayType(ShortType, true)) + .add("column16", new ArrayType(IntegerType, true)) + .add("column17", new ArrayType(LongType, true)) + .add("column18", new ArrayType(FloatType, true)) + .add("column19", new ArrayType(DoubleType, true)) + .add("column20", new MapType(StringType, StringType, true)) + .add("column21", structType) + .add("column22", FloatType) + .add("column23", BooleanType) + .add("column24", ByteType); + + structType3 = + new StructType() + .add(ROW_KIND_FIELD, DataTypes.ByteType) + .add(TABLE_ID, DataTypes.StringType) + .add("int", IntegerType) + .add("string", StringType) + .add("boolean", BooleanType) + .add("float", FloatType) + .add("double", DoubleType) + .add("byte", ByteType) + .add("short", ShortType) + .add("long", LongType) + .add("decimal", new DecimalType(10, 2)) + .add("date", DateType) + .add("timestamp", TimestampType) + .add("null", NullType) + .add("array_string", new ArrayType(StringType, true)) + .add("array_boolean", new ArrayType(BooleanType, true)) + .add("array_byte", new ArrayType(ByteType, true)) + .add("array_short", new ArrayType(ShortType, true)) + .add("array_int", new ArrayType(IntegerType, true)) + .add("array_long", new ArrayType(LongType, true)) + .add("array_float", new ArrayType(FloatType, true)) + .add("array_double", new ArrayType(DoubleType, true)) + .add("map", new MapType(StringType, StringType, true)) + .add("row", structType); + } + + public void initData() { + + SeaTunnelRow row1 = + new SeaTunnelRow( + new Object[] { + 233, + "string3", + true, + 231.1f, + 3533.33, + (byte) 7, + (short) 2, + Long.MAX_VALUE - 2, + new BigDecimal("65.55"), + LocalDate.parse("2001-01-01"), + LocalDateTime.parse("2031-01-01T00:00:00"), + null, + new Object[] {"string1fsa", "stringdsa2", "strfdsaing3"}, + new Object[] {false, true, true}, + new Object[] {(byte) 6, (byte) 2, (byte) 1}, + new Object[] {(short) 7, (short) 8, (short) 9}, + new Object[] {3, 77, 22}, + new Object[] {143L, 642L, 533L}, + new Object[] {24.1f, 54.2f, 1.3f}, + new Object[] {431.11, 2422.22, 3243.33}, + new HashMap() { + { + put("keyfs1", "valfdsue1"); + put("kedfasy2", "vafdslue2"); + put("kefdsay3", "vfdasalue3"); + } + } + }); + + seaTunnelRow1 = + new SeaTunnelRow( + new Object[] { + 233, + "string3", + true, + 231.1f, + 3533.33, + (byte) 7, + (short) 2, + Long.MAX_VALUE - 2, + new BigDecimal("65.55"), + LocalDate.parse("2001-01-01"), + LocalDateTime.parse("2031-01-01T00:00:00"), + null, + new Object[] {"string1fsa", "stringdsa2", "strfdsaing3"}, + new Object[] {false, true, true}, + new Object[] {(byte) 6, (byte) 2, (byte) 1}, + new Object[] {(short) 7, (short) 8, (short) 9}, + new Object[] {3, 77, 22}, + new Object[] {143L, 642L, 533L}, + new Object[] {24.1f, 54.2f, 1.3f}, + new Object[] {431.11, 2422.22, 3243.33}, + new HashMap() { + { + put("keyfs1", "valfdsue1"); + put("kedfasy2", "vafdslue2"); + put("kefdsay3", "vfdasalue3"); + } + }, + row1 + }); + seaTunnelRow1.setRowKind(RowKind.INSERT); + seaTunnelRow1.setTableId("test.test.test1"); + + MutableValue[] mutableValues = new MutableValue[21]; + mutableValues[0] = new MutableInt(); + mutableValues[0].update(233); + mutableValues[1] = new MutableAny(); + mutableValues[1].update(UTF8String.fromString("string3")); + mutableValues[2] = new MutableBoolean(); + mutableValues[2].update(true); + mutableValues[3] = new MutableFloat(); + mutableValues[3].update(231.1f); + mutableValues[4] = new MutableDouble(); + mutableValues[4].update(3533.33); + mutableValues[5] = new MutableByte(); + mutableValues[5].update((byte) 7); + mutableValues[6] = new MutableShort(); + mutableValues[6].update((short) 2); + mutableValues[7] = new MutableLong(); + mutableValues[7].update(Long.MAX_VALUE - 2); + mutableValues[8] = new MutableAny(); + mutableValues[8].update(Decimal.apply(new BigDecimal("65.55"))); + mutableValues[9] = new MutableInt(); + mutableValues[9].update((int) LocalDate.parse("2001-01-01").toEpochDay()); + mutableValues[10] = new MutableAny(); + mutableValues[10].update( + InstantConverterUtils.toEpochMicro( + Timestamp.valueOf(LocalDateTime.parse("2031-01-01T00:00:00")).toInstant())); + mutableValues[11] = new MutableAny(); + mutableValues[12] = new MutableAny(); + mutableValues[12].update( + ArrayData.toArrayData( + new Object[] { + UTF8String.fromString("string1fsa"), + UTF8String.fromString("stringdsa2"), + UTF8String.fromString("strfdsaing3") + })); + + mutableValues[13] = new MutableAny(); + mutableValues[13].update(ArrayData.toArrayData(new Object[] {false, true, true})); + + mutableValues[14] = new MutableAny(); + mutableValues[14].update( + ArrayData.toArrayData(new Object[] {(byte) 6, (byte) 2, (byte) 1})); + + mutableValues[15] = new MutableAny(); + mutableValues[15].update( + ArrayData.toArrayData(new Object[] {(short) 7, (short) 8, (short) 9})); + + mutableValues[16] = new MutableAny(); + mutableValues[16].update(ArrayData.toArrayData(new Object[] {3, 77, 22})); + + mutableValues[17] = new MutableAny(); + mutableValues[17].update(ArrayData.toArrayData(new Object[] {143L, 642L, 533L})); + + mutableValues[18] = new MutableAny(); + mutableValues[18].update(ArrayData.toArrayData(new Object[] {24.1f, 54.2f, 1.3f})); + + mutableValues[19] = new MutableAny(); + mutableValues[19].update(ArrayData.toArrayData(new Object[] {431.11, 2422.22, 3243.33})); + + mutableValues[20] = new MutableAny(); + mutableValues[20].update( + ArrayBasedMapData.apply( + new Object[] { + UTF8String.fromString("kefdsay3"), + UTF8String.fromString("keyfs1"), + UTF8String.fromString("kedfasy2") + }, + new Object[] { + UTF8String.fromString("vfdasalue3"), + UTF8String.fromString("valfdsue1"), + UTF8String.fromString("vafdslue2") + })); + + SpecificInternalRow specificInternalRow = new SpecificInternalRow(mutableValues); + + MutableValue[] mutableValues1 = new MutableValue[24]; + + mutableValues1[0] = new MutableByte(); + mutableValues1[0].update(RowKind.INSERT.toByteValue()); + mutableValues1[1] = new MutableAny(); + mutableValues1[1].update(UTF8String.fromString("test.test.test1")); + mutableValues1[2] = new MutableInt(); + mutableValues1[2].update(233); + mutableValues1[3] = new MutableAny(); + mutableValues1[3].update(UTF8String.fromString("string3")); + mutableValues1[4] = new MutableBoolean(); + mutableValues1[4].update(true); + mutableValues1[5] = new MutableFloat(); + mutableValues1[5].update(231.1f); + mutableValues1[6] = new MutableDouble(); + mutableValues1[6].update(3533.33); + mutableValues1[7] = new MutableByte(); + mutableValues1[7].update((byte) 7); + mutableValues1[8] = new MutableShort(); + mutableValues1[8].update((short) 2); + mutableValues1[9] = new MutableLong(); + mutableValues1[9].update(Long.MAX_VALUE - 2); + mutableValues1[10] = new MutableAny(); + mutableValues1[10].update(Decimal.apply(new BigDecimal("65.55"))); + mutableValues1[11] = new MutableInt(); + mutableValues1[11].update((int) LocalDate.parse("2001-01-01").toEpochDay()); + mutableValues1[12] = new MutableAny(); + mutableValues1[12].update( + InstantConverterUtils.toEpochMicro( + Timestamp.valueOf(LocalDateTime.parse("2031-01-01T00:00:00")).toInstant())); + mutableValues1[13] = new MutableAny(); + mutableValues1[14] = new MutableAny(); + mutableValues1[14].update( + ArrayData.toArrayData( + new Object[] { + UTF8String.fromString("string1fsa"), + UTF8String.fromString("stringdsa2"), + UTF8String.fromString("strfdsaing3") + })); + + mutableValues1[15] = new MutableAny(); + mutableValues1[15].update(ArrayData.toArrayData(new Object[] {false, true, true})); + + mutableValues1[16] = new MutableAny(); + mutableValues1[16].update( + ArrayData.toArrayData(new Object[] {(byte) 6, (byte) 2, (byte) 1})); + + mutableValues1[17] = new MutableAny(); + mutableValues1[17].update( + ArrayData.toArrayData(new Object[] {(short) 7, (short) 8, (short) 9})); + + mutableValues1[18] = new MutableAny(); + mutableValues1[18].update(ArrayData.toArrayData(new Object[] {3, 77, 22})); + + mutableValues1[19] = new MutableAny(); + mutableValues1[19].update(ArrayData.toArrayData(new Object[] {143L, 642L, 533L})); + + mutableValues1[20] = new MutableAny(); + mutableValues1[20].update(ArrayData.toArrayData(new Object[] {24.1f, 54.2f, 1.3f})); + + mutableValues1[21] = new MutableAny(); + mutableValues1[21].update(ArrayData.toArrayData(new Object[] {431.11, 2422.22, 3243.33})); + + mutableValues1[22] = new MutableAny(); + mutableValues1[22].update( + ArrayBasedMapData.apply( + new Object[] { + UTF8String.fromString("kefdsay3"), + UTF8String.fromString("keyfs1"), + UTF8String.fromString("kedfasy2") + }, + new Object[] { + UTF8String.fromString("vfdasalue3"), + UTF8String.fromString("valfdsue1"), + UTF8String.fromString("vafdslue2") + })); + + mutableValues1[23] = new MutableAny(); + mutableValues1[23].update(specificInternalRow); + + specificInternalRow1 = new SpecificInternalRow(mutableValues1); + + MutableValue[] mutableValues2 = new MutableValue[27]; + + for (int i = 0; i < mutableValues1.length; i++) { + mutableValues2[i] = mutableValues1[i].copy(); + } + mutableValues2[24] = new MutableAny(); + mutableValues2[25] = new MutableAny(); + mutableValues2[26] = new MutableAny(); + + specificInternalRow2 = new SpecificInternalRow(mutableValues2); + + seaTunnelRow3 = + new SeaTunnelRow( + new Object[] { + 233, + "string3", + 231.1f, + 231.1f, + true, + true, + 3533.33, + (byte) 7, + (byte) 7, + Long.MAX_VALUE - 2, + (short) 2, + new BigDecimal("65.55"), + LocalDateTime.parse("2031-01-01T00:00:00"), + LocalDate.parse("2001-01-01"), + null + }); + seaTunnelRow3.setRowKind(RowKind.INSERT); + seaTunnelRow3.setTableId("test.test.test3"); + + // [0, 1, 3, 22, 2, 23, 4, 5, 24, 7, 6, 8, 10, 9, 11] + MutableValue[] mutableValues3 = new MutableValue[27]; + mutableValues3[0] = new MutableByte(); + mutableValues3[0].update(RowKind.INSERT.toByteValue()); + mutableValues3[1] = new MutableAny(); + mutableValues3[1].update(UTF8String.fromString("test.test.test3")); + + mutableValues3[2] = new MutableInt(); + mutableValues3[2].update(233); + + mutableValues3[3] = new MutableAny(); + mutableValues3[3].update(UTF8String.fromString("string3")); + + mutableValues3[5] = new MutableFloat(); + mutableValues3[5].update(231.1f); + + mutableValues3[24] = new MutableFloat(); + mutableValues3[24].update(231.1f); + + mutableValues3[4] = new MutableBoolean(); + mutableValues3[4].update(true); + + mutableValues3[25] = new MutableBoolean(); + mutableValues3[25].update(true); + + mutableValues3[6] = new MutableDouble(); + mutableValues3[6].update(3533.33); + + mutableValues3[7] = new MutableByte(); + mutableValues3[7].update((byte) 7); + + mutableValues3[26] = new MutableByte(); + mutableValues3[26].update((byte) 7); + + mutableValues3[9] = new MutableLong(); + mutableValues3[9].update(Long.MAX_VALUE - 2); + + mutableValues3[8] = new MutableShort(); + mutableValues3[8].update((short) 2); + + mutableValues3[10] = new MutableAny(); + mutableValues3[10].update(Decimal.apply(new BigDecimal("65.55"))); + + mutableValues3[12] = new MutableLong(); + mutableValues3[12].update( + InstantConverterUtils.toEpochMicro( + Timestamp.valueOf(LocalDateTime.parse("2031-01-01T00:00:00")).toInstant())); + + mutableValues3[11] = new MutableInt(); + mutableValues3[11].update((int) LocalDate.parse("2001-01-01").toEpochDay()); + + for (int i = 0; i < mutableValues3.length; i++) { + if (mutableValues3[i] == null) { + mutableValues3[i] = new MutableAny(); + } + } + specificInternalRow3 = new SpecificInternalRow(mutableValues3); + } +} From e57a11aaa52e02c490062bde458b1ec433b5de38 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Wed, 14 Aug 2024 16:30:18 +0800 Subject: [PATCH 057/361] [ASF] Add new collaborator (#7399) --- .asf.yaml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.asf.yaml b/.asf.yaml index dcab78f6fd9..693bef54556 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -33,6 +33,8 @@ github: squash: true merge: false rebase: false + collaborators: + - arshadmohammad protected_branches: dev: required_status_checks: From 16c5065b2fe63bbf6676c33a0dd61416b25d7154 Mon Sep 17 00:00:00 2001 From: corgy-w <73771213+corgy-w@users.noreply.github.com> Date: Wed, 14 Aug 2024 17:25:49 +0800 Subject: [PATCH 058/361] [Feature][Zeta] Added other metrics info of multi-table (#7338) * [Feature][Zeta] Added other metrics info of multi-table * [Feature][Zeta] Added other metrics restApi * [Feature][Zeta] Added other metrics restApi test * [Feature][Zeta] update rest-api doc * [Improve][Connector-V2] optimize code * [Improve][Connector-V2] optimize code --------- Co-authored-by: wangchao --- docs/en/seatunnel-engine/rest-api.md | 18 +- docs/zh/seatunnel-engine/rest-api.md | 18 +- .../engine/e2e/MultiTableMetricsIT.java | 101 ++++++-- .../batch_fake_multi_table_to_console.conf | 102 ++++++-- .../engine/client/SeaTunnelClientTest.java | 50 +++- .../metrics/TaskMetricsCalcContext.java | 211 +++++++++++++++++ .../rest/RestHttpGetCommandProcessor.java | 223 ++++++++++++++---- .../server/task/SeaTunnelSourceCollector.java | 60 +---- .../server/task/flow/SinkFlowLifeCycle.java | 62 +---- 9 files changed, 657 insertions(+), 188 deletions(-) create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/metrics/TaskMetricsCalcContext.java diff --git a/docs/en/seatunnel-engine/rest-api.md b/docs/en/seatunnel-engine/rest-api.md index 99bba92dae0..eb5eacfed13 100644 --- a/docs/en/seatunnel-engine/rest-api.md +++ b/docs/en/seatunnel-engine/rest-api.md @@ -192,8 +192,22 @@ This API has been deprecated, please use /hazelcast/rest/maps/job-info/:jobId in ] }, "metrics": { - "sourceReceivedCount": "", - "sinkWriteCount": "" + "SourceReceivedCount": "", + "SourceReceivedQPS": "", + "SourceReceivedBytes": "", + "SourceReceivedBytesPerSeconds": "", + "SinkWriteCount": "", + "SinkWriteQPS": "", + "SinkWriteBytes": "", + "SinkWriteBytesPerSeconds": "", + "TableSourceReceivedCount": {}, + "TableSourceReceivedBytes": {}, + "TableSourceReceivedBytesPerSeconds": {}, + "TableSourceReceivedQPS": {}, + "TableSinkWriteCount": {}, + "TableSinkWriteQPS": {}, + "TableSinkWriteBytes": {}, + "TableSinkWriteBytesPerSeconds": {} }, "finishedTime": "", "errorMsg": null, diff --git a/docs/zh/seatunnel-engine/rest-api.md b/docs/zh/seatunnel-engine/rest-api.md index 1b0166425ba..d38ad61268c 100644 --- a/docs/zh/seatunnel-engine/rest-api.md +++ b/docs/zh/seatunnel-engine/rest-api.md @@ -134,8 +134,22 @@ network: ] }, "metrics": { - "sourceReceivedCount": "", - "sinkWriteCount": "" + "SourceReceivedCount": "", + "SourceReceivedQPS": "", + "SourceReceivedBytes": "", + "SourceReceivedBytesPerSeconds": "", + "SinkWriteCount": "", + "SinkWriteQPS": "", + "SinkWriteBytes": "", + "SinkWriteBytesPerSeconds": "", + "TableSourceReceivedCount": {}, + "TableSourceReceivedBytes": {}, + "TableSourceReceivedBytesPerSeconds": {}, + "TableSourceReceivedQPS": {}, + "TableSinkWriteCount": {}, + "TableSinkWriteQPS": {}, + "TableSinkWriteBytes": {}, + "TableSinkWriteBytesPerSeconds": {} }, "finishedTime": "", "errorMsg": null, diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java index 59942eb4cc8..61df054d074 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java @@ -35,6 +35,7 @@ import com.hazelcast.client.config.ClientConfig; import com.hazelcast.instance.impl.HazelcastInstanceImpl; +import io.restassured.response.Response; import java.util.Collections; import java.util.concurrent.TimeUnit; @@ -82,33 +83,101 @@ public void multiTableMetrics() { Collections.singletonList(node1) .forEach( instance -> { - given().get( - HOST - + instance.getCluster() - .getLocalMember() - .getAddress() - .getPort() - + RestConstant.JOB_INFO_URL - + "/" - + batchJobProxy.getJobId()) - .then() + Response response = + given().get( + HOST + + instance.getCluster() + .getLocalMember() + .getAddress() + .getPort() + + RestConstant.JOB_INFO_URL + + "/" + + batchJobProxy.getJobId()); + // In the test example, the data size of a single [3, "C", 100] is 13 + int dataSize = 13; + response.prettyPrint(); + response.then() .statusCode(200) .body("jobName", equalTo("batch_fake_multi_table_to_console")) .body("jobStatus", equalTo("FINISHED")) - .body("metrics.SourceReceivedCount", equalTo("50")) - .body("metrics.SinkWriteCount", equalTo("50")) + .body("metrics.SourceReceivedCount", equalTo("15")) + .body("metrics.SinkWriteCount", equalTo("15")) .body( "metrics.TableSourceReceivedCount.'fake.table1'", - equalTo("20")) + equalTo("10")) .body( "metrics.TableSourceReceivedCount.'fake.public.table2'", - equalTo("30")) + equalTo("5")) .body( "metrics.TableSinkWriteCount.'fake.table1'", - equalTo("20")) + equalTo("10")) .body( "metrics.TableSinkWriteCount.'fake.public.table2'", - equalTo("30")); + equalTo("5")) + .body( + "metrics.SourceReceivedBytes", + equalTo(String.valueOf(dataSize * 15))) + .body( + "metrics.SinkWriteBytes", + equalTo(String.valueOf(dataSize * 15))) + .body( + "metrics.TableSourceReceivedBytes.'fake.table1'", + equalTo(String.valueOf(dataSize * 10))) + .body( + "metrics.TableSourceReceivedBytes.'fake.public.table2'", + equalTo(String.valueOf(dataSize * 5))) + .body( + "metrics.TableSinkWriteBytes.'fake.table1'", + equalTo(String.valueOf(dataSize * 10))) + .body( + "metrics.TableSinkWriteBytes.'fake.public.table2'", + equalTo(String.valueOf(dataSize * 5))); + Assertions.assertTrue( + Double.parseDouble(response.path("metrics.SourceReceivedQPS")) + > 0 + && Double.parseDouble( + response.path( + "metrics.TableSourceReceivedQPS.'fake.table1'")) + > 0 + && Double.parseDouble( + response.path( + "metrics.TableSourceReceivedQPS.'fake.public.table2'")) + > 0 + && Double.parseDouble( + response.path("metrics.SinkWriteQPS")) + > 0 + && Double.parseDouble( + response.path( + "metrics.TableSinkWriteQPS.'fake.table1'")) + > 0 + && Double.parseDouble( + response.path( + "metrics.TableSinkWriteQPS.'fake.public.table2'")) + > 0 + && Double.parseDouble( + response.path( + "metrics.SourceReceivedBytesPerSeconds")) + > 0 + && Double.parseDouble( + response.path( + "metrics.TableSourceReceivedBytesPerSeconds.'fake.table1'")) + > 0 + && Double.parseDouble( + response.path( + "metrics.TableSourceReceivedBytesPerSeconds.'fake.public.table2'")) + > 0 + && Double.parseDouble( + response.path( + "metrics.SinkWriteBytesPerSeconds")) + > 0 + && Double.parseDouble( + response.path( + "metrics.TableSinkWriteBytesPerSeconds.'fake.table1'")) + > 0 + && Double.parseDouble( + response.path( + "metrics.TableSinkWriteBytesPerSeconds.'fake.public.table2'")) + > 0); }); } diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf index c51929a0edb..7459cc150e4 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf @@ -26,29 +26,93 @@ env { source { # This is a example source plugin **only for test and demonstrate the feature source plugin** FakeSource { - result_table_name = "fake1" - row.num = 20 - schema = { - table = "fake.table1" - fields { - name = "string" - age = "int" + result_table_name = "fake1" + schema = { + table = "fake.table1" + fields { + id = bigint + name = string + score = int + } } + rows = [ + { + kind = INSERT + fields = [1, "A", 100] + }, + { + kind = INSERT + fields = [2, "B", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + } + { + kind = UPDATE_BEFORE + fields = [1, "A", 100] + }, + { + kind = UPDATE_AFTER + fields = [1, "A", 300] + }, + { + kind = DELETE + fields = [2, "B", 100] + }, + { + kind = INSERT + fields = [2, "B", 100] + } + ] } - } - FakeSource { - result_table_name = "fake2" - row.num = 30 - schema = { - table = "fake.public.table2" - fields { - name = "string" - age = "int" - sex = "int" + FakeSource { + result_table_name = "fake2" + schema = { + table = "fake.public.table2" + fields { + id = bigint + name = string + score = int + } + } + rows = [ + { + kind = INSERT + fields = [1, "A", 100] + }, + { + kind = INSERT + fields = [2, "B", 100] + }, + { + kind = DELETE + fields = [2, "B", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + } + ] } - } - } + } transform { diff --git a/seatunnel-engine/seatunnel-engine-client/src/test/java/org/apache/seatunnel/engine/client/SeaTunnelClientTest.java b/seatunnel-engine/seatunnel-engine-client/src/test/java/org/apache/seatunnel/engine/client/SeaTunnelClientTest.java index 100aa0b3203..a8275a13b74 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/test/java/org/apache/seatunnel/engine/client/SeaTunnelClientTest.java +++ b/seatunnel-engine/seatunnel-engine-client/src/test/java/org/apache/seatunnel/engine/client/SeaTunnelClientTest.java @@ -62,8 +62,12 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES_PER_SECONDS; import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_COUNT; import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_QPS; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES_PER_SECONDS; import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_COUNT; import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_QPS; import static org.awaitility.Awaitility.await; @@ -592,6 +596,23 @@ public void testGetMultiTableJobMetrics() { jobMetrics.contains(SOURCE_RECEIVED_COUNT + "#fake.public.table2")); Assertions.assertTrue(jobMetrics.contains(SINK_WRITE_COUNT + "#fake.table1")); Assertions.assertTrue(jobMetrics.contains(SINK_WRITE_COUNT + "#fake.public.table2")); + Assertions.assertTrue(jobMetrics.contains(SOURCE_RECEIVED_BYTES + "#fake.table1")); + Assertions.assertTrue( + jobMetrics.contains(SOURCE_RECEIVED_BYTES + "#fake.public.table2")); + Assertions.assertTrue(jobMetrics.contains(SINK_WRITE_BYTES + "#fake.table1")); + Assertions.assertTrue(jobMetrics.contains(SINK_WRITE_BYTES + "#fake.public.table2")); + Assertions.assertTrue(jobMetrics.contains(SOURCE_RECEIVED_QPS + "#fake.table1")); + Assertions.assertTrue(jobMetrics.contains(SOURCE_RECEIVED_QPS + "#fake.public.table2")); + Assertions.assertTrue(jobMetrics.contains(SINK_WRITE_QPS + "#fake.table1")); + Assertions.assertTrue(jobMetrics.contains(SINK_WRITE_QPS + "#fake.public.table2")); + Assertions.assertTrue( + jobMetrics.contains(SOURCE_RECEIVED_BYTES_PER_SECONDS + "#fake.table1")); + Assertions.assertTrue( + jobMetrics.contains(SOURCE_RECEIVED_BYTES_PER_SECONDS + "#fake.public.table2")); + Assertions.assertTrue( + jobMetrics.contains(SINK_WRITE_BYTES_PER_SECONDS + "#fake.table1")); + Assertions.assertTrue( + jobMetrics.contains(SINK_WRITE_BYTES_PER_SECONDS + "#fake.public.table2")); log.info("jobMetrics : {}", jobMetrics); JsonNode jobMetricsStr = new ObjectMapper().readTree(jobMetrics); @@ -600,10 +621,6 @@ public void testGetMultiTableJobMetrics() { Spliterators.spliteratorUnknownSize( jobMetricsStr.fieldNames(), 0), false) - .filter( - metricName -> - metricName.startsWith(SOURCE_RECEIVED_COUNT) - || metricName.startsWith(SINK_WRITE_COUNT)) .collect(Collectors.toList()); Map totalCount = @@ -654,6 +671,31 @@ public void testGetMultiTableJobMetrics() { .filter(e -> e.getKey().startsWith(SINK_WRITE_COUNT)) .mapToLong(Map.Entry::getValue) .sum()); + Assertions.assertEquals( + totalCount.get(SOURCE_RECEIVED_BYTES), + tableCount.entrySet().stream() + .filter(e -> e.getKey().startsWith(SOURCE_RECEIVED_BYTES + "#")) + .mapToLong(Map.Entry::getValue) + .sum()); + Assertions.assertEquals( + totalCount.get(SINK_WRITE_BYTES), + tableCount.entrySet().stream() + .filter(e -> e.getKey().startsWith(SINK_WRITE_BYTES + "#")) + .mapToLong(Map.Entry::getValue) + .sum()); + // Instantaneous rates in the same direction are directly added + Assertions.assertEquals( + totalCount.get(SOURCE_RECEIVED_QPS), + tableCount.entrySet().stream() + .filter(e -> e.getKey().startsWith(SOURCE_RECEIVED_QPS + "#")) + .mapToLong(Map.Entry::getValue) + .sum()); + Assertions.assertEquals( + totalCount.get(SINK_WRITE_QPS), + tableCount.entrySet().stream() + .filter(e -> e.getKey().startsWith(SINK_WRITE_QPS + "#")) + .mapToLong(Map.Entry::getValue) + .sum()); } catch (ExecutionException | InterruptedException | JsonProcessingException e) { throw new RuntimeException(e); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/metrics/TaskMetricsCalcContext.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/metrics/TaskMetricsCalcContext.java new file mode 100644 index 00000000000..eab9ecbd348 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/metrics/TaskMetricsCalcContext.java @@ -0,0 +1,211 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.engine.server.metrics; + +import org.apache.seatunnel.api.common.metrics.Counter; +import org.apache.seatunnel.api.common.metrics.Meter; +import org.apache.seatunnel.api.common.metrics.MetricsContext; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.constants.PluginType; + +import org.apache.commons.lang3.StringUtils; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES_PER_SECONDS; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_COUNT; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_QPS; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES_PER_SECONDS; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_COUNT; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_QPS; + +public class TaskMetricsCalcContext { + + private final MetricsContext metricsContext; + + private final PluginType type; + + private Counter count; + + private Map countPerTable = new ConcurrentHashMap<>(); + + private Meter QPS; + + private Map QPSPerTable = new ConcurrentHashMap<>(); + + private Counter bytes; + + private Map bytesPerTable = new ConcurrentHashMap<>(); + + private Meter bytesPerSeconds; + + private Map bytesPerSecondsPerTable = new ConcurrentHashMap<>(); + + public TaskMetricsCalcContext( + MetricsContext metricsContext, + PluginType type, + boolean isMulti, + List tables) { + this.metricsContext = metricsContext; + this.type = type; + initializeMetrics(isMulti, tables); + } + + private void initializeMetrics(boolean isMulti, List tables) { + if (type.equals(PluginType.SINK)) { + this.initializeMetrics( + isMulti, + tables, + SINK_WRITE_COUNT, + SINK_WRITE_QPS, + SINK_WRITE_BYTES, + SINK_WRITE_BYTES_PER_SECONDS); + } else if (type.equals(PluginType.SOURCE)) { + this.initializeMetrics( + isMulti, + tables, + SOURCE_RECEIVED_COUNT, + SOURCE_RECEIVED_QPS, + SOURCE_RECEIVED_BYTES, + SOURCE_RECEIVED_BYTES_PER_SECONDS); + } + } + + private void initializeMetrics( + boolean isMulti, + List tables, + String countName, + String qpsName, + String bytesName, + String bytesPerSecondsName) { + count = metricsContext.counter(countName); + QPS = metricsContext.meter(qpsName); + bytes = metricsContext.counter(bytesName); + bytesPerSeconds = metricsContext.meter(bytesPerSecondsName); + if (isMulti) { + tables.forEach( + tablePath -> { + countPerTable.put( + tablePath.getFullName(), + metricsContext.counter(countName + "#" + tablePath.getFullName())); + QPSPerTable.put( + tablePath.getFullName(), + metricsContext.meter(qpsName + "#" + tablePath.getFullName())); + bytesPerTable.put( + tablePath.getFullName(), + metricsContext.counter(bytesName + "#" + tablePath.getFullName())); + bytesPerSecondsPerTable.put( + tablePath.getFullName(), + metricsContext.meter( + bytesPerSecondsName + "#" + tablePath.getFullName())); + }); + } + } + + public void updateMetrics(Object data) { + count.inc(); + QPS.markEvent(); + if (data instanceof SeaTunnelRow) { + SeaTunnelRow row = (SeaTunnelRow) data; + bytes.inc(row.getBytesSize()); + bytesPerSeconds.markEvent(row.getBytesSize()); + String tableId = row.getTableId(); + + if (StringUtils.isNotBlank(tableId)) { + String tableName = TablePath.of(tableId).getFullName(); + + // Processing count + processMetrics( + countPerTable, + Counter.class, + tableName, + SINK_WRITE_COUNT, + SOURCE_RECEIVED_COUNT, + Counter::inc); + + // Processing bytes + processMetrics( + bytesPerTable, + Counter.class, + tableName, + SINK_WRITE_BYTES, + SOURCE_RECEIVED_BYTES, + counter -> counter.inc(row.getBytesSize())); + + // Processing QPS + processMetrics( + QPSPerTable, + Meter.class, + tableName, + SINK_WRITE_QPS, + SOURCE_RECEIVED_QPS, + Meter::markEvent); + + // Processing bytes rate + processMetrics( + bytesPerSecondsPerTable, + Meter.class, + tableName, + SINK_WRITE_BYTES_PER_SECONDS, + SOURCE_RECEIVED_BYTES_PER_SECONDS, + meter -> meter.markEvent(row.getBytesSize())); + } + } + } + + private void processMetrics( + Map metricMap, + Class cls, + String tableName, + String sinkMetric, + String sourceMetric, + MetricProcessor processor) { + T metric = metricMap.get(tableName); + if (Objects.nonNull(metric)) { + processor.process(metric); + } else { + String metricName = + PluginType.SINK.equals(type) + ? sinkMetric + "#" + tableName + : sourceMetric + "#" + tableName; + T newMetric = createMetric(metricsContext, metricName, cls); + processor.process(newMetric); + metricMap.put(tableName, newMetric); + } + } + + private T createMetric( + MetricsContext metricsContext, String metricName, Class metricClass) { + if (metricClass == Counter.class) { + return metricClass.cast(metricsContext.counter(metricName)); + } else if (metricClass == Meter.class) { + return metricClass.cast(metricsContext.meter(metricName)); + } + throw new IllegalArgumentException("Unsupported metric class: " + metricClass.getName()); + } + + @FunctionalInterface + interface MetricProcessor { + void process(T t); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java index d5d60b7cbb4..fec77708b68 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java @@ -44,6 +44,8 @@ import org.apache.seatunnel.engine.server.resourcemanager.resource.OverviewInfo; import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; +import org.apache.commons.lang3.ArrayUtils; + import com.hazelcast.cluster.Address; import com.hazelcast.cluster.Cluster; import com.hazelcast.cluster.Member; @@ -65,12 +67,20 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; -import java.util.Spliterators; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; +import java.util.stream.Stream; import java.util.stream.StreamSupport; import static com.hazelcast.internal.ascii.rest.HttpStatusCode.SC_500; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES_PER_SECONDS; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_COUNT; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_QPS; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES_PER_SECONDS; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_COUNT; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_QPS; import static org.apache.seatunnel.engine.server.rest.RestConstant.FINISHED_JOBS_INFO; import static org.apache.seatunnel.engine.server.rest.RestConstant.JOB_INFO_URL; import static org.apache.seatunnel.engine.server.rest.RestConstant.OVERVIEW; @@ -81,14 +91,22 @@ public class RestHttpGetCommandProcessor extends HttpCommandProcessor { - private static final String SOURCE_RECEIVED_COUNT = "SourceReceivedCount"; private static final String TABLE_SOURCE_RECEIVED_COUNT = "TableSourceReceivedCount"; - private static final String SINK_WRITE_COUNT = "SinkWriteCount"; private static final String TABLE_SINK_WRITE_COUNT = "TableSinkWriteCount"; + private static final String TABLE_SOURCE_RECEIVED_QPS = "TableSourceReceivedQPS"; + private static final String TABLE_SINK_WRITE_QPS = "TableSinkWriteQPS"; + private static final String TABLE_SOURCE_RECEIVED_BYTES = "TableSourceReceivedBytes"; + private static final String TABLE_SINK_WRITE_BYTES = "TableSinkWriteBytes"; + private static final String TABLE_SOURCE_RECEIVED_BYTES_PER_SECONDS = + "TableSourceReceivedBytesPerSeconds"; + private static final String TABLE_SINK_WRITE_BYTES_PER_SECONDS = + "TableSinkWriteBytesPerSeconds"; + private final Log4j2HttpGetCommandProcessor original; private NodeEngine nodeEngine; public RestHttpGetCommandProcessor(TextCommandService textCommandService) { + this(textCommandService, new Log4j2HttpGetCommandProcessor(textCommandService)); } @@ -369,43 +387,165 @@ private void getRunningThread(HttpGetCommand command) { private Map getJobMetrics(String jobMetrics) { Map metricsMap = new HashMap<>(); - long sourceReadCount = 0L; - long sinkWriteCount = 0L; - Map tableSourceReceivedCountMap = new HashMap<>(); - Map tableSinkWriteCountMap = new HashMap<>(); + // To add metrics, populate the corresponding array, + String[] countMetricsNames = { + SOURCE_RECEIVED_COUNT, SINK_WRITE_COUNT, SOURCE_RECEIVED_BYTES, SINK_WRITE_BYTES + }; + String[] rateMetricsNames = { + SOURCE_RECEIVED_QPS, + SINK_WRITE_QPS, + SOURCE_RECEIVED_BYTES_PER_SECONDS, + SINK_WRITE_BYTES_PER_SECONDS + }; + String[] tableCountMetricsNames = { + TABLE_SOURCE_RECEIVED_COUNT, + TABLE_SINK_WRITE_COUNT, + TABLE_SOURCE_RECEIVED_BYTES, + TABLE_SINK_WRITE_BYTES + }; + String[] tableRateMetricsNames = { + TABLE_SOURCE_RECEIVED_QPS, + TABLE_SINK_WRITE_QPS, + TABLE_SOURCE_RECEIVED_BYTES_PER_SECONDS, + TABLE_SINK_WRITE_BYTES_PER_SECONDS + }; + Long[] metricsSums = + Stream.generate(() -> 0L).limit(countMetricsNames.length).toArray(Long[]::new); + Double[] metricsRates = + Stream.generate(() -> 0D).limit(rateMetricsNames.length).toArray(Double[]::new); + + // Used to store various indicators at the table + Map[] tableMetricsMaps = + new Map[] { + new HashMap<>(), // Source Received Count + new HashMap<>(), // Sink Write Count + new HashMap<>(), // Source Received Bytes + new HashMap<>(), // Sink Write Bytes + new HashMap<>(), // Source Received QPS + new HashMap<>(), // Sink Write QPS + new HashMap<>(), // Source Received Bytes Per Second + new HashMap<>() // Sink Write Bytes Per Second + }; + try { JsonNode jobMetricsStr = new ObjectMapper().readTree(jobMetrics); - StreamSupport.stream( - Spliterators.spliteratorUnknownSize(jobMetricsStr.fieldNames(), 0), - false) - .filter(metricName -> metricName.contains("#")) - .forEach( + + jobMetricsStr + .fieldNames() + .forEachRemaining( metricName -> { - String tableName = - TablePath.of(metricName.split("#")[1]).getFullName(); - if (metricName.startsWith(SOURCE_RECEIVED_COUNT)) { - tableSourceReceivedCountMap.put( - tableName, jobMetricsStr.get(metricName)); - } - if (metricName.startsWith(SOURCE_RECEIVED_COUNT)) { - tableSinkWriteCountMap.put( - tableName, jobMetricsStr.get(metricName)); + if (metricName.contains("#")) { + String tableName = + TablePath.of(metricName.split("#")[1]).getFullName(); + JsonNode metricNode = jobMetricsStr.get(metricName); + processMetric( + metricName, tableName, metricNode, tableMetricsMaps); } }); - JsonNode sourceReceivedCountJson = jobMetricsStr.get(SOURCE_RECEIVED_COUNT); - JsonNode sinkWriteCountJson = jobMetricsStr.get(SINK_WRITE_COUNT); - for (int i = 0; i < jobMetricsStr.get(SOURCE_RECEIVED_COUNT).size(); i++) { - JsonNode sourceReader = sourceReceivedCountJson.get(i); - JsonNode sinkWriter = sinkWriteCountJson.get(i); - sourceReadCount += sourceReader.get("value").asLong(); - sinkWriteCount += sinkWriter.get("value").asLong(); - } - } catch (JsonProcessingException | NullPointerException e) { + + // Aggregation summary and rate metrics + aggregateMetrics( + jobMetricsStr, + metricsSums, + metricsRates, + ArrayUtils.addAll(countMetricsNames, rateMetricsNames)); + + } catch (JsonProcessingException e) { return metricsMap; } - Map tableSourceReceivedCount = - tableSourceReceivedCountMap.entrySet().stream() + populateMetricsMap( + metricsMap, + tableMetricsMaps, + ArrayUtils.addAll(tableCountMetricsNames, tableRateMetricsNames), + countMetricsNames.length); + populateMetricsMap( + metricsMap, + Stream.concat(Arrays.stream(metricsSums), Arrays.stream(metricsRates)) + .toArray(Number[]::new), + ArrayUtils.addAll(countMetricsNames, rateMetricsNames), + metricsSums.length); + + return metricsMap; + } + + private void processMetric( + String metricName, + String tableName, + JsonNode metricNode, + Map[] tableMetricsMaps) { + if (metricNode == null) return; + + // Define index constant + final int SOURCE_COUNT_IDX = 0, + SINK_COUNT_IDX = 1, + SOURCE_BYTES_IDX = 2, + SINK_BYTES_IDX = 3, + SOURCE_QPS_IDX = 4, + SINK_QPS_IDX = 5, + SOURCE_BYTES_SEC_IDX = 6, + SINK_BYTES_SEC_IDX = 7; + if (metricName.startsWith(SOURCE_RECEIVED_COUNT + "#")) { + tableMetricsMaps[SOURCE_COUNT_IDX].put(tableName, metricNode); + } else if (metricName.startsWith(SINK_WRITE_COUNT + "#")) { + tableMetricsMaps[SINK_COUNT_IDX].put(tableName, metricNode); + } else if (metricName.startsWith(SOURCE_RECEIVED_BYTES + "#")) { + tableMetricsMaps[SOURCE_BYTES_IDX].put(tableName, metricNode); + } else if (metricName.startsWith(SINK_WRITE_BYTES + "#")) { + tableMetricsMaps[SINK_BYTES_IDX].put(tableName, metricNode); + } else if (metricName.startsWith(SOURCE_RECEIVED_QPS + "#")) { + tableMetricsMaps[SOURCE_QPS_IDX].put(tableName, metricNode); + } else if (metricName.startsWith(SINK_WRITE_QPS + "#")) { + tableMetricsMaps[SINK_QPS_IDX].put(tableName, metricNode); + } else if (metricName.startsWith(SOURCE_RECEIVED_BYTES_PER_SECONDS + "#")) { + tableMetricsMaps[SOURCE_BYTES_SEC_IDX].put(tableName, metricNode); + } else if (metricName.startsWith(SINK_WRITE_BYTES_PER_SECONDS + "#")) { + tableMetricsMaps[SINK_BYTES_SEC_IDX].put(tableName, metricNode); + } + } + + private void aggregateMetrics( + JsonNode jobMetricsStr, + Long[] metricsSums, + Double[] metricsRates, + String[] metricsNames) { + for (int i = 0; i < metricsNames.length; i++) { + JsonNode metricNode = jobMetricsStr.get(metricsNames[i]); + if (metricNode != null && metricNode.isArray()) { + for (JsonNode node : metricNode) { + // Match Rate Metrics vs. Value Metrics + if (i < metricsSums.length) { + metricsSums[i] += node.path("value").asLong(); + } else { + metricsRates[i - metricsSums.length] += node.path("value").asDouble(); + } + } + } + } + } + + private void populateMetricsMap( + Map metricsMap, + Object[] metrics, + String[] metricNames, + int countMetricNames) { + for (int i = 0; i < metrics.length; i++) { + if (metrics[i] != null) { + if (metrics[i] instanceof Map) { + metricsMap.put( + metricNames[i], + aggregateMap( + (Map) metrics[i], i >= countMetricNames)); + } else { + metricsMap.put(metricNames[i], metrics[i]); + } + } + } + } + + public static Map aggregateMap(Map inputMap, boolean isRate) { + return isRate + ? inputMap.entrySet().stream() .collect( Collectors.toMap( Map.Entry::getKey, @@ -413,11 +553,12 @@ private Map getJobMetrics(String jobMetrics) { StreamSupport.stream( entry.getValue().spliterator(), false) - .mapToLong( - node -> node.get("value").asLong()) - .sum())); - Map tableSinkWriteCount = - tableSinkWriteCountMap.entrySet().stream() + .mapToDouble( + node -> + node.path("value") + .asDouble()) + .sum())) + : inputMap.entrySet().stream() .collect( Collectors.toMap( Map.Entry::getKey, @@ -426,14 +567,8 @@ private Map getJobMetrics(String jobMetrics) { entry.getValue().spliterator(), false) .mapToLong( - node -> node.get("value").asLong()) + node -> node.path("value").asLong()) .sum())); - - metricsMap.put(SOURCE_RECEIVED_COUNT, sourceReadCount); - metricsMap.put(SINK_WRITE_COUNT, sinkWriteCount); - metricsMap.put(TABLE_SOURCE_RECEIVED_COUNT, tableSourceReceivedCount); - metricsMap.put(TABLE_SINK_WRITE_COUNT, tableSinkWriteCount); - return metricsMap; } private SeaTunnelServer getSeaTunnelServer(boolean shouldBeMaster) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java index e1b24947893..7f2e34bdcb8 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java @@ -17,8 +17,6 @@ package org.apache.seatunnel.engine.server.task; -import org.apache.seatunnel.api.common.metrics.Counter; -import org.apache.seatunnel.api.common.metrics.Meter; import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.table.catalog.TablePath; @@ -30,13 +28,14 @@ import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.constants.PluginType; import org.apache.seatunnel.core.starter.flowcontrol.FlowControlGate; import org.apache.seatunnel.core.starter.flowcontrol.FlowControlStrategy; import org.apache.seatunnel.engine.common.exception.SeaTunnelEngineException; +import org.apache.seatunnel.engine.server.metrics.TaskMetricsCalcContext; import org.apache.seatunnel.engine.server.task.flow.OneInputFlowLifeCycle; import org.apache.commons.collections4.CollectionUtils; -import org.apache.commons.lang3.StringUtils; import lombok.extern.slf4j.Slf4j; @@ -44,15 +43,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES_PER_SECONDS; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_COUNT; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_QPS; - @Slf4j public class SeaTunnelSourceCollector implements Collector { @@ -62,19 +54,12 @@ public class SeaTunnelSourceCollector implements Collector { private final MetricsContext metricsContext; + private final TaskMetricsCalcContext taskMetricsCalcContext; + private final AtomicBoolean schemaChangeBeforeCheckpointSignal = new AtomicBoolean(false); private final AtomicBoolean schemaChangeAfterCheckpointSignal = new AtomicBoolean(false); - private final Counter sourceReceivedCount; - - private final Map sourceReceivedCountPerTable = new ConcurrentHashMap<>(); - - private final Meter sourceReceivedQPS; - private final Counter sourceReceivedBytes; - - private final Meter sourceReceivedBytesPerSeconds; - private volatile boolean emptyThisPollNext; private final DataTypeChangeEventHandler dataTypeChangeEventHandler = new DataTypeChangeEventDispatcher(); @@ -98,20 +83,12 @@ public SeaTunnelSourceCollector( .iterator() .forEachRemaining(type -> this.rowTypeMap.put(type.getKey(), type.getValue())); } - if (CollectionUtils.isNotEmpty(tablePaths)) { - tablePaths.forEach( - tablePath -> - sourceReceivedCountPerTable.put( - tablePath.getFullName(), - metricsContext.counter( - SOURCE_RECEIVED_COUNT - + "#" - + tablePath.getFullName()))); - } - sourceReceivedCount = metricsContext.counter(SOURCE_RECEIVED_COUNT); - sourceReceivedQPS = metricsContext.meter(SOURCE_RECEIVED_QPS); - sourceReceivedBytes = metricsContext.counter(SOURCE_RECEIVED_BYTES); - sourceReceivedBytesPerSeconds = metricsContext.meter(SOURCE_RECEIVED_BYTES_PER_SECONDS); + this.taskMetricsCalcContext = + new TaskMetricsCalcContext( + metricsContext, + PluginType.SOURCE, + CollectionUtils.isNotEmpty(tablePaths), + tablePaths); flowControlGate = FlowControlGate.create(flowControlStrategy); } @@ -129,26 +106,11 @@ public void collect(T row) { throw new SeaTunnelEngineException( "Unsupported row type: " + rowType.getClass().getName()); } - sourceReceivedBytes.inc(size); - sourceReceivedBytesPerSeconds.markEvent(size); flowControlGate.audit((SeaTunnelRow) row); - if (StringUtils.isNotEmpty(tableId)) { - String tableName = TablePath.of(tableId).getFullName(); - Counter sourceTableCounter = sourceReceivedCountPerTable.get(tableName); - if (Objects.nonNull(sourceTableCounter)) { - sourceTableCounter.inc(); - } else { - Counter counter = - metricsContext.counter(SOURCE_RECEIVED_COUNT + "#" + tableName); - counter.inc(); - sourceReceivedCountPerTable.put(tableName, counter); - } - } + taskMetricsCalcContext.updateMetrics(row); } sendRecordToNext(new Record<>(row)); emptyThisPollNext = false; - sourceReceivedCount.inc(); - sourceReceivedQPS.markEvent(); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java index de8257f1e94..cacaa75aaef 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java @@ -17,8 +17,6 @@ package org.apache.seatunnel.engine.server.task.flow; -import org.apache.seatunnel.api.common.metrics.Counter; -import org.apache.seatunnel.api.common.metrics.Meter; import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.event.EventListener; import org.apache.seatunnel.api.serialization.Serializer; @@ -30,13 +28,14 @@ import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.event.SchemaChangeEvent; import org.apache.seatunnel.api.table.type.Record; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.constants.PluginType; import org.apache.seatunnel.engine.core.checkpoint.InternalCheckpointListener; import org.apache.seatunnel.engine.core.dag.actions.SinkAction; import org.apache.seatunnel.engine.server.checkpoint.ActionStateKey; import org.apache.seatunnel.engine.server.checkpoint.ActionSubtaskState; import org.apache.seatunnel.engine.server.event.JobEventListener; import org.apache.seatunnel.engine.server.execution.TaskLocation; +import org.apache.seatunnel.engine.server.metrics.TaskMetricsCalcContext; import org.apache.seatunnel.engine.server.task.SeaTunnelTask; import org.apache.seatunnel.engine.server.task.context.SinkWriterContext; import org.apache.seatunnel.engine.server.task.operation.GetTaskGroupAddressOperation; @@ -45,8 +44,6 @@ import org.apache.seatunnel.engine.server.task.operation.sink.SinkRegisterOperation; import org.apache.seatunnel.engine.server.task.record.Barrier; -import org.apache.commons.lang3.StringUtils; - import com.hazelcast.cluster.Address; import lombok.extern.slf4j.Slf4j; @@ -56,18 +53,12 @@ import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES_PER_SECONDS; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_COUNT; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_QPS; import static org.apache.seatunnel.engine.common.utils.ExceptionUtil.sneaky; import static org.apache.seatunnel.engine.server.task.AbstractTask.serializeStates; @@ -96,15 +87,7 @@ public class SinkFlowLifeCycle sinkWriteCountPerTable = new ConcurrentHashMap<>(); - - private Meter sinkWriteQPS; - - private Counter sinkWriteBytes; - - private Meter sinkWriteBytesPerSeconds; + private TaskMetricsCalcContext taskMetricsCalcContext; private final boolean containAggCommitter; @@ -129,19 +112,13 @@ public SinkFlowLifeCycle( this.containAggCommitter = containAggCommitter; this.metricsContext = metricsContext; this.eventListener = new JobEventListener(taskLocation, runningTask.getExecutionContext()); - sinkWriteCount = metricsContext.counter(SINK_WRITE_COUNT); - sinkWriteQPS = metricsContext.meter(SINK_WRITE_QPS); - sinkWriteBytes = metricsContext.counter(SINK_WRITE_BYTES); - sinkWriteBytesPerSeconds = metricsContext.meter(SINK_WRITE_BYTES_PER_SECONDS); - if (sinkAction.getSink() instanceof MultiTableSink) { - List sinkTables = ((MultiTableSink) sinkAction.getSink()).getSinkTables(); - sinkTables.forEach( - tablePath -> - sinkWriteCountPerTable.put( - tablePath.getFullName(), - metricsContext.counter( - SINK_WRITE_COUNT + "#" + tablePath.getFullName()))); + List sinkTables = new ArrayList<>(); + boolean isMulti = sinkAction.getSink() instanceof MultiTableSink; + if (isMulti) { + sinkTables = ((MultiTableSink) sinkAction.getSink()).getSinkTables(); } + this.taskMetricsCalcContext = + new TaskMetricsCalcContext(metricsContext, PluginType.SINK, isMulti, sinkTables); } @Override @@ -267,26 +244,7 @@ public void received(Record record) { return; } writer.write((T) record.getData()); - sinkWriteCount.inc(); - sinkWriteQPS.markEvent(); - if (record.getData() instanceof SeaTunnelRow) { - long size = ((SeaTunnelRow) record.getData()).getBytesSize(); - sinkWriteBytes.inc(size); - sinkWriteBytesPerSeconds.markEvent(size); - String tableId = ((SeaTunnelRow) record.getData()).getTableId(); - if (StringUtils.isNotBlank(tableId)) { - String tableName = TablePath.of(tableId).getFullName(); - Counter sinkTableCounter = sinkWriteCountPerTable.get(tableName); - if (Objects.nonNull(sinkTableCounter)) { - sinkTableCounter.inc(); - } else { - Counter counter = - metricsContext.counter(SINK_WRITE_COUNT + "#" + tableName); - counter.inc(); - sinkWriteCountPerTable.put(tableName, counter); - } - } - } + taskMetricsCalcContext.updateMetrics(record.getData()); } } catch (Exception e) { throw new RuntimeException(e); From b5140f598ee8f02ba7e6d25428c56a5904aa01ad Mon Sep 17 00:00:00 2001 From: Jarvis Date: Wed, 14 Aug 2024 22:33:57 +0800 Subject: [PATCH 059/361] [Improvement] add starrocks jdbc dialect (#7294) --- docs/en/connector-v2/sink/Jdbc.md | 4 +- docs/en/connector-v2/source/Jdbc.md | 2 +- docs/zh/connector-v2/sink/Jdbc.md | 4 +- .../internal/dialect/DatabaseIdentifier.java | 1 + .../dialect/mysql/MySqlDialectFactory.java | 5 +++ .../dialect/starrocks/StarRocksDialect.java | 40 +++++++++++++++++ .../seatunnel/jdbc/JdbcStarRocksdbIT.java | 5 ++- .../resources/jdbc_starrocks_dialect.conf | 44 +++++++++++++++++++ 8 files changed, 100 insertions(+), 5 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/starrocks/StarRocksDialect.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/resources/jdbc_starrocks_dialect.conf diff --git a/docs/en/connector-v2/sink/Jdbc.md b/docs/en/connector-v2/sink/Jdbc.md index c46933b486b..8ec58506b4e 100644 --- a/docs/en/connector-v2/sink/Jdbc.md +++ b/docs/en/connector-v2/sink/Jdbc.md @@ -82,7 +82,9 @@ Use this sql write upstream input datas to database. e.g `INSERT ...` ### compatible_mode [string] -The compatible mode of database, required when the database supports multiple compatible modes. For example, when using OceanBase database, you need to set it to 'mysql' or 'oracle'. +The compatible mode of database, required when the database supports multiple compatible modes. + +For example, when using OceanBase database, you need to set it to 'mysql' or 'oracle'. when using StarRocks, you need set it to `starrocks`. Postgres 9.5 version or below,please set it to `postgresLow` to support cdc diff --git a/docs/en/connector-v2/source/Jdbc.md b/docs/en/connector-v2/source/Jdbc.md index 7fab8d50b25..1b9acc025b0 100644 --- a/docs/en/connector-v2/source/Jdbc.md +++ b/docs/en/connector-v2/source/Jdbc.md @@ -46,7 +46,7 @@ supports query SQL and can achieve projection effect. | user | String | No | - | userName | | password | String | No | - | password | | query | String | No | - | Query statement | -| compatible_mode | String | No | - | The compatible mode of database, required when the database supports multiple compatible modes. For example, when using OceanBase database, you need to set it to 'mysql' or 'oracle'. | +| compatible_mode | String | No | - | The compatible mode of database, required when the database supports multiple compatible modes.
For example, when using OceanBase database, you need to set it to 'mysql' or 'oracle'.
when using starrocks, you need set it to `starrocks` | | connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete. | | partition_column | String | No | - | The column name for split data. | | partition_upper_bound | Long | No | - | The partition_column max value for scan, if not set SeaTunnel will query database get max value. | diff --git a/docs/zh/connector-v2/sink/Jdbc.md b/docs/zh/connector-v2/sink/Jdbc.md index d61292cb921..0cc8605a377 100644 --- a/docs/zh/connector-v2/sink/Jdbc.md +++ b/docs/zh/connector-v2/sink/Jdbc.md @@ -79,7 +79,9 @@ JDBC 连接的 URL。参考案例:`jdbc:postgresql://localhost/test` ### compatible_mode [string] -数据库的兼容模式,当数据库支持多种兼容模式时需要。例如,使用 OceanBase 数据库时,需要将其设置为 'mysql' 或 'oracle' 。 +数据库的兼容模式,当数据库支持多种兼容模式时需要。 + +例如,使用 OceanBase 数据库时,需要将其设置为 'mysql' 或 'oracle' 。使用StarRocks时,需要将其设置为`starrocks`。 Postgres 9.5及以下版本,请设置为 `postgresLow` 来支持 CDC diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java index bf00298a742..45f849c28bd 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java @@ -25,6 +25,7 @@ public class DatabaseIdentifier { public static final String INFORMIX = "Informix"; public static final String KINGBASE = "KingBase"; public static final String MYSQL = "MySQL"; + public static final String STARROCKS = "StarRocks"; public static final String ORACLE = "Oracle"; public static final String PHOENIX = "Phoenix"; public static final String POSTGRESQL = "Postgres"; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlDialectFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlDialectFactory.java index a4f89a4dc85..f8278a60ccf 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlDialectFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlDialectFactory.java @@ -17,8 +17,10 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.mysql; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectFactory; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.starrocks.StarRocksDialect; import com.google.auto.service.AutoService; @@ -39,6 +41,9 @@ public JdbcDialect create() { @Override public JdbcDialect create(@Nonnull String compatibleMode, String fieldIde) { + if (DatabaseIdentifier.STARROCKS.equalsIgnoreCase(compatibleMode)) { + return new StarRocksDialect(fieldIde); + } return new MysqlDialect(fieldIde); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/starrocks/StarRocksDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/starrocks/StarRocksDialect.java new file mode 100644 index 00000000000..d7ee7965273 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/starrocks/StarRocksDialect.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.starrocks; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.mysql.MysqlDialect; + +public class StarRocksDialect extends MysqlDialect { + + public StarRocksDialect() {} + + public StarRocksDialect(String fieldIde) { + this.fieldIde = fieldIde; + } + + @Override + public String dialectName() { + return DatabaseIdentifier.STARROCKS; + } + + @Override + public String hashModForField(String fieldName, int mod) { + return "ABS(md5sum_numeric(" + quoteIdentifier(fieldName) + ") % " + mod + ")"; + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcStarRocksdbIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcStarRocksdbIT.java index e7fc94e642e..1d41c480c34 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcStarRocksdbIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcStarRocksdbIT.java @@ -37,7 +37,7 @@ public class JdbcStarRocksdbIT extends AbstractJdbcIT { - private static final String DOCKER_IMAGE = "d87904488/starrocks-starter:2.2.1"; + private static final String DOCKER_IMAGE = "starrocks/allin1-ubuntu:2.5.12"; private static final String DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; private static final String NETWORK_ALIASES = "e2e_starRocksdb"; private static final int SR_PORT = 9030; @@ -51,7 +51,8 @@ public class JdbcStarRocksdbIT extends AbstractJdbcIT { private static final String SINK_TABLE = "e2e_table_sink"; private static final List CONFIG_FILE = - Lists.newArrayList("/jdbc_starrocks_source_to_sink.conf"); + Lists.newArrayList( + "/jdbc_starrocks_source_to_sink.conf", "/jdbc_starrocks_dialect.conf"); private static final String CREATE_SQL = "create table %s (\n" diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/resources/jdbc_starrocks_dialect.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/resources/jdbc_starrocks_dialect.conf new file mode 100644 index 00000000000..69fe5538f58 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/resources/jdbc_starrocks_dialect.conf @@ -0,0 +1,44 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" + jars = "file:///tmp/jars/mysql-connector-java-8.0.16.jar" +} + +source { + Jdbc { + driver = com.mysql.cj.jdbc.Driver + url = "jdbc:mysql://e2e_starRocksdb:9030" + user = root + password = "" + query = "select BIGINT_COL, LARGEINT_COL, SMALLINT_COL, TINYINT_COL, BOOLEAN_COL, DECIMAL_COL, DOUBLE_COL, FLOAT_COL, INT_COL, CHAR_COL, VARCHAR_11_COL, STRING_COL, DATETIME_COL, DATE_COL from `test`.`e2e_table_source`" + partition_column = "STRING_COL" + compatible_mode = "starrocks" + } +} + +sink { + Jdbc { + driver = com.mysql.cj.jdbc.Driver + url = "jdbc:mysql://e2e_starRocksdb:9030" + user = root + password = "" + query = "INSERT INTO `test`.`e2e_table_sink` (BIGINT_COL, LARGEINT_COL, SMALLINT_COL, TINYINT_COL, BOOLEAN_COL, DECIMAL_COL, DOUBLE_COL, FLOAT_COL, INT_COL, CHAR_COL, VARCHAR_11_COL, STRING_COL, DATETIME_COL, DATE_COL) VALUES(?,?,?,?,?,?,?,?,?,?,?,?,?,?)" + } +} From e3001207c887f73cd44648b4adb79173572bb4a9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=8D=A2=E5=AE=97=E6=9F=B1?= Date: Wed, 14 Aug 2024 22:34:47 +0800 Subject: [PATCH 060/361] [Feature][Connector-V2][Iceberg] Support Iceberg Kerberos (#7246) --- .../api/kerberos/KerberosConfig.java | 43 ++++++++++++ .../common/exception/CommonErrorCode.java | 3 +- .../seatunnel/file/config/BaseSinkConfig.java | 22 +----- .../iceberg/IcebergCatalogLoader.java | 67 ++++++++++++++++++- .../catalog/IcebergCatalogFactory.java | 7 +- .../iceberg/config/CommonConfig.java | 18 ++++- .../iceberg/data/IcebergTypeMapper.java | 1 - .../iceberg/sink/IcebergSinkFactory.java | 3 + .../enumerator/AbstractSplitEnumerator.java | 2 +- .../source/reader/IcebergSourceReader.java | 2 +- .../seatunnel/iceberg/utils/SchemaUtils.java | 11 ++- .../iceberg/catalog/IcebergCatalogTest.java | 7 ++ 12 files changed, 150 insertions(+), 36 deletions(-) create mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/kerberos/KerberosConfig.java diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/kerberos/KerberosConfig.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/kerberos/KerberosConfig.java new file mode 100644 index 00000000000..d501a3ea49a --- /dev/null +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/kerberos/KerberosConfig.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.api.kerberos; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; + +public class KerberosConfig { + + public static final Option KERBEROS_PRINCIPAL = + Options.key("kerberos_principal") + .stringType() + .noDefaultValue() + .withDescription("When use kerberos, we should set kerberos user principal"); + + public static final Option KRB5_PATH = + Options.key("krb5_path") + .stringType() + .defaultValue("/etc/krb5.conf") + .withDescription( + "When use kerberos, we should set krb5 path file path such as '/seatunnel/krb5.conf' or use the default path '/etc/krb5.conf'"); + + public static final Option KERBEROS_KEYTAB_PATH = + Options.key("kerberos_keytab_path") + .stringType() + .noDefaultValue() + .withDescription("When using kerberos, We should specify the keytab path"); +} diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java index 79621c42168..99cb7353cf6 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java @@ -49,11 +49,9 @@ public enum CommonErrorCode implements SeaTunnelErrorCode { WRITE_SEATUNNEL_ROW_ERROR( "COMMON-23", " write SeaTunnelRow failed, the SeaTunnelRow value is ''."), - SQL_TEMPLATE_HANDLED_ERROR( "COMMON-24", "The table of has no , but the template \n