From 1aec71c9042123bf93b9ea34540519f6f32e9b43 Mon Sep 17 00:00:00 2001 From: wudi <676366545@qq.com> Date: Tue, 28 May 2024 20:43:53 +0800 Subject: [PATCH 1/5] Increase case coverage --- flink-doris-connector/pom.xml | 7 + .../doris/flink/catalog/DorisCatalog.java | 25 +- .../flink/catalog/DorisCatalogFactory.java | 3 +- .../flink/catalog/doris/DorisSystem.java | 17 - .../apache/doris/flink/cfg/DorisOptions.java | 9 - .../flink/exception/CreateTableException.java | 16 - .../exception/DorisBatchLoadException.java | 8 - .../flink/exception/DorisSystemException.java | 12 - .../flink/exception/StreamLoadException.java | 16 - .../apache/doris/flink/rest/RestService.java | 91 ----- .../doris/flink/rest/models/BackendRow.java | 73 ---- .../doris/flink/serialization/RowBatch.java | 9 +- .../doris/flink/sink/copy/BatchStageLoad.java | 6 + .../doris/flink/sink/copy/CopySQLBuilder.java | 3 +- .../flink/sink/copy/DorisCopyWriter.java | 6 + .../flink/table/DorisDynamicTableSource.java | 13 +- .../doris/flink/tools/cdc/CdcTools.java | 2 +- ...orisJsonDebeziumDeserializationSchema.java | 2 +- .../{ => serializer}/MongoDBDatabaseSync.java | 19 +- .../MongoDBJsonDebeziumSchemaSerializer.java | 6 +- .../MongoJsonDebeziumDataChange.java | 4 +- .../MongoJsonDebeziumSchemaChange.java | 4 +- .../tools/cdc/mysql/MysqlDatabaseSync.java | 2 +- .../tools/cdc/oracle/OracleDatabaseSync.java | 2 +- .../cdc/postgres/PostgresDatabaseSync.java | 2 +- .../cdc/sqlserver/SqlServerDatabaseSync.java | 2 +- .../org/apache/doris/flink/util/IOUtils.java | 49 --- .../org/apache/doris/flink/DorisTestBase.java | 8 + .../catalog/DorisCatalogFactoryTest.java | 70 ++++ ...CatalogTest.java => DorisCatalogTest.java} | 83 ++--- .../convert/DorisRowConverterTest.java | 4 + .../flink/rest/TestPartitionDefinition.java | 70 ++++ .../doris/flink/rest/TestRestService.java | 336 ++++++++++++++++++ .../doris/flink/rest/models/TestSchema.java} | 31 +- .../flink/serialization/TestRouting.java | 45 +++ .../flink/serialization/TestRowBatch.java | 23 ++ .../doris/flink/sink/DorisSinkITCase.java | 71 ++++ .../apache/doris/flink/sink/HttpTestUtil.java | 19 + .../flink/sink/batch/TestRecordWithMeta.java | 33 ++ .../copy/TestCopyCommittableSerializer.java | 36 ++ .../flink/sink/copy/TestDorisCopyWriter.java | 102 ++++++ .../writer/TestRecordWithMetaSerializer.java | 47 +++ .../doris/flink/source/DorisSourceITCase.java | 64 ++++ .../table/DorisDynamicTableSourceTest.java | 219 +++++++++++- .../tools/cdc/CdcMongoSyncDatabaseCase.java | 2 +- .../doris/flink/utils/FactoryMocks.java | 8 + 46 files changed, 1289 insertions(+), 390 deletions(-) delete mode 100644 flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/BackendRow.java rename flink-doris-connector/src/main/java/org/apache/doris/flink/{deserialization => tools/cdc/deserialize}/DorisJsonDebeziumDeserializationSchema.java (99%) rename flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/{ => serializer}/MongoDBDatabaseSync.java (98%) rename flink-doris-connector/src/main/java/org/apache/doris/flink/{sink/writer => tools/cdc/mongodb}/serializer/MongoDBJsonDebeziumSchemaSerializer.java (96%) rename flink-doris-connector/src/main/java/org/apache/doris/flink/{sink/writer/serializer/jsondebezium => tools/cdc/mongodb/serializer}/MongoJsonDebeziumDataChange.java (96%) rename flink-doris-connector/src/main/java/org/apache/doris/flink/{sink/writer/serializer/jsondebezium => tools/cdc/mongodb/serializer}/MongoJsonDebeziumSchemaChange.java (97%) delete mode 100644 flink-doris-connector/src/main/java/org/apache/doris/flink/util/IOUtils.java create mode 100644 flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogFactoryTest.java rename flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/{CatalogTest.java => DorisCatalogTest.java} (85%) create mode 100644 flink-doris-connector/src/test/java/org/apache/doris/flink/rest/TestPartitionDefinition.java create mode 100644 flink-doris-connector/src/test/java/org/apache/doris/flink/rest/TestRestService.java rename flink-doris-connector/src/{main/java/org/apache/doris/flink/rest/models/Backend.java => test/java/org/apache/doris/flink/rest/models/TestSchema.java} (62%) create mode 100644 flink-doris-connector/src/test/java/org/apache/doris/flink/serialization/TestRouting.java create mode 100644 flink-doris-connector/src/test/java/org/apache/doris/flink/sink/batch/TestRecordWithMeta.java create mode 100644 flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestCopyCommittableSerializer.java create mode 100644 flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestDorisCopyWriter.java create mode 100644 flink-doris-connector/src/test/java/org/apache/doris/flink/sink/writer/TestRecordWithMetaSerializer.java diff --git a/flink-doris-connector/pom.xml b/flink-doris-connector/pom.xml index db05e2e57..515dc67c5 100644 --- a/flink-doris-connector/pom.xml +++ b/flink-doris-connector/pom.xml @@ -341,6 +341,13 @@ under the License. ${testcontainers.version} test + + org.apache.flink + flink-table-common + ${flink.version} + test-jar + test + diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/DorisCatalog.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/DorisCatalog.java index a09e8fc3d..8968e7b5a 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/DorisCatalog.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/DorisCatalog.java @@ -17,6 +17,13 @@ package org.apache.doris.flink.catalog; +import org.apache.commons.compress.utils.Lists; +import org.apache.doris.flink.catalog.doris.DataModel; +import org.apache.doris.flink.catalog.doris.DorisSystem; +import org.apache.doris.flink.catalog.doris.FieldSchema; +import org.apache.doris.flink.catalog.doris.TableSchema; +import org.apache.doris.flink.cfg.DorisConnectionOptions; +import org.apache.doris.flink.table.DorisDynamicTableFactory; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.table.api.Schema; import org.apache.flink.table.catalog.AbstractCatalog; @@ -48,14 +55,6 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; - -import org.apache.commons.compress.utils.Lists; -import org.apache.doris.flink.catalog.doris.DataModel; -import org.apache.doris.flink.catalog.doris.DorisSystem; -import org.apache.doris.flink.catalog.doris.FieldSchema; -import org.apache.doris.flink.catalog.doris.TableSchema; -import org.apache.doris.flink.cfg.DorisConnectionOptions; -import org.apache.doris.flink.table.DorisDynamicTableFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -559,4 +558,14 @@ public void alterPartitionColumnStatistics( throws PartitionNotExistException, CatalogException { throw new UnsupportedOperationException(); } + + @VisibleForTesting + public DorisConnectionOptions getConnectionOptions() { + return connectionOptions; + } + + @VisibleForTesting + public Map getProperties() { + return properties; + } } diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/DorisCatalogFactory.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/DorisCatalogFactory.java index 06bbbc1dc..30fc8e5c3 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/DorisCatalogFactory.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/DorisCatalogFactory.java @@ -29,6 +29,7 @@ import java.util.Set; import static org.apache.doris.flink.catalog.DorisCatalogOptions.DEFAULT_DATABASE; +import static org.apache.doris.flink.catalog.DorisCatalogOptions.TABLE_PROPERTIES_PREFIX; import static org.apache.doris.flink.table.DorisConfigOptions.DORIS_BATCH_SIZE; import static org.apache.doris.flink.table.DorisConfigOptions.DORIS_DESERIALIZE_ARROW_ASYNC; import static org.apache.doris.flink.table.DorisConfigOptions.DORIS_DESERIALIZE_QUEUE_SIZE; @@ -112,7 +113,7 @@ public Set> optionalOptions() { public Catalog createCatalog(Context context) { final FactoryUtil.CatalogFactoryHelper helper = FactoryUtil.createCatalogFactoryHelper(this, context); - helper.validateExcept(STREAM_LOAD_PROP_PREFIX); + helper.validateExcept(STREAM_LOAD_PROP_PREFIX, TABLE_PROPERTIES_PREFIX); DorisConnectionOptions connectionOptions = new DorisConnectionOptions.DorisConnectionOptionsBuilder() diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/doris/DorisSystem.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/doris/DorisSystem.java index 3608e95d3..31d32e01f 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/doris/DorisSystem.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/doris/DorisSystem.java @@ -85,23 +85,6 @@ public boolean tableExists(String database, String table) { return databaseExists(database) && listTables(database).contains(table); } - public boolean columnExists(String database, String table, String columnName) { - if (tableExists(database, table)) { - List columns = - extractColumnValuesBySQL( - "SELECT COLUMN_NAME FROM information_schema.`COLUMNS` WHERE TABLE_SCHEMA = ? AND TABLE_NAME = ? AND COLUMN_NAME = ?", - 1, - null, - database, - table, - columnName); - if (columns != null && !columns.isEmpty()) { - return true; - } - } - return false; - } - public List listTables(String databaseName) { if (!databaseExists(databaseName)) { throw new DorisRuntimeException("database" + databaseName + " is not exists"); diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisOptions.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisOptions.java index f1dd2a575..7d42fb797 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisOptions.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisOptions.java @@ -17,10 +17,6 @@ package org.apache.doris.flink.cfg; -import org.apache.doris.flink.util.IOUtils; - -import java.util.Properties; - import static org.apache.flink.util.Preconditions.checkNotNull; /** Options for the Doris connector. */ @@ -65,11 +61,6 @@ public void setTableIdentifier(String tableIdentifier) { this.tableIdentifier = tableIdentifier; } - public String save() throws IllegalArgumentException { - Properties copy = new Properties(); - return IOUtils.propsToString(copy); - } - public static Builder builder() { return new Builder(); } diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/CreateTableException.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/CreateTableException.java index 380cbf6aa..202ab6e35 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/CreateTableException.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/CreateTableException.java @@ -26,20 +26,4 @@ public CreateTableException() { public CreateTableException(String message) { super(message); } - - public CreateTableException(String message, Throwable cause) { - super(message, cause); - } - - public CreateTableException(Throwable cause) { - super(cause); - } - - protected CreateTableException( - String message, - Throwable cause, - boolean enableSuppression, - boolean writableStackTrace) { - super(message, cause, enableSuppression, writableStackTrace); - } } diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisBatchLoadException.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisBatchLoadException.java index c2af5bcd3..2cb165656 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisBatchLoadException.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisBatchLoadException.java @@ -34,12 +34,4 @@ public DorisBatchLoadException(String message, Throwable cause) { public DorisBatchLoadException(Throwable cause) { super(cause); } - - protected DorisBatchLoadException( - String message, - Throwable cause, - boolean enableSuppression, - boolean writableStackTrace) { - super(message, cause, enableSuppression, writableStackTrace); - } } diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisSystemException.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisSystemException.java index bba1cd44d..4d69e1c62 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisSystemException.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisSystemException.java @@ -30,16 +30,4 @@ public DorisSystemException(String message) { public DorisSystemException(String message, Throwable cause) { super(message, cause); } - - public DorisSystemException(Throwable cause) { - super(cause); - } - - protected DorisSystemException( - String message, - Throwable cause, - boolean enableSuppression, - boolean writableStackTrace) { - super(message, cause, enableSuppression, writableStackTrace); - } } diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/StreamLoadException.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/StreamLoadException.java index 210d2bd59..30f51e0c4 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/StreamLoadException.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/StreamLoadException.java @@ -25,20 +25,4 @@ public StreamLoadException() { public StreamLoadException(String message) { super(message); } - - public StreamLoadException(String message, Throwable cause) { - super(message, cause); - } - - public StreamLoadException(Throwable cause) { - super(cause); - } - - protected StreamLoadException( - String message, - Throwable cause, - boolean enableSuppression, - boolean writableStackTrace) { - super(message, cause, enableSuppression, writableStackTrace); - } } diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/RestService.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/RestService.java index 000141c58..11134194c 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/RestService.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/RestService.java @@ -34,8 +34,6 @@ import org.apache.doris.flink.exception.DorisSchemaChangeException; import org.apache.doris.flink.exception.IllegalArgumentException; import org.apache.doris.flink.exception.ShouldNeverHappenException; -import org.apache.doris.flink.rest.models.Backend; -import org.apache.doris.flink.rest.models.BackendRow; import org.apache.doris.flink.rest.models.BackendV2; import org.apache.doris.flink.rest.models.BackendV2.BackendRowV2; import org.apache.doris.flink.rest.models.QueryPlan; @@ -307,95 +305,6 @@ static List allEndpoints(String feNodes, Logger logger) { return nodes; } - /** - * choice a Doris BE node to request. - * - * @param options configuration of request - * @param logger slf4j logger - * @return the chosen one Doris BE node - * @throws IllegalArgumentException BE nodes is illegal - */ - @VisibleForTesting - public static String randomBackend( - DorisOptions options, DorisReadOptions readOptions, Logger logger) - throws DorisException, IOException { - List backends = getBackendsV2(options, readOptions, logger); - logger.trace("Parse beNodes '{}'.", backends); - if (backends == null || backends.isEmpty()) { - logger.error(ILLEGAL_ARGUMENT_MESSAGE, "beNodes", backends); - throw new IllegalArgumentException("beNodes", String.valueOf(backends)); - } - Collections.shuffle(backends); - BackendRowV2 backend = backends.get(0); - return backend.getIp() + ":" + backend.getHttpPort(); - } - - public static String getBackend( - DorisOptions options, DorisReadOptions readOptions, Logger logger) - throws DorisRuntimeException { - try { - return randomBackend(options, readOptions, logger); - } catch (Exception e) { - throw new DorisRuntimeException("Failed to get backend via " + options.getFenodes(), e); - } - } - - /** - * get Doris BE nodes to request. - * - * @param options configuration of request - * @param logger slf4j logger - * @return the chosen one Doris BE node - * @throws IllegalArgumentException BE nodes is illegal. This method is deprecated. Because it - * needs ADMIN_PRIV to get backends, which is not suitable for common users. Use - * getBackendsV2 instead - */ - @Deprecated - @VisibleForTesting - static List getBackends( - DorisOptions options, DorisReadOptions readOptions, Logger logger) - throws DorisException, IOException { - String feNodes = options.getFenodes(); - String feNode = randomEndpoint(feNodes, logger); - String beUrl = "http://" + feNode + BACKENDS; - HttpGet httpGet = new HttpGet(beUrl); - String response = send(options, readOptions, httpGet, logger); - logger.info("Backend Info:{}", response); - List backends = parseBackend(response, logger); - return backends; - } - - @Deprecated - static List parseBackend(String response, Logger logger) - throws DorisException, IOException { - ObjectMapper mapper = new ObjectMapper(); - Backend backend; - try { - backend = mapper.readValue(response, Backend.class); - } catch (JsonParseException e) { - String errMsg = "Doris BE's response is not a json. res: " + response; - logger.error(errMsg, e); - throw new DorisException(errMsg, e); - } catch (JsonMappingException e) { - String errMsg = "Doris BE's response cannot map to schema. res: " + response; - logger.error(errMsg, e); - throw new DorisException(errMsg, e); - } catch (IOException e) { - String errMsg = "Parse Doris BE's response to json failed. res: " + response; - logger.error(errMsg, e); - throw new DorisException(errMsg, e); - } - - if (backend == null) { - logger.error(SHOULD_NOT_HAPPEN_MESSAGE); - throw new ShouldNeverHappenException(); - } - List backendRows = - backend.getRows().stream().filter(v -> v.getAlive()).collect(Collectors.toList()); - logger.debug("Parsing schema result is '{}'.", backendRows); - return backendRows; - } - /** * get Doris BE nodes to request. * diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/BackendRow.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/BackendRow.java deleted file mode 100644 index 9f1f24e56..000000000 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/BackendRow.java +++ /dev/null @@ -1,73 +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.doris.flink.rest.models; - -import com.fasterxml.jackson.annotation.JsonIgnoreProperties; -import com.fasterxml.jackson.annotation.JsonProperty; - -@Deprecated -@JsonIgnoreProperties(ignoreUnknown = true) -public class BackendRow { - - @JsonProperty(value = "HttpPort") - private String httpPort; - - @JsonProperty(value = "IP") - private String ip; - - @JsonProperty(value = "Alive") - private Boolean alive; - - public String getHttpPort() { - return httpPort; - } - - public void setHttpPort(String httpPort) { - this.httpPort = httpPort; - } - - public String getIp() { - return ip; - } - - public void setIp(String ip) { - this.ip = ip; - } - - public Boolean getAlive() { - return alive; - } - - public void setAlive(Boolean alive) { - this.alive = alive; - } - - @Override - public String toString() { - return "BackendRow{" - + "HttpPort='" - + httpPort - + '\'' - + ", IP='" - + ip - + '\'' - + ", Alive=" - + alive - + '}'; - } -} diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/serialization/RowBatch.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/serialization/RowBatch.java index b4f35ab40..a7b58df52 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/serialization/RowBatch.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/serialization/RowBatch.java @@ -17,6 +17,7 @@ package org.apache.doris.flink.serialization; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.util.Preconditions; import org.apache.arrow.memory.RootAllocator; @@ -488,10 +489,16 @@ private LocalDateTime getDateTime(int rowIndex, FieldVector fieldVector) { return dateTime; } - private String completeMilliseconds(String stringValue) { + @VisibleForTesting + public static String completeMilliseconds(String stringValue) { if (stringValue.length() == DATETIMEV2_PATTERN.length()) { return stringValue; } + + if (stringValue.length() < DATETIME_PATTERN.length()) { + return stringValue; + } + StringBuilder sb = new StringBuilder(stringValue); if (stringValue.length() == DATETIME_PATTERN.length()) { sb.append("."); diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/copy/BatchStageLoad.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/copy/BatchStageLoad.java index c192d76f4..0080dee17 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/copy/BatchStageLoad.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/copy/BatchStageLoad.java @@ -17,6 +17,7 @@ package org.apache.doris.flink.sink.copy; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.util.Preconditions; import org.apache.doris.flink.cfg.DorisExecutionOptions; @@ -421,4 +422,9 @@ public Thread newThread(Runnable r) { return t; } } + + @VisibleForTesting + public void setHttpClientBuilder(HttpClientBuilder httpClientBuilder) { + this.httpClientBuilder = httpClientBuilder; + } } diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/copy/CopySQLBuilder.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/copy/CopySQLBuilder.java index 1fef84683..33843a064 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/copy/CopySQLBuilder.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/copy/CopySQLBuilder.java @@ -17,6 +17,7 @@ package org.apache.doris.flink.sink.copy; +import org.apache.doris.flink.catalog.doris.DorisSystem; import org.apache.doris.flink.cfg.DorisExecutionOptions; import java.util.Arrays; @@ -52,7 +53,7 @@ public CopySQLBuilder( public String buildCopySQL() { StringBuilder sb = new StringBuilder(); sb.append("COPY INTO ") - .append(tableIdentifier) + .append(DorisSystem.quoteTableIdentifier(tableIdentifier)) .append(" FROM @~('{") .append(String.join(",", fileList)) .append("}') ") diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/copy/DorisCopyWriter.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/copy/DorisCopyWriter.java index 1b3a08997..469b3f2d2 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/copy/DorisCopyWriter.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/copy/DorisCopyWriter.java @@ -17,6 +17,7 @@ package org.apache.doris.flink.sink.copy; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; import org.apache.flink.util.Preconditions; @@ -194,4 +195,9 @@ private void checkFlushException() { throw new RuntimeException("Writing records to streamload failed.", flushException); } } + + @VisibleForTesting + public void setBatchStageLoad(BatchStageLoad batchStageLoad) { + this.batchStageLoad = batchStageLoad; + } } diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSource.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSource.java index e2b837c32..7cb605876 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSource.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSource.java @@ -17,6 +17,7 @@ package org.apache.doris.flink.table; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.Projection; @@ -79,13 +80,6 @@ public DorisDynamicTableSource( this.physicalRowDataType = physicalRowDataType; } - public DorisDynamicTableSource( - DorisOptions options, DorisReadOptions readOptions, TableSchema physicalSchema) { - this.options = options; - this.readOptions = readOptions; - this.physicalSchema = physicalSchema; - } - @Override public ChangelogMode getChangelogMode() { // in our example the format decides about the changelog mode @@ -213,4 +207,9 @@ public void applyProjection(int[][] projectedFields, DataType producedDataType) .collect(Collectors.joining(", "))); } } + + @VisibleForTesting + public List getResolvedFilterQuery() { + return resolvedFilterQuery; + } } diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/CdcTools.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/CdcTools.java index 194ef8753..8f8cabf54 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/CdcTools.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/CdcTools.java @@ -23,7 +23,7 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; -import org.apache.doris.flink.tools.cdc.mongodb.MongoDBDatabaseSync; +import org.apache.doris.flink.tools.cdc.mongodb.serializer.MongoDBDatabaseSync; import org.apache.doris.flink.tools.cdc.mysql.MysqlDatabaseSync; import org.apache.doris.flink.tools.cdc.oracle.OracleDatabaseSync; import org.apache.doris.flink.tools.cdc.postgres.PostgresDatabaseSync; diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/deserialization/DorisJsonDebeziumDeserializationSchema.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/deserialize/DorisJsonDebeziumDeserializationSchema.java similarity index 99% rename from flink-doris-connector/src/main/java/org/apache/doris/flink/deserialization/DorisJsonDebeziumDeserializationSchema.java rename to flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/deserialize/DorisJsonDebeziumDeserializationSchema.java index 729ba7d86..d1d915453 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/deserialization/DorisJsonDebeziumDeserializationSchema.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/deserialize/DorisJsonDebeziumDeserializationSchema.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.flink.deserialization; +package org.apache.doris.flink.tools.cdc.deserialize; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/MongoDBDatabaseSync.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoDBDatabaseSync.java similarity index 98% rename from flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/MongoDBDatabaseSync.java rename to flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoDBDatabaseSync.java index e2e002393..bb6bd2310 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/MongoDBDatabaseSync.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoDBDatabaseSync.java @@ -15,13 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.flink.tools.cdc.mongodb; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +package org.apache.doris.flink.tools.cdc.mongodb.serializer; import com.mongodb.ConnectionString; import com.mongodb.MongoClientSettings; @@ -42,20 +36,25 @@ import org.apache.doris.flink.cfg.DorisExecutionOptions; import org.apache.doris.flink.cfg.DorisOptions; import org.apache.doris.flink.sink.writer.serializer.DorisRecordSerializer; -import org.apache.doris.flink.sink.writer.serializer.MongoDBJsonDebeziumSchemaSerializer; import org.apache.doris.flink.tools.cdc.DatabaseSync; import org.apache.doris.flink.tools.cdc.ParsingProcessFunction; import org.apache.doris.flink.tools.cdc.SourceSchema; +import org.apache.doris.flink.tools.cdc.mongodb.MongoDBSchema; +import org.apache.doris.flink.tools.cdc.mongodb.MongoParsingProcessFunction; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.bson.Document; -import javax.annotation.Nullable; - import java.sql.Connection; import java.sql.SQLException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import javax.annotation.Nullable; import static com.ververica.cdc.connectors.mongodb.internal.MongoDBEnvelope.encodeValue; import static org.apache.flink.util.Preconditions.checkNotNull; diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/serializer/MongoDBJsonDebeziumSchemaSerializer.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoDBJsonDebeziumSchemaSerializer.java similarity index 96% rename from flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/serializer/MongoDBJsonDebeziumSchemaSerializer.java rename to flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoDBJsonDebeziumSchemaSerializer.java index b2ff5988d..5c9c1b9e9 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/serializer/MongoDBJsonDebeziumSchemaSerializer.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoDBJsonDebeziumSchemaSerializer.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.flink.sink.writer.serializer; +package org.apache.doris.flink.tools.cdc.mongodb.serializer; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.JsonNode; @@ -23,11 +23,11 @@ import com.fasterxml.jackson.databind.node.JsonNodeFactory; import org.apache.doris.flink.cfg.DorisExecutionOptions; import org.apache.doris.flink.cfg.DorisOptions; +import org.apache.doris.flink.sink.writer.serializer.DorisRecord; +import org.apache.doris.flink.sink.writer.serializer.DorisRecordSerializer; import org.apache.doris.flink.sink.writer.serializer.jsondebezium.CdcDataChange; import org.apache.doris.flink.sink.writer.serializer.jsondebezium.CdcSchemaChange; import org.apache.doris.flink.sink.writer.serializer.jsondebezium.JsonDebeziumChangeContext; -import org.apache.doris.flink.sink.writer.serializer.jsondebezium.MongoJsonDebeziumDataChange; -import org.apache.doris.flink.sink.writer.serializer.jsondebezium.MongoJsonDebeziumSchemaChange; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/serializer/jsondebezium/MongoJsonDebeziumDataChange.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoJsonDebeziumDataChange.java similarity index 96% rename from flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/serializer/jsondebezium/MongoJsonDebeziumDataChange.java rename to flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoJsonDebeziumDataChange.java index 407a7e7a0..e448dbc57 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/serializer/jsondebezium/MongoJsonDebeziumDataChange.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoJsonDebeziumDataChange.java @@ -15,8 +15,10 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.flink.sink.writer.serializer.jsondebezium; +package org.apache.doris.flink.tools.cdc.mongodb.serializer; +import org.apache.doris.flink.sink.writer.serializer.jsondebezium.CdcDataChange; +import org.apache.doris.flink.sink.writer.serializer.jsondebezium.JsonDebeziumChangeContext; import org.apache.flink.util.StringUtils; import com.fasterxml.jackson.core.JsonProcessingException; diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/serializer/jsondebezium/MongoJsonDebeziumSchemaChange.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoJsonDebeziumSchemaChange.java similarity index 97% rename from flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/serializer/jsondebezium/MongoJsonDebeziumSchemaChange.java rename to flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoJsonDebeziumSchemaChange.java index 69dbf0b16..01eebd45e 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/writer/serializer/jsondebezium/MongoJsonDebeziumSchemaChange.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoJsonDebeziumSchemaChange.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.flink.sink.writer.serializer.jsondebezium; +package org.apache.doris.flink.tools.cdc.mongodb.serializer; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; @@ -27,6 +27,8 @@ import org.apache.doris.flink.exception.DorisRuntimeException; import org.apache.doris.flink.exception.IllegalArgumentException; import org.apache.doris.flink.sink.schema.SchemaChangeManager; +import org.apache.doris.flink.sink.writer.serializer.jsondebezium.CdcSchemaChange; +import org.apache.doris.flink.sink.writer.serializer.jsondebezium.JsonDebeziumChangeContext; import org.apache.doris.flink.tools.cdc.SourceSchema; import org.apache.doris.flink.tools.cdc.mongodb.MongoDBType; import org.apache.doris.flink.tools.cdc.mongodb.MongoDateConverter; diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mysql/MysqlDatabaseSync.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mysql/MysqlDatabaseSync.java index 8da57f6e8..03d3d0767 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mysql/MysqlDatabaseSync.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mysql/MysqlDatabaseSync.java @@ -35,9 +35,9 @@ import com.ververica.cdc.debezium.JsonDebeziumDeserializationSchema; import com.ververica.cdc.debezium.table.DebeziumOptions; import org.apache.doris.flink.catalog.doris.DataModel; -import org.apache.doris.flink.deserialization.DorisJsonDebeziumDeserializationSchema; import org.apache.doris.flink.tools.cdc.DatabaseSync; import org.apache.doris.flink.tools.cdc.SourceSchema; +import org.apache.doris.flink.tools.cdc.deserialize.DorisJsonDebeziumDeserializationSchema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/oracle/OracleDatabaseSync.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/oracle/OracleDatabaseSync.java index 89214fd3a..360351e41 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/oracle/OracleDatabaseSync.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/oracle/OracleDatabaseSync.java @@ -33,9 +33,9 @@ import com.ververica.cdc.debezium.JsonDebeziumDeserializationSchema; import com.ververica.cdc.debezium.table.DebeziumOptions; import org.apache.doris.flink.catalog.doris.DataModel; -import org.apache.doris.flink.deserialization.DorisJsonDebeziumDeserializationSchema; import org.apache.doris.flink.tools.cdc.DatabaseSync; import org.apache.doris.flink.tools.cdc.SourceSchema; +import org.apache.doris.flink.tools.cdc.deserialize.DorisJsonDebeziumDeserializationSchema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/postgres/PostgresDatabaseSync.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/postgres/PostgresDatabaseSync.java index c9387c120..74390325b 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/postgres/PostgresDatabaseSync.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/postgres/PostgresDatabaseSync.java @@ -33,9 +33,9 @@ import com.ververica.cdc.debezium.JsonDebeziumDeserializationSchema; import com.ververica.cdc.debezium.table.DebeziumOptions; import org.apache.doris.flink.catalog.doris.DataModel; -import org.apache.doris.flink.deserialization.DorisJsonDebeziumDeserializationSchema; import org.apache.doris.flink.tools.cdc.DatabaseSync; import org.apache.doris.flink.tools.cdc.SourceSchema; +import org.apache.doris.flink.tools.cdc.deserialize.DorisJsonDebeziumDeserializationSchema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/sqlserver/SqlServerDatabaseSync.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/sqlserver/SqlServerDatabaseSync.java index 9f286ff32..3f674fb2d 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/sqlserver/SqlServerDatabaseSync.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/sqlserver/SqlServerDatabaseSync.java @@ -34,9 +34,9 @@ import com.ververica.cdc.debezium.JsonDebeziumDeserializationSchema; import com.ververica.cdc.debezium.table.DebeziumOptions; import org.apache.doris.flink.catalog.doris.DataModel; -import org.apache.doris.flink.deserialization.DorisJsonDebeziumDeserializationSchema; import org.apache.doris.flink.tools.cdc.DatabaseSync; import org.apache.doris.flink.tools.cdc.SourceSchema; +import org.apache.doris.flink.tools.cdc.deserialize.DorisJsonDebeziumDeserializationSchema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/util/IOUtils.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/util/IOUtils.java deleted file mode 100644 index 203010cbe..000000000 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/util/IOUtils.java +++ /dev/null @@ -1,49 +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.doris.flink.util; - -import java.io.IOException; -import java.io.StringReader; -import java.io.StringWriter; -import java.util.Properties; - -public class IOUtils { - public static String propsToString(Properties props) throws IllegalArgumentException { - StringWriter sw = new StringWriter(); - if (props != null) { - try { - props.store(sw, ""); - } catch (IOException ex) { - throw new IllegalArgumentException("Cannot parse props to String.", ex); - } - } - return sw.toString(); - } - - public static Properties propsFromString(String source) throws IllegalArgumentException { - Properties copy = new Properties(); - if (source != null) { - try { - copy.load(new StringReader(source)); - } catch (IOException ex) { - throw new IllegalArgumentException("Cannot parse props from String.", ex); - } - } - return copy; - } -} diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisTestBase.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisTestBase.java index 8141eafbf..8efa117e4 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisTestBase.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisTestBase.java @@ -65,6 +65,14 @@ protected static String getFenodes() { return DORIS_CONTAINER.getHost() + ":8030"; } + protected static String getJdbcUrl() { + return String.format(URL, DORIS_CONTAINER.getHost()); + } + + protected static String getHost() { + return DORIS_CONTAINER.getHost(); + } + static { startContainers(); } diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogFactoryTest.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogFactoryTest.java new file mode 100644 index 000000000..d45984541 --- /dev/null +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogFactoryTest.java @@ -0,0 +1,70 @@ +// 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.doris.flink.catalog; + +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.factories.FactoryUtil; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class DorisCatalogFactoryTest { + + @Test + public void testCreateCatalog() { + final Map options = new HashMap<>(); + options.put("type", "doris"); + options.put("jdbc-url", "jdbc:mysql://127.0.0.1:9030"); + options.put("fenodes", "127.0.0.1:8030"); + options.put("default-database", "doris_db"); + options.put("username", "root"); + options.put("password", ""); + options.put("doris.request.query.timeout", "3600s"); + options.put("sink.enable-2pc", "false"); + options.put("sink.properties.format", "json"); + options.put("sink.properties.read_json_by_line", "true"); + options.put("table.properties.replication_num", "1"); + + Catalog catalog = FactoryUtil.createCatalog( + "test_catalog", + options, + null, + Thread.currentThread().getContextClassLoader() + ); + assertTrue(catalog instanceof DorisCatalog); + DorisCatalog dorisCatalog = (DorisCatalog) catalog; + assertEquals("test_catalog", dorisCatalog.getName()); + assertEquals("doris_db", dorisCatalog.getDefaultDatabase()); + assertEquals("jdbc:mysql://127.0.0.1:9030", dorisCatalog.getConnectionOptions().getJdbcUrl()); + assertEquals("127.0.0.1:8030", dorisCatalog.getConnectionOptions().getFenodes()); + assertEquals("root", dorisCatalog.getConnectionOptions().getUsername()); + assertEquals("", dorisCatalog.getConnectionOptions().getPassword()); + + Map properties = dorisCatalog.getProperties(); + assertEquals(10, properties.size()); + assertEquals("3600s", properties.get("doris.request.query.timeout")); + assertEquals("false", properties.get("sink.enable-2pc")); + assertEquals("json", properties.get("sink.properties.format")); + assertEquals("true", properties.get("sink.properties.read_json_by_line")); + assertEquals("1", properties.get("table.properties.replication_num")); + } +} diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/CatalogTest.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogTest.java similarity index 85% rename from flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/CatalogTest.java rename to flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogTest.java index e93dfdd4b..f4995fe1c 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/CatalogTest.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogTest.java @@ -28,6 +28,7 @@ import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTableImpl; import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; import org.apache.flink.table.catalog.exceptions.TableNotExistException; @@ -38,6 +39,7 @@ import org.apache.flink.util.CollectionUtil; import com.google.common.collect.Lists; +import org.apache.doris.flink.DorisTestBase; import org.apache.doris.flink.cfg.DorisConnectionOptions; import org.junit.Before; import org.junit.Ignore; @@ -58,14 +60,17 @@ import static org.junit.Assert.assertTrue; /** Class for unit tests to run on catalogs. */ -@Ignore -public class CatalogTest { +public class DorisCatalogTest extends DorisTestBase { private static final String TEST_CATALOG_NAME = "doris_catalog"; - private static final String TEST_FENODES = "127.0.0.1:8030"; - private static final String TEST_JDBCURL = "jdbc:mysql://127.0.0.1:9030"; - private static final String TEST_USERNAME = "root"; - private static final String TEST_PWD = ""; - private static final String TEST_DB = "db1"; + private static final String TEST_FENODES = getFenodes(); + private static final String TEST_JDBCURL = getJdbcUrl(); + private static final String TEST_USERNAME = USERNAME; + private static final String TEST_PWD = PASSWORD; + // private static final String TEST_FENODES = "127.0.0.1:8030"; + // private static final String TEST_JDBCURL = "jdbc:mysql://127.0.0.1:9030"; + // private static final String TEST_USERNAME = "root"; + // private static final String TEST_PWD = ""; + private static final String TEST_DB = "catalog_db"; private static final String TEST_TABLE = "t_all_types"; private static final String TEST_TABLE_SINK = "t_all_types_sink"; private static final String TEST_TABLE_SINK_GROUPBY = "t_all_types_sink_groupby"; @@ -146,7 +151,9 @@ public class CatalogTest { private TableEnvironment tEnv; @Before - public void setup() { + public void setup() + throws DatabaseAlreadyExistException, TableAlreadyExistException, + TableNotExistException, DatabaseNotExistException { DorisConnectionOptions connectionOptions = new DorisConnectionOptions.DorisConnectionOptionsBuilder() .withFenodes(TEST_FENODES) @@ -163,43 +170,45 @@ public void setup() { // Use doris catalog. tEnv.registerCatalog(TEST_CATALOG_NAME, catalog); tEnv.useCatalog(TEST_CATALOG_NAME); + + catalog.createDatabase(TEST_DB, createDb(), true); + catalog.createTable(new ObjectPath(TEST_DB, TEST_TABLE), createTable(), true); + catalog.createTable(new ObjectPath(TEST_DB, TEST_TABLE_SINK), createTable(), true); + catalog.createTable(new ObjectPath(TEST_DB, TEST_TABLE_SINK_GROUPBY), createTable(), true); } @Test + @Ignore public void testQueryFenodes() { String actual = catalog.queryFenodes(); - assertEquals("127.0.0.1:8030", actual); + assertEquals(getFenodes(), actual); } @Test public void testListDatabases() { List actual = catalog.listDatabases(); - assertEquals(Collections.singletonList(TEST_DB), actual); + assertTrue(actual.contains(TEST_DB)); } @Test - public void testDbExists() throws Exception { + public void testCreateDb() throws Exception { + catalog.createDatabase("test_create", createDb(), true); + assertTrue(catalog.databaseExists("test_create")); + String databaseNotExist = "nonexistent"; assertFalse(catalog.databaseExists(databaseNotExist)); - assertTrue(catalog.databaseExists(TEST_DB)); - } - @Test - public void testCreateDb() throws Exception { - catalog.createDatabase("db1", createDb(), true); - assertTrue(catalog.databaseExists("db1")); - } - - @Test - public void testDropDb() throws Exception { - catalog.dropDatabase("db1", false); - assertFalse(catalog.databaseExists("db1")); + catalog.dropDatabase("test_create", false); + assertFalse(catalog.databaseExists("test_create")); } @Test public void testListTables() throws DatabaseNotExistException { List actual = catalog.listTables(TEST_DB); - assertEquals(Arrays.asList(TEST_TABLE, TEST_TABLE_SINK, TEST_TABLE_SINK_GROUPBY), actual); + Collections.sort(actual); + List excepted = Arrays.asList(TEST_TABLE, TEST_TABLE_SINK, TEST_TABLE_SINK_GROUPBY); + Collections.sort(excepted); + assertEquals(excepted, actual); } @Test @@ -209,26 +218,14 @@ public void testTableExists() { } @Test - @Ignore public void testGetTable() throws TableNotExistException { // todo: string varchar mapping CatalogBaseTable table = catalog.getTable(new ObjectPath(TEST_DB, TEST_TABLE)); - System.out.println(table); - assertEquals(TABLE_SCHEMA, table.getUnresolvedSchema()); - } - - @Test - @Ignore - public void testCreateTable() - throws TableNotExistException, TableAlreadyExistException, DatabaseNotExistException { - // todo: Record primary key not null information - ObjectPath tablePath = new ObjectPath(TEST_DB, TEST_TABLE); - catalog.dropTable(tablePath, true); - catalog.createTable(tablePath, createTable(), true); - CatalogBaseTable tableGet = catalog.getTable(tablePath); - System.out.println(tableGet.getUnresolvedSchema()); + Schema actual = table.getUnresolvedSchema(); System.out.println(TABLE_SCHEMA_1); - assertEquals(TABLE_SCHEMA_1, tableGet.getUnresolvedSchema()); + assertEquals( + TABLE_SCHEMA_1.getFieldNames(), + actual.getColumns().stream().map(Schema.UnresolvedColumn::getName).toArray()); } @Test @@ -240,6 +237,7 @@ public void testDropTable() throws TableNotExistException { // ------ test select query. ------ @Test + @Ignore public void testSelectField() { List results = CollectionUtil.iteratorToList( @@ -253,6 +251,7 @@ public void testSelectField() { } @Test + @Ignore public void testWithoutCatalogDB() { List results = CollectionUtil.iteratorToList( @@ -263,6 +262,7 @@ public void testWithoutCatalogDB() { } @Test + @Ignore public void testWithoutCatalog() { List results = CollectionUtil.iteratorToList( @@ -273,6 +273,7 @@ public void testWithoutCatalog() { } @Test + @Ignore public void testFullPath() { List results = CollectionUtil.iteratorToList( @@ -288,6 +289,7 @@ public void testFullPath() { } @Test + @Ignore public void testSelectToInsert() throws Exception { String sql = @@ -303,6 +305,7 @@ public void testSelectToInsert() throws Exception { } @Test + @Ignore public void testGroupByInsert() throws Exception { // Changes primary key for the next record. tEnv.executeSql( diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/deserialization/convert/DorisRowConverterTest.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/deserialization/convert/DorisRowConverterTest.java index c9016c540..d9df394f1 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/deserialization/convert/DorisRowConverterTest.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/deserialization/convert/DorisRowConverterTest.java @@ -99,6 +99,10 @@ public void testConvert() throws IOException { timestamp1, timestamp2); GenericRowData rowData = converter.convertInternal(record); + DorisRowConverter converterWithDataType = + new DorisRowConverter(schema.getColumnDataTypes().toArray(new DataType[0])); + GenericRowData genericRowData = converterWithDataType.convertInternal(record); + Assert.assertEquals(rowData, genericRowData); RowDataSerializer serializer = new Builder() diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/rest/TestPartitionDefinition.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/rest/TestPartitionDefinition.java new file mode 100644 index 000000000..8f520504b --- /dev/null +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/rest/TestPartitionDefinition.java @@ -0,0 +1,70 @@ +// 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.doris.flink.rest; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashSet; +import java.util.Set; + +public class TestPartitionDefinition { + private static final String DATABASE_1 = "database1"; + private static final String TABLE_1 = "table1"; + private static final String BE_1 = "be1"; + private static final String QUERY_PLAN_1 = "queryPlan1"; + private static final long TABLET_ID_1 = 1L; + + private static final String DATABASE_2 = "database2"; + private static final String TABLE_2 = "table2"; + private static final String BE_2 = "be2"; + private static final String QUERY_PLAN_2 = "queryPlan2"; + private static final long TABLET_ID_2 = 2L; + + @Test + public void testCompare() throws Exception { + Set tabletSet1 = new HashSet<>(); + tabletSet1.add(TABLET_ID_1); + Set tabletSet2 = new HashSet<>(); + tabletSet2.add(TABLET_ID_2); + Set tabletSet3 = new HashSet<>(); + tabletSet3.add(TABLET_ID_1); + tabletSet3.add(TABLET_ID_2); + + PartitionDefinition pd1 = + new PartitionDefinition(DATABASE_1, TABLE_1, BE_1, tabletSet1, QUERY_PLAN_1); + PartitionDefinition pd3 = + new PartitionDefinition(DATABASE_2, TABLE_1, BE_1, tabletSet1, QUERY_PLAN_1); + PartitionDefinition pd4 = + new PartitionDefinition(DATABASE_1, TABLE_2, BE_1, tabletSet1, QUERY_PLAN_1); + PartitionDefinition pd5 = + new PartitionDefinition(DATABASE_1, TABLE_1, BE_2, tabletSet1, QUERY_PLAN_1); + PartitionDefinition pd6 = + new PartitionDefinition(DATABASE_1, TABLE_1, BE_1, tabletSet2, QUERY_PLAN_1); + PartitionDefinition pd7 = + new PartitionDefinition(DATABASE_1, TABLE_1, BE_1, tabletSet3, QUERY_PLAN_1); + PartitionDefinition pd8 = + new PartitionDefinition(DATABASE_1, TABLE_1, BE_1, tabletSet1, QUERY_PLAN_2); + Assert.assertTrue(pd1.compareTo(pd3) < 0); + Assert.assertTrue(pd1.compareTo(pd4) < 0); + Assert.assertTrue(pd1.compareTo(pd5) < 0); + Assert.assertTrue(pd1.compareTo(pd6) < 0); + Assert.assertTrue(pd1.compareTo(pd7) < 0); + Assert.assertTrue(pd1.compareTo(pd8) < 0); + } +} diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/rest/TestRestService.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/rest/TestRestService.java new file mode 100644 index 000000000..7c49429ee --- /dev/null +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/rest/TestRestService.java @@ -0,0 +1,336 @@ +// 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.doris.flink.rest; + +import org.apache.doris.flink.cfg.DorisOptions; +import org.apache.doris.flink.cfg.DorisReadOptions; +import org.apache.doris.flink.exception.DorisException; +import org.apache.doris.flink.exception.IllegalArgumentException; +import org.apache.doris.flink.rest.models.BackendV2; +import org.apache.doris.flink.rest.models.Field; +import org.apache.doris.flink.rest.models.QueryPlan; +import org.apache.doris.flink.rest.models.Schema; +import org.apache.doris.flink.rest.models.Tablet; +import org.apache.doris.flink.sink.BackendUtil; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.mockito.MockedStatic; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +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.Set; + +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_TABLET_SIZE_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_TABLET_SIZE_MIN; +import static org.hamcrest.core.StringStartsWith.startsWith; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mockStatic; + +public class TestRestService { + private static final Logger logger = LoggerFactory.getLogger(TestRestService.class); + + @Rule public ExpectedException thrown = ExpectedException.none(); + + static MockedStatic backendUtilMockedStatic; + + @Before + public void setUp() throws Exception { + backendUtilMockedStatic = mockStatic(BackendUtil.class); + backendUtilMockedStatic.when(() -> BackendUtil.tryHttpConnection(any())).thenReturn(true); + } + + @After + public void after() { + backendUtilMockedStatic.close(); + } + + @Test + public void testParseIdentifier() throws Exception { + String validIdentifier = "a.b"; + String[] names = RestService.parseIdentifier(validIdentifier, logger); + Assert.assertEquals(2, names.length); + Assert.assertEquals("a", names[0]); + Assert.assertEquals("b", names[1]); + + String invalidIdentifier1 = "a"; + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "argument 'table.identifier' is illegal, value is '" + invalidIdentifier1 + "'."); + RestService.parseIdentifier(invalidIdentifier1, logger); + + String invalidIdentifier3 = "a.b.c"; + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "argument 'table.identifier' is illegal, value is '" + invalidIdentifier3 + "'."); + RestService.parseIdentifier(invalidIdentifier3, logger); + + String emptyIdentifier = ""; + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "argument 'table.identifier' is illegal, value is '" + emptyIdentifier + "'."); + RestService.parseIdentifier(emptyIdentifier, logger); + + String nullIdentifier = null; + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "argument 'table.identifier' is illegal, value is '" + nullIdentifier + "'."); + RestService.parseIdentifier(nullIdentifier, logger); + } + + @Test + public void testChoiceFe() throws Exception { + String validFes = "1,2,3"; + String fe = RestService.randomEndpoint(validFes, logger); + List feNodes = new ArrayList<>(3); + feNodes.add("1"); + feNodes.add("2"); + feNodes.add("3"); + Assert.assertTrue(feNodes.contains(fe)); + + String emptyFes = ""; + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("argument 'fenodes' is illegal, value is '" + emptyFes + "'."); + RestService.randomEndpoint(emptyFes, logger); + + String nullFes = null; + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("argument 'fenodes' is illegal, value is '" + nullFes + "'."); + RestService.randomEndpoint(nullFes, logger); + } + + @Test + public void testFeResponseToSchema() throws Exception { + String res = + "{\"properties\":[{\"type\":\"TINYINT\",\"name\":\"k1\",\"comment\":\"\",\"aggregation_type\":\"\"},{\"name\":\"k5\"," + + "\"scale\":\"0\",\"comment\":\"\",\"type\":\"DECIMALV2\",\"precision\":\"9\",\"aggregation_type\":\"\"}],\"status\":200}"; + Schema expected = new Schema(); + expected.setStatus(200); + Field k1 = new Field("k1", "TINYINT", "", 0, 0, ""); + Field k5 = new Field("k5", "DECIMALV2", "", 9, 0, ""); + expected.put(k1); + expected.put(k5); + Assert.assertEquals(expected, RestService.parseSchema(res, logger)); + + String notJsonRes = "not json"; + thrown.expect(DorisException.class); + thrown.expectMessage(startsWith("Doris FE's response is not a json. res:")); + RestService.parseSchema(notJsonRes, logger); + + String notSchemaRes = + "{\"property\":[{\"type\":\"TINYINT\",\"name\":\"k1\",\"comment\":\"\"}," + + "{\"name\":\"k5\",\"scale\":\"0\",\"comment\":\"\",\"type\":\"DECIMALV2\",\"precision\":\"9\"}]," + + "\"status\":200}"; + thrown.expect(DorisException.class); + thrown.expectMessage(startsWith("Doris FE's response cannot map to schema. res: ")); + RestService.parseSchema(notSchemaRes, logger); + + String notOkRes = + "{\"properties\":[{\"type\":\"TINYINT\",\"name\":\"k1\",\"comment\":\"\"},{\"name\":\"k5\"," + + "\"scale\":\"0\",\"comment\":\"\",\"type\":\"DECIMALV2\",\"precision\":\"9\"}],\"status\":20}"; + thrown.expect(DorisException.class); + thrown.expectMessage(startsWith("Doris FE's response is not OK, status is ")); + RestService.parseSchema(notOkRes, logger); + } + + @Test + public void testFeResponseToQueryPlan() throws Exception { + String res = + "{\"partitions\":{" + + "\"11017\":{\"routings\":[\"be1\",\"be2\"],\"version\":3,\"versionHash\":1,\"schemaHash\":1}," + + "\"11019\":{\"routings\":[\"be3\",\"be4\"],\"version\":3,\"versionHash\":1,\"schemaHash\":1}}," + + "\"opaqued_query_plan\":\"query_plan\",\"status\":200}"; + + List routings11017 = new ArrayList<>(2); + routings11017.add("be1"); + routings11017.add("be2"); + + Tablet tablet11017 = new Tablet(); + tablet11017.setSchemaHash(1); + tablet11017.setVersionHash(1); + tablet11017.setVersion(3); + tablet11017.setRoutings(routings11017); + + List routings11019 = new ArrayList<>(2); + routings11019.add("be3"); + routings11019.add("be4"); + + Tablet tablet11019 = new Tablet(); + tablet11019.setSchemaHash(1); + tablet11019.setVersionHash(1); + tablet11019.setVersion(3); + tablet11019.setRoutings(routings11019); + + Map partitions = new LinkedHashMap<>(); + partitions.put("11017", tablet11017); + partitions.put("11019", tablet11019); + + QueryPlan expected = new QueryPlan(); + expected.setPartitions(partitions); + expected.setStatus(200); + expected.setOpaquedQueryPlan("query_plan"); + + QueryPlan actual = RestService.getQueryPlan(res, logger); + Assert.assertEquals(expected, actual); + + String notJsonRes = "not json"; + thrown.expect(DorisException.class); + thrown.expectMessage(startsWith("Doris FE's response is not a json. res:")); + RestService.parseSchema(notJsonRes, logger); + + String notQueryPlanRes = "{\"hello\": \"world\"}"; + thrown.expect(DorisException.class); + thrown.expectMessage(startsWith("Doris FE's response cannot map to schema. res: ")); + RestService.parseSchema(notQueryPlanRes, logger); + + String notOkRes = + "{\"partitions\":{\"11017\":{\"routings\":[\"be1\",\"be2\"],\"version\":3," + + "\"versionHash\":1,\"schemaHash\":1}},\"opaqued_query_plan\":\"queryPlan\",\"status\":20}"; + thrown.expect(DorisException.class); + thrown.expectMessage(startsWith("Doris FE's response is not OK, status is ")); + RestService.parseSchema(notOkRes, logger); + } + + @Test + public void testSelectTabletBe() throws Exception { + String res = + "{\"partitions\":{" + + "\"11017\":{\"routings\":[\"be1\",\"be2\"],\"version\":3,\"versionHash\":1,\"schemaHash\":1}," + + "\"11019\":{\"routings\":[\"be3\",\"be4\"],\"version\":3,\"versionHash\":1,\"schemaHash\":1}," + + "\"11021\":{\"routings\":[\"be3\"],\"version\":3,\"versionHash\":1,\"schemaHash\":1}}," + + "\"opaqued_query_plan\":\"query_plan\",\"status\":200}"; + + QueryPlan queryPlan = RestService.getQueryPlan(res, logger); + + List be1Tablet = new ArrayList<>(); + be1Tablet.add(11017L); + List be3Tablet = new ArrayList<>(); + be3Tablet.add(11019L); + be3Tablet.add(11021L); + Map> expected = new HashMap<>(); + expected.put("be1", be1Tablet); + expected.put("be3", be3Tablet); + + Assert.assertEquals(expected, RestService.selectBeForTablet(queryPlan, logger)); + + String noBeRes = + "{\"partitions\":{" + + "\"11021\":{\"routings\":[],\"version\":3,\"versionHash\":1,\"schemaHash\":1}}," + + "\"opaqued_query_plan\":\"query_plan\",\"status\":200}"; + thrown.expect(DorisException.class); + thrown.expectMessage(startsWith("Cannot choice Doris BE for tablet")); + RestService.selectBeForTablet(RestService.getQueryPlan(noBeRes, logger), logger); + + String notNumberRes = + "{\"partitions\":{" + + "\"11021xxx\":{\"routings\":[\"be1\"],\"version\":3,\"versionHash\":1,\"schemaHash\":1}}," + + "\"opaqued_query_plan\":\"query_plan\",\"status\":200}"; + thrown.expect(DorisException.class); + thrown.expectMessage(startsWith("Parse tablet id ")); + RestService.selectBeForTablet(RestService.getQueryPlan(noBeRes, logger), logger); + } + + @Test + public void testGetTabletSize() { + DorisReadOptions.Builder builder = DorisReadOptions.builder(); + Assert.assertEquals( + DORIS_TABLET_SIZE_DEFAULT.intValue(), + RestService.tabletCountLimitForOnePartition(builder.build(), logger)); + + builder.setRequestTabletSize(null); + Assert.assertEquals( + DORIS_TABLET_SIZE_DEFAULT.intValue(), + RestService.tabletCountLimitForOnePartition(builder.build(), logger)); + + builder.setRequestTabletSize(10); + Assert.assertEquals( + 10, RestService.tabletCountLimitForOnePartition(builder.build(), logger)); + + builder.setRequestTabletSize(1); + Assert.assertEquals( + DORIS_TABLET_SIZE_MIN.intValue(), + RestService.tabletCountLimitForOnePartition(builder.build(), logger)); + } + + @Test + public void testTabletsMapToPartition() throws Exception { + List tablets1 = new ArrayList<>(); + tablets1.add(1L); + tablets1.add(2L); + List tablets2 = new ArrayList<>(); + tablets2.add(3L); + tablets2.add(4L); + Map> beToTablets = new HashMap<>(); + beToTablets.put("be1", tablets1); + beToTablets.put("be2", tablets2); + + String opaquedQueryPlan = "query_plan"; + String cluster = "c"; + String database = "d"; + String table = "t"; + + Set be1Tablet = new HashSet<>(); + be1Tablet.add(1L); + be1Tablet.add(2L); + PartitionDefinition pd1 = + new PartitionDefinition(database, table, "be1", be1Tablet, opaquedQueryPlan); + + Set be2Tablet = new HashSet<>(); + be2Tablet.add(3L); + be2Tablet.add(4L); + PartitionDefinition pd2 = + new PartitionDefinition(database, table, "be2", be2Tablet, opaquedQueryPlan); + + List expected = new ArrayList<>(); + expected.add(pd1); + expected.add(pd2); + Collections.sort(expected); + DorisOptions options = DorisOptions.builder().setFenodes("127.0.0.1:8030").build(); + DorisReadOptions readOptions = DorisReadOptions.builder().setRequestTabletSize(2).build(); + List actual = + RestService.tabletsMapToPartition( + options, + readOptions, + beToTablets, + opaquedQueryPlan, + database, + table, + logger); + Collections.sort(actual); + + Assert.assertEquals(expected, actual); + } + + @Test + public void testParseBackendV2() throws Exception { + String response = + "{\"backends\":[{\"ip\":\"192.168.1.1\",\"http_port\":8042,\"is_alive\":true}, {\"ip\":\"192.168.1.2\",\"http_port\":8042,\"is_alive\":true}]}"; + List backendRows = RestService.parseBackendV2(response, logger); + Assert.assertEquals(2, backendRows.size()); + } +} diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Backend.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/rest/models/TestSchema.java similarity index 62% rename from flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Backend.java rename to flink-doris-connector/src/test/java/org/apache/doris/flink/rest/models/TestSchema.java index 60eaaeecd..66f7134ee 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Backend.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/rest/models/TestSchema.java @@ -17,24 +17,23 @@ package org.apache.doris.flink.rest.models; -import com.fasterxml.jackson.annotation.JsonIgnoreProperties; -import com.fasterxml.jackson.annotation.JsonProperty; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; -import java.util.List; +public class TestSchema { + @Rule public ExpectedException thrown = ExpectedException.none(); -/** Be response model. */ -@Deprecated -@JsonIgnoreProperties(ignoreUnknown = true) -public class Backend { + @Test + public void testPutGet() { + Schema ts = new Schema(1); + Field f = new Field(); + ts.put(f); + Assert.assertEquals(f, ts.get(0)); - @JsonProperty(value = "rows") - private List rows; - - public List getRows() { - return rows; - } - - public void setRows(List rows) { - this.rows = rows; + thrown.expect(IndexOutOfBoundsException.class); + thrown.expectMessage("Index: 1, Fields size:1"); + ts.get(1); } } diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/serialization/TestRouting.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/serialization/TestRouting.java new file mode 100644 index 000000000..95e18f8df --- /dev/null +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/serialization/TestRouting.java @@ -0,0 +1,45 @@ +// 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.doris.flink.serialization; + +import org.apache.doris.flink.exception.IllegalArgumentException; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import static org.hamcrest.core.StringStartsWith.startsWith; + +public class TestRouting { + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testRouting() throws Exception { + Routing r1 = new Routing("10.11.12.13:1234"); + Assert.assertEquals("10.11.12.13", r1.getHost()); + Assert.assertEquals(1234, r1.getPort()); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(startsWith("argument ")); + new Routing("10.11.12.13:wxyz"); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(startsWith("Parse ")); + new Routing("10.11.12.13"); + } +} diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/serialization/TestRowBatch.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/serialization/TestRowBatch.java index d9a64222b..97074e363 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/serialization/TestRowBatch.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/serialization/TestRowBatch.java @@ -1196,4 +1196,27 @@ public void testIPV4() throws DorisException, IOException { thrown.expectMessage(startsWith("Get row offset:")); rowBatch.next(); } + + @Test + public void testCompleteMilliseconds() { + String dt = RowBatch.completeMilliseconds("2021-01-01 10:01:02.123456"); + Assert.assertEquals(dt, "2021-01-01 10:01:02.123456"); + dt = RowBatch.completeMilliseconds("2021-01-01 10:01:02"); + Assert.assertEquals(dt, "2021-01-01 10:01:02.000000"); + dt = RowBatch.completeMilliseconds("2021-01-01 10:01:02.1"); + Assert.assertEquals(dt, "2021-01-01 10:01:02.100000"); + dt = RowBatch.completeMilliseconds("2021-01-01 10:01:02.12"); + Assert.assertEquals(dt, "2021-01-01 10:01:02.120000"); + dt = RowBatch.completeMilliseconds("2021-01-01 10:01:02.123"); + Assert.assertEquals(dt, "2021-01-01 10:01:02.123000"); + dt = RowBatch.completeMilliseconds("2021-01-01 10:01:02.1234"); + Assert.assertEquals(dt, "2021-01-01 10:01:02.123400"); + dt = RowBatch.completeMilliseconds("2021-01-01 10:01:02.12345"); + Assert.assertEquals(dt, "2021-01-01 10:01:02.123450"); + dt = RowBatch.completeMilliseconds("2021-01-01 10:01:02.123456"); + Assert.assertEquals(dt, "2021-01-01 10:01:02.123456"); + + dt = RowBatch.completeMilliseconds("2021-01-01"); + Assert.assertEquals(dt, "2021-01-01"); + } } diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/DorisSinkITCase.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/DorisSinkITCase.java index ad6923cda..5bf75e108 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/DorisSinkITCase.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/DorisSinkITCase.java @@ -17,6 +17,7 @@ package org.apache.doris.flink.sink; +import org.apache.doris.flink.sink.batch.DorisBatchSink; import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; @@ -45,6 +46,8 @@ public class DorisSinkITCase extends DorisTestBase { static final String TABLE_CSV = "tbl_csv"; static final String TABLE_JSON = "tbl_json"; static final String TABLE_JSON_TBL = "tbl_json_tbl"; + static final String TABLE_CSV_BATCH_TBL = "tbl_csv_batch_tbl"; + static final String TABLE_CSV_BATCH_DS = "tbl_csv_batch_DS"; @Test public void testSinkCsvFormat() throws Exception { @@ -148,6 +151,74 @@ public void testTableSinkJsonFormat() throws Exception { checkResult(expected, query, 2); } + @Test + public void testTableBatch() throws Exception { + initializeTable(TABLE_CSV_BATCH_TBL); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + + String sinkDDL = + String.format( + "CREATE TABLE doris_sink (" + + " name STRING," + + " age INT" + + ") WITH (" + + " 'connector' = 'doris'," + + " 'fenodes' = '%s'," + + " 'table.identifier' = '%s'," + + " 'username' = '%s'," + + " 'password' = '%s'," + + " 'sink.enable.batch-mode' = 'true'" + + ")", + getFenodes(), DATABASE + "." + TABLE_CSV_BATCH_TBL, USERNAME, PASSWORD); + tEnv.executeSql(sinkDDL); + tEnv.executeSql("INSERT INTO doris_sink SELECT 'doris',1 union all SELECT 'flink',2"); + + Thread.sleep(10000); + List expected = Arrays.asList("doris,1", "flink,2"); + String query = + String.format("select name,age from %s.%s order by 1", DATABASE, TABLE_CSV_BATCH_TBL); + checkResult(expected, query, 2); + } + + @Test + public void testDataStreamBatch() throws Exception { + initializeTable(TABLE_CSV_BATCH_DS); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + DorisBatchSink.Builder builder = DorisBatchSink.builder(); + final DorisReadOptions.Builder readOptionBuilder = DorisReadOptions.builder(); + + DorisOptions.Builder dorisBuilder = DorisOptions.builder(); + dorisBuilder + .setFenodes(getFenodes()) + .setTableIdentifier(DATABASE + "." + TABLE_CSV_BATCH_DS) + .setUsername(USERNAME) + .setPassword(PASSWORD); + Properties properties = new Properties(); + properties.setProperty("column_separator", ","); + properties.setProperty("line_delimiter", "\n"); + properties.setProperty("format", "csv"); + DorisExecutionOptions.Builder executionBuilder = DorisExecutionOptions.builder(); + executionBuilder.setLabelPrefix(UUID.randomUUID().toString()).setStreamLoadProp(properties); + + builder.setDorisReadOptions(readOptionBuilder.build()) + .setDorisExecutionOptions(executionBuilder.build()) + .setSerializer(new SimpleStringSerializer()) + .setDorisOptions(dorisBuilder.build()); + + env.fromElements("doris,1","flink,2").sinkTo(builder.build()); + env.execute(); + + Thread.sleep(10000); + List expected = Arrays.asList("doris,1", "flink,2"); + String query = + String.format("select name,age from %s.%s order by 1", DATABASE, TABLE_CSV_BATCH_DS); + checkResult(expected, query, 2); + } + private void initializeTable(String table) throws Exception { try (Connection connection = DriverManager.getConnection( diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/HttpTestUtil.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/HttpTestUtil.java index ca5039948..ff305e672 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/HttpTestUtil.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/HttpTestUtil.java @@ -20,6 +20,7 @@ import org.apache.http.ProtocolVersion; import org.apache.http.StatusLine; import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.message.BasicHeader; import org.apache.http.message.BasicStatusLine; import static org.mockito.Mockito.mock; @@ -130,6 +131,7 @@ public class HttpTestUtil { new BasicStatusLine(new ProtocolVersion("http", 1, 0), 200, ""); public static StatusLine abnormalLine = new BasicStatusLine(new ProtocolVersion("http", 1, 0), 404, ""); + public static StatusLine redirectLine = new BasicStatusLine(new ProtocolVersion("http", 1, 0), 307, ""); public static CloseableHttpResponse getResponse(String response, boolean ok) { HttpEntityMock httpEntityMock = new HttpEntityMock(); @@ -143,4 +145,21 @@ public static CloseableHttpResponse getResponse(String response, boolean ok) { when(httpResponse.getEntity()).thenReturn(httpEntityMock); return httpResponse; } + + public static CloseableHttpResponse getResponse(String response, boolean ok, boolean isRedirect) { + HttpEntityMock httpEntityMock = new HttpEntityMock(); + httpEntityMock.setValue(response); + CloseableHttpResponse httpResponse = mock(CloseableHttpResponse.class); + if (isRedirect) { + when(httpResponse.getStatusLine()).thenReturn(redirectLine); + when(httpResponse.getFirstHeader("location")).thenReturn(new BasicHeader("location","http://aliyun.com/xx")); + + }else if(ok) { + when(httpResponse.getStatusLine()).thenReturn(normalLine); + } else { + when(httpResponse.getStatusLine()).thenReturn(abnormalLine); + } + when(httpResponse.getEntity()).thenReturn(httpEntityMock); + return httpResponse; + } } diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/batch/TestRecordWithMeta.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/batch/TestRecordWithMeta.java new file mode 100644 index 000000000..9eddcf404 --- /dev/null +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/batch/TestRecordWithMeta.java @@ -0,0 +1,33 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.flink.sink.batch; + +import org.junit.Assert; +import org.junit.Test; + +public class TestRecordWithMeta { + + @Test + public void testRecordWithMeta() { + RecordWithMeta recordWithMeta = new RecordWithMeta("db", "tbl", "doris,1"); + Assert.assertEquals(recordWithMeta.getRecord(), "doris,1"); + Assert.assertEquals(recordWithMeta.getDatabase(), "db"); + Assert.assertEquals(recordWithMeta.getTable(), "tbl"); + Assert.assertEquals(recordWithMeta.getTableIdentifier(), "db.tbl"); + } +} diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestCopyCommittableSerializer.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestCopyCommittableSerializer.java new file mode 100644 index 000000000..1763f6ad1 --- /dev/null +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestCopyCommittableSerializer.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.doris.flink.sink.copy; + +import org.junit.Assert; +import org.junit.Test; + +public class TestCopyCommittableSerializer { + + @Test + public void testSerialize() throws Exception { + DorisCopyCommittable expectCommittable = + new DorisCopyCommittable( + "fe:8040", + "COPY INTO db.table FROM @u FILES=('label_0_1') FILE_FORMAT=('type'='csv','line_delimiter'='\n','column_separator'=',')"); + CopyCommittableSerializer serializer = new CopyCommittableSerializer(); + DorisCopyCommittable committable = + serializer.deserialize(1, serializer.serialize(expectCommittable)); + Assert.assertEquals(expectCommittable, committable); + } +} diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestDorisCopyWriter.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestDorisCopyWriter.java new file mode 100644 index 000000000..37377eae6 --- /dev/null +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestDorisCopyWriter.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 +// +// 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.doris.flink.sink.copy; + +import org.apache.doris.flink.cfg.DorisExecutionOptions; +import org.apache.doris.flink.cfg.DorisOptions; +import org.apache.doris.flink.cfg.DorisReadOptions; +import org.apache.doris.flink.sink.HttpTestUtil; +import org.apache.doris.flink.sink.OptionUtils; +import org.apache.doris.flink.sink.writer.DorisWriterState; +import org.apache.doris.flink.sink.writer.LabelGenerator; +import org.apache.doris.flink.sink.writer.serializer.SimpleStringSerializer; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClientBuilder; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collection; +import java.util.List; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestDorisCopyWriter { + + DorisOptions dorisOptions; + DorisReadOptions readOptions; + DorisExecutionOptions executionOptions; + + @Before + public void setUp() { + dorisOptions = OptionUtils.buildDorisOptions(); + readOptions = OptionUtils.buildDorisReadOptions(); + executionOptions = OptionUtils.buildExecutionOptional(); + } + + @Test + public void testPrepareCommit() throws Exception { + HttpClientBuilder httpClientBuilder = mock(HttpClientBuilder.class); + CloseableHttpClient httpClient = mock(CloseableHttpClient.class); + when(httpClientBuilder.build()).thenReturn(httpClient); + BatchStageLoad stageLoad = new BatchStageLoad(dorisOptions, readOptions, executionOptions, new LabelGenerator( "label", true)); + stageLoad.setHttpClientBuilder(httpClientBuilder); + + CloseableHttpResponse uploadResponse = HttpTestUtil.getResponse("", false, true); + CloseableHttpResponse preCommitResponse = HttpTestUtil.getResponse("", true, false); + when(httpClient.execute(any())).thenReturn(uploadResponse).thenReturn(preCommitResponse); + + Sink.InitContext initContext = mock(Sink.InitContext.class); + //when(initContext.getRestoredCheckpointId()).thenReturn(OptionalLong.of(1)); + DorisCopyWriter copyWriter = new DorisCopyWriter(initContext, new SimpleStringSerializer(), dorisOptions, readOptions, executionOptions); + copyWriter.setBatchStageLoad(stageLoad); + stageLoad.setCurrentCheckpointID(1); + //no data + Collection committableList = copyWriter.prepareCommit(); + Assert.assertEquals(0, committableList.size()); + //write data + copyWriter.write("xxx",null); + copyWriter.flush(true); + committableList = copyWriter.prepareCommit(); + Assert.assertEquals(1, committableList.size()); + + Assert.assertEquals(1, committableList.size()); + DorisCopyCommittable committable = committableList.toArray(new DorisCopyCommittable[0])[0]; + Assert.assertEquals("127.0.0.1:8030", committable.getHostPort()); + System.out.println(committable.getCopySQL()); + //todo: compare properties + Assert.assertTrue(committable.getCopySQL().startsWith("COPY INTO `db`.`table` FROM @~('{label_table_0_1_0}')")); + } + + @Test + public void testSnapshot() throws Exception { + CloseableHttpClient httpClient = mock(CloseableHttpClient.class); + + Sink.InitContext initContext = mock(Sink.InitContext.class); + //when(initContext.getRestoredCheckpointId()).thenReturn(OptionalLong.of(1)); + DorisCopyWriter copyWriter = new DorisCopyWriter(initContext, new SimpleStringSerializer(), dorisOptions, readOptions, executionOptions); + BatchStageLoad stageLoad = new BatchStageLoad(dorisOptions, readOptions, executionOptions, new LabelGenerator("label", true)); + copyWriter.setBatchStageLoad(stageLoad); + List writerStates = copyWriter.snapshotState(1); + Assert.assertTrue(writerStates.isEmpty()); + } +} diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/writer/TestRecordWithMetaSerializer.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/writer/TestRecordWithMetaSerializer.java new file mode 100644 index 000000000..e5970f4af --- /dev/null +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/writer/TestRecordWithMetaSerializer.java @@ -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. + +package org.apache.doris.flink.sink.writer; + +import org.apache.doris.flink.sink.batch.RecordWithMeta; +import org.apache.doris.flink.sink.writer.serializer.DorisRecord; +import org.apache.doris.flink.sink.writer.serializer.RecordWithMetaSerializer; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class TestRecordWithMetaSerializer { + + @Test + public void testSerialize() throws IOException { + RecordWithMetaSerializer serializer = new RecordWithMetaSerializer(); + RecordWithMeta record = new RecordWithMeta(); + record.setRecord("doris,1"); + DorisRecord serialize = serializer.serialize(record); + Assert.assertNull(null, serialize); + + record.setDatabase("database"); + record.setTable("table"); + serialize = serializer.serialize(record); + DorisRecord expected = DorisRecord.of("database", "table", "doris,1".getBytes()); + Assert.assertEquals(expected.getDatabase(), serialize.getDatabase()); + Assert.assertEquals(expected.getTable(), serialize.getTable()); + Assert.assertEquals(new String(expected.getRow()), new String(serialize.getRow())); + Assert.assertEquals(expected.getTableIdentifier(), serialize.getTableIdentifier()); + } +} diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/source/DorisSourceITCase.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/source/DorisSourceITCase.java index bc30c5722..72ecb5c0d 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/source/DorisSourceITCase.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/source/DorisSourceITCase.java @@ -28,6 +28,8 @@ import org.apache.doris.flink.DorisTestBase; import org.apache.doris.flink.cfg.DorisOptions; import org.apache.doris.flink.cfg.DorisReadOptions; +import org.apache.doris.flink.cfg.DorisStreamOptions; +import org.apache.doris.flink.datastream.DorisSourceFunction; import org.apache.doris.flink.deserialization.SimpleListDeserializationSchema; import org.junit.Assert; import org.junit.Test; @@ -38,12 +40,15 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Properties; /** DorisSource ITCase. */ public class DorisSourceITCase extends DorisTestBase { static final String DATABASE = "test_source"; static final String TABLE_READ = "tbl_read"; + static final String TABLE_READ_OLD_API = "tbl_read_old_api"; static final String TABLE_READ_TBL = "tbl_read_tbl"; + static final String TABLE_READ_TBL_OLD_API = "tbl_read_tbl_old_api"; @Test public void testSource() throws Exception { @@ -77,6 +82,31 @@ public void testSource() throws Exception { Assert.assertArrayEquals(actual.toArray(), expected.toArray()); } + @Test + public void testOldSourceApi() throws Exception { + initializeTable(TABLE_READ_OLD_API); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + Properties properties = new Properties(); + properties.put("fenodes", getFenodes()); + properties.put("username", USERNAME); + properties.put("password", PASSWORD); + properties.put("table.identifier", DATABASE + "." + TABLE_READ_OLD_API); + DorisStreamOptions options = new DorisStreamOptions(properties); + + List actual = new ArrayList<>(); + try (CloseableIterator> iterator = + env.addSource( + new DorisSourceFunction( + options, new SimpleListDeserializationSchema())) + .executeAndCollect()) { + while (iterator.hasNext()) { + actual.add(iterator.next().toString()); + } + } + List expected = Arrays.asList("[doris, 18]", "[flink, 10]"); + Assert.assertArrayEquals(actual.toArray(), expected.toArray()); + } + @Test public void testTableSource() throws Exception { initializeTable(TABLE_READ_TBL); @@ -111,6 +141,40 @@ public void testTableSource() throws Exception { Assert.assertArrayEquals(expected, actual.toArray()); } + @Test + public void testTableSourceOldApi() throws Exception { + initializeTable(TABLE_READ_TBL_OLD_API); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + + String sourceDDL = + String.format( + "CREATE TABLE doris_source (" + + " name STRING," + + " age INT" + + ") WITH (" + + " 'connector' = 'doris'," + + " 'fenodes' = '%s'," + + " 'table.identifier' = '%s'," + + " 'source.use-old-api' = 'true'," + + " 'username' = '%s'," + + " 'password' = '%s'" + + ")", + getFenodes(), DATABASE + "." + TABLE_READ_TBL_OLD_API, USERNAME, PASSWORD); + tEnv.executeSql(sourceDDL); + TableResult tableResult = tEnv.executeSql("SELECT * FROM doris_source"); + + List actual = new ArrayList<>(); + try (CloseableIterator iterator = tableResult.collect()) { + while (iterator.hasNext()) { + actual.add(iterator.next().toString()); + } + } + String[] expected = new String[] {"+I[doris, 18]", "+I[flink, 10]"}; + Assert.assertArrayEquals(expected, actual.toArray()); + } + private void initializeTable(String table) throws Exception { try (Connection connection = DriverManager.getConnection( diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisDynamicTableSourceTest.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisDynamicTableSourceTest.java index 290e19363..45d01cf8b 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisDynamicTableSourceTest.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisDynamicTableSourceTest.java @@ -19,25 +19,45 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.connector.source.Source; +import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.source.InputFormatProvider; import org.apache.flink.table.connector.source.ScanTableSource; import org.apache.flink.table.connector.source.SourceProvider; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; +import org.apache.flink.table.types.DataType; import org.apache.doris.flink.cfg.DorisLookupOptions; import org.apache.doris.flink.cfg.DorisReadOptions; +import org.apache.doris.flink.rest.PartitionDefinition; +import org.apache.doris.flink.rest.RestService; import org.apache.doris.flink.sink.OptionUtils; import org.apache.doris.flink.source.DorisSource; import org.apache.doris.flink.source.enumerator.PendingSplitsCheckpoint; import org.apache.doris.flink.source.split.DorisSourceSplit; import org.apache.doris.flink.utils.FactoryMocks; -import org.junit.Ignore; +import org.junit.Assert; import org.junit.Test; +import org.mockito.MockedStatic; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; + +import static org.apache.flink.table.expressions.ApiExpressionUtils.valueLiteral; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mockStatic; public class DorisDynamicTableSourceTest { @@ -71,19 +91,27 @@ public void testDorisUseNewApiDefault() { assertDorisSource(provider); } - @Ignore @Test public void testDorisUseOldApi() { DorisReadOptions.Builder builder = OptionUtils.dorisReadOptionsBuilder(); builder.setUseOldApi(true); + MockedStatic restServiceMockedStatic = mockStatic(RestService.class); + restServiceMockedStatic + .when(() -> RestService.findPartitions(any(), any(), any())) + .thenReturn( + Collections.singletonList( + new PartitionDefinition("", "", "", new HashSet<>(), ""))); final DorisDynamicTableSource actualDorisSource = new DorisDynamicTableSource( OptionUtils.buildDorisOptions(), builder.build(), - TableSchema.fromResolvedSchema(FactoryMocks.SCHEMA)); + DorisLookupOptions.builder().build(), + TableSchema.fromResolvedSchema(FactoryMocks.SCHEMA), + FactoryMocks.PHYSICAL_DATA_TYPE); ScanTableSource.ScanRuntimeProvider provider = actualDorisSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); assertDorisInputFormat(provider); + restServiceMockedStatic.close(); } private void assertDorisInputFormat(ScanTableSource.ScanRuntimeProvider provider) { @@ -105,4 +133,189 @@ private void assertDorisSource(ScanTableSource.ScanRuntimeProvider provider) { sourceProvider.createSource(); assertThat(source, instanceOf(DorisSource.class)); } + + @Test + public void testApplyProjection() { + int[][] projectionArray = new int[2][1]; + projectionArray[0][0] = 0; + projectionArray[1][0] = 2; + DataType projectionDataType = + ResolvedSchema.of( + Column.physical("a", DataTypes.STRING()), + Column.physical("c", DataTypes.BOOLEAN())) + .toPhysicalRowDataType(); + DorisReadOptions readOptions = OptionUtils.dorisReadOptionsBuilder().build(); + final DorisDynamicTableSource actualDorisSource = + new DorisDynamicTableSource( + OptionUtils.buildDorisOptions(), + readOptions, + DorisLookupOptions.builder().build(), + TableSchema.fromResolvedSchema(FactoryMocks.SCHEMA), + FactoryMocks.PHYSICAL_DATA_TYPE); + actualDorisSource.applyProjection(projectionArray, projectionDataType); + Assert.assertEquals(readOptions.getReadFields(), "`a`, `c`"); + } + + @Test + public void testFilter() { + DorisReadOptions readOptions = OptionUtils.dorisReadOptionsBuilder().build(); + final DorisDynamicTableSource actualDorisSource = + new DorisDynamicTableSource( + OptionUtils.buildDorisOptions(), + readOptions, + DorisLookupOptions.builder().build(), + TableSchema.fromResolvedSchema(FactoryMocks.SCHEMA), + FactoryMocks.PHYSICAL_DATA_TYPE); + ResolvedExpression aRef = new FieldReferenceExpression("a", DataTypes.STRING(), 0, 2); + ResolvedExpression aRefCharLength = + new CallExpression( + BuiltInFunctionDefinitions.CHAR_LENGTH, + Collections.singletonList(aRef), + DataTypes.INT()); + ResolvedExpression aExp = + new CallExpression( + BuiltInFunctionDefinitions.LESS_THAN, + Arrays.asList(aRefCharLength, valueLiteral(10)), + DataTypes.BOOLEAN()); + actualDorisSource.applyFilters(Collections.singletonList(aExp)); + Assert.assertTrue(actualDorisSource.getResolvedFilterQuery().isEmpty()); + + ResolvedExpression a1Ref = new FieldReferenceExpression("a", DataTypes.STRING(), 0, 2); + ResolvedExpression a1Exp = + new CallExpression( + BuiltInFunctionDefinitions.EQUALS, + Arrays.asList(a1Ref, valueLiteral("doris")), + DataTypes.BOOLEAN()); + actualDorisSource.applyFilters(Arrays.asList(a1Exp)); + assertEquals(Arrays.asList("(a = 'doris')"), actualDorisSource.getResolvedFilterQuery()); + + ResolvedExpression b1Ref = new FieldReferenceExpression("b", DataTypes.INT(), 0, 2); + ResolvedExpression b1Exp = + new CallExpression( + BuiltInFunctionDefinitions.EQUALS, + Arrays.asList(b1Ref, valueLiteral(1)), + DataTypes.BOOLEAN()); + actualDorisSource.getResolvedFilterQuery().clear(); + actualDorisSource.applyFilters(Arrays.asList(b1Exp)); + assertEquals(Arrays.asList("(b = 1)"), actualDorisSource.getResolvedFilterQuery()); + + actualDorisSource.getResolvedFilterQuery().clear(); + actualDorisSource.applyFilters( + Arrays.asList( + b1Exp, + new CallExpression( + BuiltInFunctionDefinitions.NOT_EQUALS, + Arrays.asList(b1Ref, valueLiteral(1)), + DataTypes.BOOLEAN()), + new CallExpression( + BuiltInFunctionDefinitions.GREATER_THAN, + Arrays.asList(b1Ref, valueLiteral(1)), + DataTypes.BOOLEAN()), + new CallExpression( + BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL, + Arrays.asList(b1Ref, valueLiteral(1)), + DataTypes.BOOLEAN()), + new CallExpression( + BuiltInFunctionDefinitions.LESS_THAN, + Arrays.asList(b1Ref, valueLiteral(1)), + DataTypes.BOOLEAN()), + new CallExpression( + BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, + Arrays.asList(b1Ref, valueLiteral(1)), + DataTypes.BOOLEAN()))); + assertEquals( + Arrays.asList("(b = 1)", "(b <> 1)", "(b > 1)", "(b >= 1)", "(b < 1)", "(b <= 1)"), + actualDorisSource.getResolvedFilterQuery()); + + actualDorisSource.getResolvedFilterQuery().clear(); + actualDorisSource.applyFilters( + Arrays.asList( + new CallExpression( + BuiltInFunctionDefinitions.OR, + Arrays.asList(a1Exp, b1Exp), + DataTypes.BOOLEAN()))); + assertEquals( + Arrays.asList("((a = 'doris') OR (b = 1))"), + actualDorisSource.getResolvedFilterQuery()); + + actualDorisSource.getResolvedFilterQuery().clear(); + actualDorisSource.applyFilters( + Arrays.asList( + new CallExpression( + BuiltInFunctionDefinitions.AND, + Arrays.asList(a1Exp, b1Exp), + DataTypes.BOOLEAN()))); + assertEquals( + Arrays.asList("((a = 'doris') AND (b = 1))"), + actualDorisSource.getResolvedFilterQuery()); + + ResolvedExpression aLikeExp = + new CallExpression( + BuiltInFunctionDefinitions.LIKE, + Arrays.asList(a1Ref, valueLiteral("d")), + DataTypes.BOOLEAN()); + actualDorisSource.getResolvedFilterQuery().clear(); + actualDorisSource.applyFilters(Collections.singletonList(aLikeExp)); + assertEquals(Arrays.asList("(a LIKE 'd')"), actualDorisSource.getResolvedFilterQuery()); + + ResolvedExpression aInExp = + new CallExpression( + BuiltInFunctionDefinitions.IN, + Arrays.asList(a1Ref, valueLiteral("doris")), + DataTypes.BOOLEAN()); + actualDorisSource.getResolvedFilterQuery().clear(); + actualDorisSource.applyFilters(Collections.singletonList(aInExp)); + assertTrue(actualDorisSource.getResolvedFilterQuery().isEmpty()); + + ResolvedExpression aNoNullExp = + new CallExpression( + BuiltInFunctionDefinitions.IS_NOT_NULL, + Arrays.asList(a1Ref), + DataTypes.BOOLEAN()); + actualDorisSource.getResolvedFilterQuery().clear(); + actualDorisSource.applyFilters(Collections.singletonList(aNoNullExp)); + assertEquals(Arrays.asList("(a IS NOT NULL)"), actualDorisSource.getResolvedFilterQuery()); + + ResolvedExpression aNullExp = + new CallExpression( + BuiltInFunctionDefinitions.IS_NULL, + Arrays.asList(a1Ref), + DataTypes.BOOLEAN()); + actualDorisSource.getResolvedFilterQuery().clear(); + actualDorisSource.applyFilters(Collections.singletonList(aNullExp)); + assertEquals(Arrays.asList("(a IS NULL)"), actualDorisSource.getResolvedFilterQuery()); + } + + @Test + public void testFilterDate() { + DorisReadOptions readOptions = OptionUtils.dorisReadOptionsBuilder().build(); + final DorisDynamicTableSource actualDorisSource = + new DorisDynamicTableSource( + OptionUtils.buildDorisOptions(), + readOptions, + DorisLookupOptions.builder().build(), + TableSchema.fromResolvedSchema(FactoryMocks.SCHEMA_DT), + FactoryMocks.SCHEMA_DT.toPhysicalRowDataType()); + ResolvedExpression dRef = new FieldReferenceExpression("d", DataTypes.DATE(), 0, 2); + ResolvedExpression dExp = + new CallExpression( + BuiltInFunctionDefinitions.EQUALS, + Arrays.asList(dRef, valueLiteral("2021-01-01")), + DataTypes.BOOLEAN()); + actualDorisSource.applyFilters(Arrays.asList(dExp)); + assertEquals( + Arrays.asList("(d = '2021-01-01')"), actualDorisSource.getResolvedFilterQuery()); + + ResolvedExpression eRef = new FieldReferenceExpression("e", DataTypes.TIMESTAMP(), 0, 2); + ResolvedExpression eExp = + new CallExpression( + BuiltInFunctionDefinitions.EQUALS, + Arrays.asList(eRef, valueLiteral("2021-01-01 11:12:13")), + DataTypes.BOOLEAN()); + actualDorisSource.getResolvedFilterQuery().clear(); + actualDorisSource.applyFilters(Arrays.asList(eExp)); + assertEquals( + Arrays.asList("(e = '2021-01-01 11:12:13')"), + actualDorisSource.getResolvedFilterQuery()); + } } diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/CdcMongoSyncDatabaseCase.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/CdcMongoSyncDatabaseCase.java index ef66e8825..e247e4298 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/CdcMongoSyncDatabaseCase.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/CdcMongoSyncDatabaseCase.java @@ -20,7 +20,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.doris.flink.tools.cdc.mongodb.MongoDBDatabaseSync; +import org.apache.doris.flink.tools.cdc.mongodb.serializer.MongoDBDatabaseSync; import java.util.HashMap; import java.util.Map; diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/utils/FactoryMocks.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/utils/FactoryMocks.java index 41ea4cfc9..74d0b413a 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/utils/FactoryMocks.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/utils/FactoryMocks.java @@ -34,6 +34,14 @@ public final class FactoryMocks { Column.physical("b", DataTypes.INT()), Column.physical("c", DataTypes.BOOLEAN())); + public static final ResolvedSchema SCHEMA_DT = + ResolvedSchema.of( + Column.physical("a", DataTypes.STRING()), + Column.physical("b", DataTypes.INT()), + Column.physical("c", DataTypes.BOOLEAN()), + Column.physical("d", DataTypes.DATE()), + Column.physical("e", DataTypes.TIMESTAMP())); + public static final DataType PHYSICAL_DATA_TYPE = SCHEMA.toPhysicalRowDataType(); public static final RowType PHYSICAL_TYPE = (RowType) PHYSICAL_DATA_TYPE.getLogicalType(); From 4a690f7c286e1f5f8c1ba5d6569a18df665a34b1 Mon Sep 17 00:00:00 2001 From: wudi <676366545@qq.com> Date: Tue, 28 May 2024 20:46:15 +0800 Subject: [PATCH 2/5] update code style --- .../doris/flink/catalog/DorisCatalog.java | 15 +++--- .../serializer/MongoDBDatabaseSync.java | 14 +++--- .../MongoJsonDebeziumDataChange.java | 4 +- .../catalog/DorisCatalogFactoryTest.java | 16 +++--- .../doris/flink/sink/DorisSinkITCase.java | 10 ++-- .../apache/doris/flink/sink/HttpTestUtil.java | 11 ++-- .../flink/sink/copy/TestDorisCopyWriter.java | 50 ++++++++++++++----- 7 files changed, 78 insertions(+), 42 deletions(-) diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/DorisCatalog.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/DorisCatalog.java index 8968e7b5a..c56f64a02 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/DorisCatalog.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/DorisCatalog.java @@ -17,13 +17,6 @@ package org.apache.doris.flink.catalog; -import org.apache.commons.compress.utils.Lists; -import org.apache.doris.flink.catalog.doris.DataModel; -import org.apache.doris.flink.catalog.doris.DorisSystem; -import org.apache.doris.flink.catalog.doris.FieldSchema; -import org.apache.doris.flink.catalog.doris.TableSchema; -import org.apache.doris.flink.cfg.DorisConnectionOptions; -import org.apache.doris.flink.table.DorisDynamicTableFactory; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.table.api.Schema; import org.apache.flink.table.catalog.AbstractCatalog; @@ -55,6 +48,14 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; + +import org.apache.commons.compress.utils.Lists; +import org.apache.doris.flink.catalog.doris.DataModel; +import org.apache.doris.flink.catalog.doris.DorisSystem; +import org.apache.doris.flink.catalog.doris.FieldSchema; +import org.apache.doris.flink.catalog.doris.TableSchema; +import org.apache.doris.flink.cfg.DorisConnectionOptions; +import org.apache.doris.flink.table.DorisDynamicTableFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoDBDatabaseSync.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoDBDatabaseSync.java index bb6bd2310..eeb9d92c2 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoDBDatabaseSync.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoDBDatabaseSync.java @@ -17,6 +17,12 @@ package org.apache.doris.flink.tools.cdc.mongodb.serializer; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + import com.mongodb.ConnectionString; import com.mongodb.MongoClientSettings; import com.mongodb.client.MongoClient; @@ -41,20 +47,16 @@ import org.apache.doris.flink.tools.cdc.SourceSchema; import org.apache.doris.flink.tools.cdc.mongodb.MongoDBSchema; import org.apache.doris.flink.tools.cdc.mongodb.MongoParsingProcessFunction; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.bson.Document; +import javax.annotation.Nullable; + import java.sql.Connection; import java.sql.SQLException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import javax.annotation.Nullable; import static com.ververica.cdc.connectors.mongodb.internal.MongoDBEnvelope.encodeValue; import static org.apache.flink.util.Preconditions.checkNotNull; diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoJsonDebeziumDataChange.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoJsonDebeziumDataChange.java index e448dbc57..4b20ebd67 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoJsonDebeziumDataChange.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoJsonDebeziumDataChange.java @@ -17,8 +17,6 @@ package org.apache.doris.flink.tools.cdc.mongodb.serializer; -import org.apache.doris.flink.sink.writer.serializer.jsondebezium.CdcDataChange; -import org.apache.doris.flink.sink.writer.serializer.jsondebezium.JsonDebeziumChangeContext; import org.apache.flink.util.StringUtils; import com.fasterxml.jackson.core.JsonProcessingException; @@ -29,6 +27,8 @@ import org.apache.doris.flink.cfg.DorisOptions; import org.apache.doris.flink.sink.writer.ChangeEvent; import org.apache.doris.flink.sink.writer.serializer.DorisRecord; +import org.apache.doris.flink.sink.writer.serializer.jsondebezium.CdcDataChange; +import org.apache.doris.flink.sink.writer.serializer.jsondebezium.JsonDebeziumChangeContext; import org.apache.doris.flink.tools.cdc.SourceSchema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogFactoryTest.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogFactoryTest.java index d45984541..2fa74fd4e 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogFactoryTest.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogFactoryTest.java @@ -19,6 +19,7 @@ import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.factories.FactoryUtil; + import org.junit.Test; import java.util.HashMap; @@ -44,17 +45,18 @@ public void testCreateCatalog() { options.put("sink.properties.read_json_by_line", "true"); options.put("table.properties.replication_num", "1"); - Catalog catalog = FactoryUtil.createCatalog( - "test_catalog", - options, - null, - Thread.currentThread().getContextClassLoader() - ); + Catalog catalog = + FactoryUtil.createCatalog( + "test_catalog", + options, + null, + Thread.currentThread().getContextClassLoader()); assertTrue(catalog instanceof DorisCatalog); DorisCatalog dorisCatalog = (DorisCatalog) catalog; assertEquals("test_catalog", dorisCatalog.getName()); assertEquals("doris_db", dorisCatalog.getDefaultDatabase()); - assertEquals("jdbc:mysql://127.0.0.1:9030", dorisCatalog.getConnectionOptions().getJdbcUrl()); + assertEquals( + "jdbc:mysql://127.0.0.1:9030", dorisCatalog.getConnectionOptions().getJdbcUrl()); assertEquals("127.0.0.1:8030", dorisCatalog.getConnectionOptions().getFenodes()); assertEquals("root", dorisCatalog.getConnectionOptions().getUsername()); assertEquals("", dorisCatalog.getConnectionOptions().getPassword()); diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/DorisSinkITCase.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/DorisSinkITCase.java index 5bf75e108..d5a8173ef 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/DorisSinkITCase.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/DorisSinkITCase.java @@ -17,7 +17,6 @@ package org.apache.doris.flink.sink; -import org.apache.doris.flink.sink.batch.DorisBatchSink; import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; @@ -27,6 +26,7 @@ import org.apache.doris.flink.cfg.DorisExecutionOptions; import org.apache.doris.flink.cfg.DorisOptions; import org.apache.doris.flink.cfg.DorisReadOptions; +import org.apache.doris.flink.sink.batch.DorisBatchSink; import org.apache.doris.flink.sink.writer.serializer.SimpleStringSerializer; import org.junit.Test; @@ -179,7 +179,8 @@ public void testTableBatch() throws Exception { Thread.sleep(10000); List expected = Arrays.asList("doris,1", "flink,2"); String query = - String.format("select name,age from %s.%s order by 1", DATABASE, TABLE_CSV_BATCH_TBL); + String.format( + "select name,age from %s.%s order by 1", DATABASE, TABLE_CSV_BATCH_TBL); checkResult(expected, query, 2); } @@ -209,13 +210,14 @@ public void testDataStreamBatch() throws Exception { .setSerializer(new SimpleStringSerializer()) .setDorisOptions(dorisBuilder.build()); - env.fromElements("doris,1","flink,2").sinkTo(builder.build()); + env.fromElements("doris,1", "flink,2").sinkTo(builder.build()); env.execute(); Thread.sleep(10000); List expected = Arrays.asList("doris,1", "flink,2"); String query = - String.format("select name,age from %s.%s order by 1", DATABASE, TABLE_CSV_BATCH_DS); + String.format( + "select name,age from %s.%s order by 1", DATABASE, TABLE_CSV_BATCH_DS); checkResult(expected, query, 2); } diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/HttpTestUtil.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/HttpTestUtil.java index ff305e672..3a9b8caa0 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/HttpTestUtil.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/HttpTestUtil.java @@ -131,7 +131,8 @@ public class HttpTestUtil { new BasicStatusLine(new ProtocolVersion("http", 1, 0), 200, ""); public static StatusLine abnormalLine = new BasicStatusLine(new ProtocolVersion("http", 1, 0), 404, ""); - public static StatusLine redirectLine = new BasicStatusLine(new ProtocolVersion("http", 1, 0), 307, ""); + public static StatusLine redirectLine = + new BasicStatusLine(new ProtocolVersion("http", 1, 0), 307, ""); public static CloseableHttpResponse getResponse(String response, boolean ok) { HttpEntityMock httpEntityMock = new HttpEntityMock(); @@ -146,15 +147,17 @@ public static CloseableHttpResponse getResponse(String response, boolean ok) { return httpResponse; } - public static CloseableHttpResponse getResponse(String response, boolean ok, boolean isRedirect) { + public static CloseableHttpResponse getResponse( + String response, boolean ok, boolean isRedirect) { HttpEntityMock httpEntityMock = new HttpEntityMock(); httpEntityMock.setValue(response); CloseableHttpResponse httpResponse = mock(CloseableHttpResponse.class); if (isRedirect) { when(httpResponse.getStatusLine()).thenReturn(redirectLine); - when(httpResponse.getFirstHeader("location")).thenReturn(new BasicHeader("location","http://aliyun.com/xx")); + when(httpResponse.getFirstHeader("location")) + .thenReturn(new BasicHeader("location", "http://aliyun.com/xx")); - }else if(ok) { + } else if (ok) { when(httpResponse.getStatusLine()).thenReturn(normalLine); } else { when(httpResponse.getStatusLine()).thenReturn(abnormalLine); diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestDorisCopyWriter.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestDorisCopyWriter.java index 37377eae6..4d7b8f631 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestDorisCopyWriter.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestDorisCopyWriter.java @@ -17,6 +17,8 @@ package org.apache.doris.flink.sink.copy; +import org.apache.flink.api.connector.sink2.Sink; + import org.apache.doris.flink.cfg.DorisExecutionOptions; import org.apache.doris.flink.cfg.DorisOptions; import org.apache.doris.flink.cfg.DorisReadOptions; @@ -25,7 +27,6 @@ import org.apache.doris.flink.sink.writer.DorisWriterState; import org.apache.doris.flink.sink.writer.LabelGenerator; import org.apache.doris.flink.sink.writer.serializer.SimpleStringSerializer; -import org.apache.flink.api.connector.sink2.Sink; import org.apache.http.client.methods.CloseableHttpResponse; import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClientBuilder; @@ -58,7 +59,12 @@ public void testPrepareCommit() throws Exception { HttpClientBuilder httpClientBuilder = mock(HttpClientBuilder.class); CloseableHttpClient httpClient = mock(CloseableHttpClient.class); when(httpClientBuilder.build()).thenReturn(httpClient); - BatchStageLoad stageLoad = new BatchStageLoad(dorisOptions, readOptions, executionOptions, new LabelGenerator( "label", true)); + BatchStageLoad stageLoad = + new BatchStageLoad( + dorisOptions, + readOptions, + executionOptions, + new LabelGenerator("label", true)); stageLoad.setHttpClientBuilder(httpClientBuilder); CloseableHttpResponse uploadResponse = HttpTestUtil.getResponse("", false, true); @@ -66,15 +72,21 @@ public void testPrepareCommit() throws Exception { when(httpClient.execute(any())).thenReturn(uploadResponse).thenReturn(preCommitResponse); Sink.InitContext initContext = mock(Sink.InitContext.class); - //when(initContext.getRestoredCheckpointId()).thenReturn(OptionalLong.of(1)); - DorisCopyWriter copyWriter = new DorisCopyWriter(initContext, new SimpleStringSerializer(), dorisOptions, readOptions, executionOptions); + // when(initContext.getRestoredCheckpointId()).thenReturn(OptionalLong.of(1)); + DorisCopyWriter copyWriter = + new DorisCopyWriter( + initContext, + new SimpleStringSerializer(), + dorisOptions, + readOptions, + executionOptions); copyWriter.setBatchStageLoad(stageLoad); stageLoad.setCurrentCheckpointID(1); - //no data + // no data Collection committableList = copyWriter.prepareCommit(); Assert.assertEquals(0, committableList.size()); - //write data - copyWriter.write("xxx",null); + // write data + copyWriter.write("xxx", null); copyWriter.flush(true); committableList = copyWriter.prepareCommit(); Assert.assertEquals(1, committableList.size()); @@ -83,8 +95,11 @@ public void testPrepareCommit() throws Exception { DorisCopyCommittable committable = committableList.toArray(new DorisCopyCommittable[0])[0]; Assert.assertEquals("127.0.0.1:8030", committable.getHostPort()); System.out.println(committable.getCopySQL()); - //todo: compare properties - Assert.assertTrue(committable.getCopySQL().startsWith("COPY INTO `db`.`table` FROM @~('{label_table_0_1_0}')")); + // todo: compare properties + Assert.assertTrue( + committable + .getCopySQL() + .startsWith("COPY INTO `db`.`table` FROM @~('{label_table_0_1_0}')")); } @Test @@ -92,9 +107,20 @@ public void testSnapshot() throws Exception { CloseableHttpClient httpClient = mock(CloseableHttpClient.class); Sink.InitContext initContext = mock(Sink.InitContext.class); - //when(initContext.getRestoredCheckpointId()).thenReturn(OptionalLong.of(1)); - DorisCopyWriter copyWriter = new DorisCopyWriter(initContext, new SimpleStringSerializer(), dorisOptions, readOptions, executionOptions); - BatchStageLoad stageLoad = new BatchStageLoad(dorisOptions, readOptions, executionOptions, new LabelGenerator("label", true)); + // when(initContext.getRestoredCheckpointId()).thenReturn(OptionalLong.of(1)); + DorisCopyWriter copyWriter = + new DorisCopyWriter( + initContext, + new SimpleStringSerializer(), + dorisOptions, + readOptions, + executionOptions); + BatchStageLoad stageLoad = + new BatchStageLoad( + dorisOptions, + readOptions, + executionOptions, + new LabelGenerator("label", true)); copyWriter.setBatchStageLoad(stageLoad); List writerStates = copyWriter.snapshotState(1); Assert.assertTrue(writerStates.isEmpty()); From 364b1730ebf94bff08c19485b384dbe6065414d2 Mon Sep 17 00:00:00 2001 From: wudi <676366545@qq.com> Date: Wed, 29 May 2024 20:17:35 +0800 Subject: [PATCH 3/5] update --- .../flink/cfg/DorisConnectionOptions.java | 23 ++ .../flink/cfg/DorisExecutionOptions.java | 53 +++++ .../doris/flink/cfg/DorisLookupOptions.java | 31 +++ .../apache/doris/flink/cfg/DorisOptions.java | 26 +++ .../doris/flink/cfg/DorisReadOptions.java | 41 ++++ .../flink/table/DorisDynamicTableSink.java | 22 ++ .../flink/table/DorisDynamicTableSource.java | 29 +++ .../table/DorisRowDataLookupFunction.java | 164 -------------- .../doris/flink/tools/cdc/CdcTools.java | 6 +- .../{serializer => }/MongoDBDatabaseSync.java | 5 +- .../doris/flink/sink/DorisSinkITCase.java | 30 ++- .../flink/sink/copy/TestDorisCopyWriter.java | 2 + .../TestJsonDebeziumSchemaChangeImplV2.java | 25 +++ .../doris/flink/source/DorisSourceITCase.java | 47 ++++ .../table/DorisDynamicTableFactoryTest.java | 211 ++++++++++++++++++ .../table/DorisDynamicTableSourceTest.java | 11 +- ...> DorisRowDataJdbcLookupFunctionTest.java} | 88 ++++++-- .../tools/cdc/CdcMongoSyncDatabaseCase.java | 2 +- .../doris/flink/tools/cdc/CdcToolsTest.java | 52 +++++ .../flink/tools/cdc/DorisDorisE2ECase.java | 17 +- .../flink/tools/cdc/MySQLDorisE2ECase.java | 101 +++++++++ 21 files changed, 783 insertions(+), 203 deletions(-) delete mode 100644 flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisRowDataLookupFunction.java rename flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/{serializer => }/MongoDBDatabaseSync.java (98%) create mode 100644 flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisDynamicTableFactoryTest.java rename flink-doris-connector/src/test/java/org/apache/doris/flink/table/{DorisRowDataLookupFunctionTest.java => DorisRowDataJdbcLookupFunctionTest.java} (59%) create mode 100644 flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/CdcToolsTest.java diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisConnectionOptions.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisConnectionOptions.java index 37338b28b..82e4d3753 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisConnectionOptions.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisConnectionOptions.java @@ -20,6 +20,7 @@ import org.apache.flink.util.Preconditions; import java.io.Serializable; +import java.util.Objects; /** Doris connection options. */ public class DorisConnectionOptions implements Serializable { @@ -86,6 +87,28 @@ public boolean isAutoRedirect() { return autoRedirect; } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DorisConnectionOptions that = (DorisConnectionOptions) o; + return autoRedirect == that.autoRedirect + && Objects.equals(fenodes, that.fenodes) + && Objects.equals(username, that.username) + && Objects.equals(password, that.password) + && Objects.equals(jdbcUrl, that.jdbcUrl) + && Objects.equals(benodes, that.benodes); + } + + @Override + public int hashCode() { + return Objects.hash(fenodes, username, password, jdbcUrl, benodes, autoRedirect); + } + /** Builder for {@link DorisConnectionOptions}. */ public static class DorisConnectionOptionsBuilder { private String fenodes; diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisExecutionOptions.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisExecutionOptions.java index 779a296a4..8f3cc240d 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisExecutionOptions.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisExecutionOptions.java @@ -22,6 +22,7 @@ import org.apache.doris.flink.sink.writer.WriteMode; import java.io.Serializable; +import java.util.Objects; import java.util.Properties; import static org.apache.doris.flink.sink.writer.LoadConstants.FORMAT_KEY; @@ -213,6 +214,58 @@ public boolean ignoreCommitError() { return ignoreCommitError; } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DorisExecutionOptions that = (DorisExecutionOptions) o; + return checkInterval == that.checkInterval + && maxRetries == that.maxRetries + && bufferSize == that.bufferSize + && bufferCount == that.bufferCount + && useCache == that.useCache + && force2PC == that.force2PC + && flushQueueSize == that.flushQueueSize + && bufferFlushMaxRows == that.bufferFlushMaxRows + && bufferFlushMaxBytes == that.bufferFlushMaxBytes + && bufferFlushIntervalMs == that.bufferFlushIntervalMs + && enableBatchMode == that.enableBatchMode + && ignoreUpdateBefore == that.ignoreUpdateBefore + && ignoreCommitError == that.ignoreCommitError + && Objects.equals(labelPrefix, that.labelPrefix) + && Objects.equals(streamLoadProp, that.streamLoadProp) + && Objects.equals(enableDelete, that.enableDelete) + && Objects.equals(enable2PC, that.enable2PC) + && writeMode == that.writeMode; + } + + @Override + public int hashCode() { + return Objects.hash( + checkInterval, + maxRetries, + bufferSize, + bufferCount, + labelPrefix, + useCache, + streamLoadProp, + enableDelete, + enable2PC, + force2PC, + flushQueueSize, + bufferFlushMaxRows, + bufferFlushMaxBytes, + bufferFlushIntervalMs, + enableBatchMode, + ignoreUpdateBefore, + writeMode, + ignoreCommitError); + } + /** Builder of {@link DorisExecutionOptions}. */ public static class Builder { private int checkInterval = DEFAULT_CHECK_INTERVAL; diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisLookupOptions.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisLookupOptions.java index 43b901a4a..76241b3a3 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisLookupOptions.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisLookupOptions.java @@ -18,6 +18,7 @@ package org.apache.doris.flink.cfg; import java.io.Serializable; +import java.util.Objects; public class DorisLookupOptions implements Serializable { @@ -75,6 +76,36 @@ public boolean isAsync() { return async; } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DorisLookupOptions that = (DorisLookupOptions) o; + return cacheMaxSize == that.cacheMaxSize + && cacheExpireMs == that.cacheExpireMs + && maxRetryTimes == that.maxRetryTimes + && jdbcReadBatchSize == that.jdbcReadBatchSize + && jdbcReadBatchQueueSize == that.jdbcReadBatchQueueSize + && jdbcReadThreadSize == that.jdbcReadThreadSize + && async == that.async; + } + + @Override + public int hashCode() { + return Objects.hash( + cacheMaxSize, + cacheExpireMs, + maxRetryTimes, + jdbcReadBatchSize, + jdbcReadBatchQueueSize, + jdbcReadThreadSize, + async); + } + public static Builder builder() { return new Builder(); } diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisOptions.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisOptions.java index 7d42fb797..bf6c7a28c 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisOptions.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisOptions.java @@ -17,6 +17,8 @@ package org.apache.doris.flink.cfg; +import java.util.Objects; + import static org.apache.flink.util.Preconditions.checkNotNull; /** Options for the Doris connector. */ @@ -65,6 +67,30 @@ public static Builder builder() { return new Builder(); } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DorisOptions that = (DorisOptions) o; + return Objects.equals(tableIdentifier, that.tableIdentifier) + && autoRedirect == that.autoRedirect + && Objects.equals(fenodes, that.fenodes) + && Objects.equals(username, that.username) + && Objects.equals(password, that.password) + && Objects.equals(jdbcUrl, that.jdbcUrl) + && Objects.equals(benodes, that.benodes); + } + + @Override + public int hashCode() { + return Objects.hash( + fenodes, username, password, jdbcUrl, benodes, autoRedirect, tableIdentifier); + } + /** Builder of {@link DorisOptions}. */ public static class Builder { private String fenodes; diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisReadOptions.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisReadOptions.java index 99bbc36f4..3669e740a 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisReadOptions.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisReadOptions.java @@ -18,6 +18,7 @@ package org.apache.doris.flink.cfg; import java.io.Serializable; +import java.util.Objects; /** Doris read Options. */ public class DorisReadOptions implements Serializable { @@ -128,6 +129,46 @@ public static DorisReadOptions defaults() { return DorisReadOptions.builder().build(); } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DorisReadOptions that = (DorisReadOptions) o; + return useOldApi == that.useOldApi + && Objects.equals(readFields, that.readFields) + && Objects.equals(filterQuery, that.filterQuery) + && Objects.equals(requestTabletSize, that.requestTabletSize) + && Objects.equals(requestConnectTimeoutMs, that.requestConnectTimeoutMs) + && Objects.equals(requestReadTimeoutMs, that.requestReadTimeoutMs) + && Objects.equals(requestQueryTimeoutS, that.requestQueryTimeoutS) + && Objects.equals(requestRetries, that.requestRetries) + && Objects.equals(requestBatchSize, that.requestBatchSize) + && Objects.equals(execMemLimit, that.execMemLimit) + && Objects.equals(deserializeQueueSize, that.deserializeQueueSize) + && Objects.equals(deserializeArrowAsync, that.deserializeArrowAsync); + } + + @Override + public int hashCode() { + return Objects.hash( + readFields, + filterQuery, + requestTabletSize, + requestConnectTimeoutMs, + requestReadTimeoutMs, + requestQueryTimeoutS, + requestRetries, + requestBatchSize, + execMemLimit, + deserializeQueueSize, + deserializeArrowAsync, + useOldApi); + } + /** Builder of {@link DorisReadOptions}. */ public static class Builder { diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSink.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSink.java index de5b32f42..17db7d2e1 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSink.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSink.java @@ -34,6 +34,7 @@ import org.slf4j.LoggerFactory; import java.util.Arrays; +import java.util.Objects; import java.util.Properties; import java.util.stream.Collectors; @@ -127,4 +128,25 @@ public DynamicTableSink copy() { public String asSummaryString() { return "Doris Table Sink"; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DorisDynamicTableSink that = (DorisDynamicTableSink) o; + return Objects.equals(options, that.options) + && Objects.equals(readOptions, that.readOptions) + && Objects.equals(executionOptions, that.executionOptions) + && Objects.equals(tableSchema, that.tableSchema) + && Objects.equals(sinkParallelism, that.sinkParallelism); + } + + @Override + public int hashCode() { + return Objects.hash(options, readOptions, executionOptions, tableSchema, sinkParallelism); + } } diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSource.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSource.java index 7cb605876..9753361c5 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSource.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSource.java @@ -50,6 +50,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Objects; import java.util.stream.Collectors; /** The {@link DorisDynamicTableSource} is used during planning. */ @@ -212,4 +213,32 @@ public void applyProjection(int[][] projectedFields, DataType producedDataType) public List getResolvedFilterQuery() { return resolvedFilterQuery; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DorisDynamicTableSource that = (DorisDynamicTableSource) o; + return Objects.equals(options, that.options) + && Objects.equals(readOptions, that.readOptions) + && Objects.equals(lookupOptions, that.lookupOptions) + && Objects.equals(physicalSchema, that.physicalSchema) + && Objects.equals(resolvedFilterQuery, that.resolvedFilterQuery) + && Objects.equals(physicalRowDataType, that.physicalRowDataType); + } + + @Override + public int hashCode() { + return Objects.hash( + options, + readOptions, + lookupOptions, + physicalSchema, + resolvedFilterQuery, + physicalRowDataType); + } } diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisRowDataLookupFunction.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisRowDataLookupFunction.java deleted file mode 100644 index a86b53f41..000000000 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisRowDataLookupFunction.java +++ /dev/null @@ -1,164 +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.doris.flink.table; - -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.functions.FunctionContext; -import org.apache.flink.table.functions.TableFunction; -import org.apache.flink.table.types.DataType; - -import com.google.common.cache.Cache; -import com.google.common.cache.CacheBuilder; -import org.apache.doris.flink.cfg.DorisLookupOptions; -import org.apache.doris.flink.cfg.DorisOptions; -import org.apache.doris.flink.cfg.DorisReadOptions; -import org.apache.doris.flink.deserialization.converter.DorisRowConverter; -import org.apache.doris.flink.exception.DorisException; -import org.apache.doris.flink.rest.PartitionDefinition; -import org.apache.doris.flink.rest.RestService; -import org.apache.doris.flink.source.reader.DorisValueReader; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; -import java.util.StringJoiner; -import java.util.concurrent.TimeUnit; - -public class DorisRowDataLookupFunction extends TableFunction { - private static final Logger logger = LoggerFactory.getLogger(DorisRowDataLookupFunction.class); - - private final DorisOptions options; - private final DorisReadOptions readOptions; - private final String[] selectFields; - private final String[] conditionFields; - - private final long cacheMaxSize; - private final long cacheExpireMs; - private final int maxRetryTimes; - - private final DorisRowConverter rowConverter; - private transient Cache> cache; - - public DorisRowDataLookupFunction( - DorisOptions options, - DorisReadOptions readOptions, - DorisLookupOptions lookupOptions, - String[] selectFields, - DataType[] fieldTypes, - String[] conditionFields) { - this.options = options; - this.readOptions = readOptions; - this.selectFields = selectFields; - this.conditionFields = conditionFields; - this.cacheMaxSize = lookupOptions.getCacheMaxSize(); - this.cacheExpireMs = lookupOptions.getCacheExpireMs(); - this.maxRetryTimes = lookupOptions.getMaxRetryTimes(); - this.rowConverter = new DorisRowConverter(fieldTypes); - } - - @Override - public void open(FunctionContext context) throws Exception { - super.open(context); - this.cache = - cacheMaxSize == -1 || cacheExpireMs == -1 - ? null - : CacheBuilder.newBuilder() - .expireAfterWrite(cacheExpireMs, TimeUnit.MILLISECONDS) - .maximumSize(cacheMaxSize) - .build(); - } - - /** - * This is a lookup method which is called by Flink framework in runtime. - * - * @param keys lookup keys - */ - public void eval(Object... keys) { - RowData keyRow = GenericRowData.of(keys); - if (cache != null) { - List cachedRows = cache.getIfPresent(keyRow); - if (cachedRows != null) { - for (RowData cachedRow : cachedRows) { - collect(cachedRow); - } - return; - } - } - - List partitions = getPartitions(keys); - for (int retry = 0; retry <= maxRetryTimes; retry++) { - try { - ArrayList rows = new ArrayList<>(); - for (PartitionDefinition part : partitions) { - try (DorisValueReader valueReader = - new DorisValueReader(part, options, readOptions)) { - while (valueReader.hasNext()) { - List record = valueReader.next(); - GenericRowData rowData = rowConverter.convertInternal(record); - rows.add(rowData); - collect(rowData); - } - } - } - if (cache != null) { - rows.trimToSize(); - cache.put(keyRow, rows); - } - break; - } catch (Exception ex) { - logger.error(String.format("Read Doris error, retry times = %d", retry), ex); - if (retry >= maxRetryTimes) { - throw new RuntimeException("Read Doris failed.", ex); - } - try { - Thread.sleep(1000 * retry); - } catch (InterruptedException e1) { - throw new RuntimeException(e1); - } - } - } - } - - private List getPartitions(Object... keys) { - readOptions.setReadFields((String.join(",", selectFields))); - StringJoiner filter = new StringJoiner(" AND "); - for (int i = 0; i < keys.length && i < conditionFields.length; i++) { - filter.add(String.format("%s = '%s'", conditionFields[i], keys[i])); - } - readOptions.setFilterQuery(filter.toString()); - try { - return RestService.findPartitions(options, readOptions, logger); - } catch (DorisException ex) { - logger.error("Failed fetch doris partitions"); - return new ArrayList<>(); - } - } - - @Override - public void close() throws Exception { - super.close(); - } - - @VisibleForTesting - public Cache> getCache() { - return cache; - } -} diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/CdcTools.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/CdcTools.java index 8f8cabf54..7443ef8aa 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/CdcTools.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/CdcTools.java @@ -17,13 +17,14 @@ package org.apache.doris.flink.tools.cdc; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.java.utils.MultipleParameterTool; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; -import org.apache.doris.flink.tools.cdc.mongodb.serializer.MongoDBDatabaseSync; +import org.apache.doris.flink.tools.cdc.mongodb.MongoDBDatabaseSync; import org.apache.doris.flink.tools.cdc.mysql.MysqlDatabaseSync; import org.apache.doris.flink.tools.cdc.oracle.OracleDatabaseSync; import org.apache.doris.flink.tools.cdc.postgres.PostgresDatabaseSync; @@ -165,7 +166,8 @@ private static void syncDatabase( env.execute(jobName); } - private static Map getConfigMap(MultipleParameterTool params, String key) { + @VisibleForTesting + public static Map getConfigMap(MultipleParameterTool params, String key) { if (!params.has(key)) { System.out.println( "Can not find key [" diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoDBDatabaseSync.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/MongoDBDatabaseSync.java similarity index 98% rename from flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoDBDatabaseSync.java rename to flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/MongoDBDatabaseSync.java index eeb9d92c2..7c0b67067 100644 --- a/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/serializer/MongoDBDatabaseSync.java +++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/tools/cdc/mongodb/MongoDBDatabaseSync.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.flink.tools.cdc.mongodb.serializer; +package org.apache.doris.flink.tools.cdc.mongodb; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.configuration.ConfigOption; @@ -45,8 +45,7 @@ import org.apache.doris.flink.tools.cdc.DatabaseSync; import org.apache.doris.flink.tools.cdc.ParsingProcessFunction; import org.apache.doris.flink.tools.cdc.SourceSchema; -import org.apache.doris.flink.tools.cdc.mongodb.MongoDBSchema; -import org.apache.doris.flink.tools.cdc.mongodb.MongoParsingProcessFunction; +import org.apache.doris.flink.tools.cdc.mongodb.serializer.MongoDBJsonDebeziumSchemaSerializer; import org.bson.Document; import javax.annotation.Nullable; diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/DorisSinkITCase.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/DorisSinkITCase.java index d5a8173ef..39cd17bc4 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/DorisSinkITCase.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/DorisSinkITCase.java @@ -136,11 +136,23 @@ public void testTableSinkJsonFormat() throws Exception { + " 'table.identifier' = '%s'," + " 'username' = '%s'," + " 'password' = '%s'," + + " 'sink.buffer-size' = '1MB'," + + " 'sink.buffer-count' = '3'," + + " 'sink.max-retries' = '1'," + + " 'sink.enable-2pc' = 'true'," + + " 'sink.use-cache' = 'false'," + + " 'sink.enable-delete' = 'true'," + + " 'sink.ignore.update-before' = 'true'," + " 'sink.properties.format' = 'json'," + " 'sink.properties.read_json_by_line' = 'true'," - + " 'sink.label-prefix' = 'doris_sink'" + + " 'sink.label-prefix' = 'doris_sink" + + UUID.randomUUID() + + "'" + ")", - getFenodes(), DATABASE + "." + TABLE_JSON_TBL, USERNAME, PASSWORD); + getFenodes(), + DATABASE + "." + TABLE_JSON_TBL, + USERNAME, + PASSWORD); tEnv.executeSql(sinkDDL); tEnv.executeSql("INSERT INTO doris_sink SELECT 'doris',1 union all SELECT 'flink',2"); @@ -170,9 +182,19 @@ public void testTableBatch() throws Exception { + " 'table.identifier' = '%s'," + " 'username' = '%s'," + " 'password' = '%s'," - + " 'sink.enable.batch-mode' = 'true'" + + " 'sink.label-prefix' = '" + + UUID.randomUUID() + + "'," + + " 'sink.enable.batch-mode' = 'true'," + + " 'sink.flush.queue-size' = '2'," + + " 'sink.buffer-flush.max-rows' = '1000'," + + " 'sink.buffer-flush.max-bytes' = '10MB'," + + " 'sink.buffer-flush.interval' = '10s'" + ")", - getFenodes(), DATABASE + "." + TABLE_CSV_BATCH_TBL, USERNAME, PASSWORD); + getFenodes(), + DATABASE + "." + TABLE_CSV_BATCH_TBL, + USERNAME, + PASSWORD); tEnv.executeSql(sinkDDL); tEnv.executeSql("INSERT INTO doris_sink SELECT 'doris',1 union all SELECT 'flink',2"); diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestDorisCopyWriter.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestDorisCopyWriter.java index 4d7b8f631..04cb63822 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestDorisCopyWriter.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/copy/TestDorisCopyWriter.java @@ -100,6 +100,7 @@ public void testPrepareCommit() throws Exception { committable .getCopySQL() .startsWith("COPY INTO `db`.`table` FROM @~('{label_table_0_1_0}')")); + copyWriter.close(); } @Test @@ -124,5 +125,6 @@ public void testSnapshot() throws Exception { copyWriter.setBatchStageLoad(stageLoad); List writerStates = copyWriter.snapshotState(1); Assert.assertTrue(writerStates.isEmpty()); + copyWriter.close(); } } diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/writer/serializer/jsondebezium/TestJsonDebeziumSchemaChangeImplV2.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/writer/serializer/jsondebezium/TestJsonDebeziumSchemaChangeImplV2.java index 2c0e91740..a3c24d150 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/writer/serializer/jsondebezium/TestJsonDebeziumSchemaChangeImplV2.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/sink/writer/serializer/jsondebezium/TestJsonDebeziumSchemaChangeImplV2.java @@ -347,6 +347,31 @@ public void testBuildOracle2DorisTypeName() throws IOException { JsonNode columns = objectMapper.readTree(columnInfo); String dorisTypeName = schemaChange.buildDorisTypeName(columns); Assert.assertEquals(dorisTypeName, "VARCHAR(384)"); + columnInfo = + "{\"name\":\"NAME\",\"jdbcType\":12,\"nativeType\":null,\"typeName\":\"FLOAT\",\"typeExpression\":\"FLOAT\",\"charsetName\":null,\"length\":0,\"scale\":null,\"position\":2,\"optional\":false,\"autoIncremented\":false,\"generated\":false,\"comment\":null}"; + columns = objectMapper.readTree(columnInfo); + dorisTypeName = schemaChange.buildDorisTypeName(columns); + Assert.assertEquals(dorisTypeName, "DOUBLE"); + } + + @Test + public void testBuildPostgres2DorisTypeName() throws IOException { + String columnInfo = + "{\"name\":\"NAME\",\"jdbcType\":12,\"nativeType\":null,\"typeName\":\"VARCHAR\",\"typeExpression\":\"VARCHAR\",\"charsetName\":null,\"length\":128,\"scale\":null,\"position\":2,\"optional\":false,\"autoIncremented\":false,\"generated\":false,\"comment\":null}"; + schemaChange.setSourceConnector("postgres"); + JsonNode columns = objectMapper.readTree(columnInfo); + String dorisTypeName = schemaChange.buildDorisTypeName(columns); + Assert.assertEquals(dorisTypeName, "VARCHAR(384)"); + } + + @Test + public void testBuildSqlserver2DorisTypeName() throws IOException { + String columnInfo = + "{\"name\":\"NAME\",\"jdbcType\":12,\"nativeType\":null,\"typeName\":\"VARCHAR\",\"typeExpression\":\"VARCHAR\",\"charsetName\":null,\"length\":128,\"scale\":null,\"position\":2,\"optional\":false,\"autoIncremented\":false,\"generated\":false,\"comment\":null}"; + schemaChange.setSourceConnector("sqlserver"); + JsonNode columns = objectMapper.readTree(columnInfo); + String dorisTypeName = schemaChange.buildDorisTypeName(columns); + Assert.assertEquals(dorisTypeName, "VARCHAR(384)"); } @Test diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/source/DorisSourceITCase.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/source/DorisSourceITCase.java index 72ecb5c0d..e357bffb1 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/source/DorisSourceITCase.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/source/DorisSourceITCase.java @@ -49,6 +49,7 @@ public class DorisSourceITCase extends DorisTestBase { static final String TABLE_READ_OLD_API = "tbl_read_old_api"; static final String TABLE_READ_TBL = "tbl_read_tbl"; static final String TABLE_READ_TBL_OLD_API = "tbl_read_tbl_old_api"; + static final String TABLE_READ_TBL_ALL_OPTIONS = "tbl_read_tbl_all_options"; @Test public void testSource() throws Exception { @@ -175,6 +176,52 @@ public void testTableSourceOldApi() throws Exception { Assert.assertArrayEquals(expected, actual.toArray()); } + @Test + public void testTableSourceAllOptions() throws Exception { + initializeTable(TABLE_READ_TBL_ALL_OPTIONS); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + + String sourceDDL = + String.format( + "CREATE TABLE doris_source (" + + " name STRING," + + " age INT" + + ") WITH (" + + " 'connector' = 'doris'," + + " 'fenodes' = '%s'," + + " 'table.identifier' = '%s'," + + " 'source.use-old-api' = 'true'," + + " 'username' = '%s'," + + " 'password' = '%s'," + + " 'doris.request.retries' = '3'," + + " 'doris.request.connect.timeout' = '60s'," + + " 'doris.request.read.timeout' = '60s'," + + " 'doris.request.query.timeout' = '3600s'," + + " 'doris.request.tablet.size' = '2'," + + " 'doris.batch.size' = '1024'," + + " 'doris.exec.mem.limit' = '2048mb'," + + " 'doris.deserialize.arrow.async' = 'true'," + + " 'doris.deserialize.queue.size' = '32'" + + ")", + getFenodes(), + DATABASE + "." + TABLE_READ_TBL_ALL_OPTIONS, + USERNAME, + PASSWORD); + tEnv.executeSql(sourceDDL); + TableResult tableResult = tEnv.executeSql("SELECT * FROM doris_source"); + + List actual = new ArrayList<>(); + try (CloseableIterator iterator = tableResult.collect()) { + while (iterator.hasNext()) { + actual.add(iterator.next().toString()); + } + } + String[] expected = new String[] {"+I[doris, 18]", "+I[flink, 10]"}; + Assert.assertArrayEquals(expected, actual.toArray()); + } + private void initializeTable(String table) throws Exception { try (Connection connection = DriverManager.getConnection( diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisDynamicTableFactoryTest.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisDynamicTableFactoryTest.java new file mode 100644 index 000000000..144183613 --- /dev/null +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisDynamicTableFactoryTest.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.doris.flink.table; + +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; + +import org.apache.doris.flink.cfg.DorisExecutionOptions; +import org.apache.doris.flink.cfg.DorisLookupOptions; +import org.apache.doris.flink.cfg.DorisOptions; +import org.apache.doris.flink.cfg.DorisReadOptions; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_BATCH_SIZE_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_DESERIALIZE_ARROW_ASYNC_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_DESERIALIZE_QUEUE_SIZE_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_EXEC_MEM_LIMIT_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_REQUEST_CONNECT_TIMEOUT_MS_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_REQUEST_QUERY_TIMEOUT_S_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_REQUEST_READ_TIMEOUT_MS_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_REQUEST_RETRIES_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_TABLET_SIZE_DEFAULT; +import static org.apache.doris.flink.utils.FactoryMocks.SCHEMA; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; +import static org.junit.Assert.assertEquals; + +public class DorisDynamicTableFactoryTest { + + @Test + public void testDorisSourceProperties() { + Map properties = getAllOptions(); + properties.put("doris.request.query.timeout", "21600s"); + properties.put("doris.request.tablet.size", "1"); + properties.put("doris.batch.size", "1024"); + properties.put("doris.exec.mem.limit", "8192mb"); + properties.put("doris.deserialize.arrow.async", "false"); + properties.put("doris.deserialize.queue.size", "64"); + + properties.put("lookup.cache.max-rows", "100"); + properties.put("lookup.cache.ttl", "20s"); + properties.put("lookup.max-retries", "1"); + properties.put("lookup.jdbc.async", "true"); + properties.put("lookup.jdbc.read.batch.size", "16"); + properties.put("lookup.jdbc.read.batch.queue-size", "16"); + properties.put("lookup.jdbc.read.thread-size", "1"); + + DynamicTableSource actual = createTableSource(SCHEMA, properties); + DorisOptions options = + DorisOptions.builder() + .setTableIdentifier("db.tbl") + .setFenodes("127.0.0.1:8030") + .setBenodes("127.0.0.1:8040") + .setAutoRedirect(true) + .setUsername("root") + .setPassword("") + .setJdbcUrl("jdbc:mysql://127.0.0.1:9030") + .build(); + DorisLookupOptions lookupOptions = + DorisLookupOptions.builder() + .setCacheExpireMs(20000) + .setCacheMaxSize(100) + .setAsync(true) + .setJdbcReadBatchQueueSize(16) + .setJdbcReadBatchSize(16) + .setJdbcReadThreadSize(1) + .setMaxRetryTimes(1) + .build(); + + final DorisReadOptions.Builder readOptionBuilder = DorisReadOptions.builder(); + readOptionBuilder + .setDeserializeArrowAsync(DORIS_DESERIALIZE_ARROW_ASYNC_DEFAULT) + .setDeserializeQueueSize(DORIS_DESERIALIZE_QUEUE_SIZE_DEFAULT) + .setExecMemLimit(DORIS_EXEC_MEM_LIMIT_DEFAULT) + .setRequestQueryTimeoutS(DORIS_REQUEST_QUERY_TIMEOUT_S_DEFAULT) + .setRequestBatchSize(DORIS_BATCH_SIZE_DEFAULT) + .setRequestConnectTimeoutMs(DORIS_REQUEST_CONNECT_TIMEOUT_MS_DEFAULT) + .setRequestReadTimeoutMs(DORIS_REQUEST_READ_TIMEOUT_MS_DEFAULT) + .setRequestRetries(DORIS_REQUEST_RETRIES_DEFAULT) + .setRequestTabletSize(DORIS_TABLET_SIZE_DEFAULT); + DorisDynamicTableSource expected = + new DorisDynamicTableSource( + options, + readOptionBuilder.build(), + lookupOptions, + TableSchema.fromResolvedSchema(SCHEMA), + SCHEMA.toPhysicalRowDataType()); + + assertEquals(actual, expected); + } + + @Test + public void testDorisSinkProperties() { + Map properties = getAllOptions(); + properties.put("doris.request.query.timeout", "21600s"); + properties.put("doris.request.tablet.size", "1"); + properties.put("doris.batch.size", "1024"); + properties.put("doris.exec.mem.limit", "8192mb"); + properties.put("doris.deserialize.arrow.async", "false"); + properties.put("doris.deserialize.queue.size", "64"); + + properties.put("sink.label-prefix", "abc"); + properties.put("sink.properties.format", "json"); + properties.put("sink.properties.read_json_by_line", "true"); + properties.put("sink.enable-delete", "true"); + properties.put("sink.enable-2pc", "true"); + properties.put("sink.buffer-size", "1MB"); + properties.put("sink.buffer-count", "3"); + properties.put("sink.max-retries", "1"); + properties.put("sink.check-interval", "10s"); + properties.put("sink.use-cache", "true"); + properties.put("sink.enable.batch-mode", "true"); + properties.put("sink.flush.queue-size", "2"); + properties.put("sink.buffer-flush.max-rows", "1000"); + properties.put("sink.buffer-flush.max-bytes", "10MB"); + properties.put("sink.buffer-flush.interval", "10s"); + properties.put("sink.ignore.update-before", "true"); + properties.put("sink.ignore.commit-error", "false"); + properties.put("sink.parallelism", "1"); + + DynamicTableSink actual = createTableSink(SCHEMA, properties); + DorisOptions options = + DorisOptions.builder() + .setTableIdentifier("db.tbl") + .setFenodes("127.0.0.1:8030") + .setBenodes("127.0.0.1:8040") + .setAutoRedirect(true) + .setUsername("root") + .setPassword("") + .setJdbcUrl("jdbc:mysql://127.0.0.1:9030") + .build(); + + Properties prop = new Properties(); + prop.put("format", "json"); + prop.put("read_json_by_line", "true"); + DorisExecutionOptions executionOptions = + DorisExecutionOptions.builder() + .setLabelPrefix("abc") + .setBufferCount(3) + .setBufferSize(1024 * 1024) + .setStreamLoadProp(prop) + .setMaxRetries(1) + .setCheckInterval(10000) + .setBatchMode(true) + .enable2PC() + .setBufferFlushIntervalMs(10000) + .setBufferFlushMaxBytes(10 * 1024 * 1024) + .setBufferFlushMaxRows(1000) + .setFlushQueueSize(2) + .setUseCache(true) + .setIgnoreCommitError(false) + .build(); + + final DorisReadOptions.Builder readOptionBuilder = DorisReadOptions.builder(); + readOptionBuilder + .setDeserializeArrowAsync(DORIS_DESERIALIZE_ARROW_ASYNC_DEFAULT) + .setDeserializeQueueSize(DORIS_DESERIALIZE_QUEUE_SIZE_DEFAULT) + .setExecMemLimit(DORIS_EXEC_MEM_LIMIT_DEFAULT) + .setRequestQueryTimeoutS(DORIS_REQUEST_QUERY_TIMEOUT_S_DEFAULT) + .setRequestBatchSize(DORIS_BATCH_SIZE_DEFAULT) + .setRequestConnectTimeoutMs(DORIS_REQUEST_CONNECT_TIMEOUT_MS_DEFAULT) + .setRequestReadTimeoutMs(DORIS_REQUEST_READ_TIMEOUT_MS_DEFAULT) + .setRequestRetries(DORIS_REQUEST_RETRIES_DEFAULT) + .setRequestTabletSize(DORIS_TABLET_SIZE_DEFAULT); + DorisDynamicTableSink expected = + new DorisDynamicTableSink( + options, + readOptionBuilder.build(), + executionOptions, + TableSchema.fromResolvedSchema(SCHEMA), + 1); + + assertEquals(actual, expected); + } + + private Map getAllOptions() { + Map options = new HashMap<>(); + options.put("connector", "doris"); + options.put("fenodes", "127.0.0.1:8030"); + options.put("benodes", "127.0.0.1:8040"); + options.put("jdbc-url", "jdbc:mysql://127.0.0.1:9030"); + options.put("table.identifier", "db.tbl"); + options.put("username", "root"); + options.put("password", ""); + options.put("auto-redirect", "true"); + options.put("doris.request.retries", "3"); + options.put("doris.request.connect.timeout", "30s"); + options.put("doris.request.read.timeout", "30s"); + return options; + } +} diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisDynamicTableSourceTest.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisDynamicTableSourceTest.java index 45d01cf8b..9063c73bf 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisDynamicTableSourceTest.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisDynamicTableSourceTest.java @@ -51,6 +51,7 @@ import java.util.Collections; import java.util.HashSet; +import static org.apache.doris.flink.utils.FactoryMocks.SCHEMA; import static org.apache.flink.table.expressions.ApiExpressionUtils.valueLiteral; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; @@ -70,7 +71,7 @@ public void testDorisUseNewApi() { OptionUtils.buildDorisOptions(), builder.build(), DorisLookupOptions.builder().build(), - TableSchema.fromResolvedSchema(FactoryMocks.SCHEMA), + TableSchema.fromResolvedSchema(SCHEMA), FactoryMocks.PHYSICAL_DATA_TYPE); ScanTableSource.ScanRuntimeProvider provider = actualDorisSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); @@ -84,7 +85,7 @@ public void testDorisUseNewApiDefault() { OptionUtils.buildDorisOptions(), OptionUtils.buildDorisReadOptions(), DorisLookupOptions.builder().build(), - TableSchema.fromResolvedSchema(FactoryMocks.SCHEMA), + TableSchema.fromResolvedSchema(SCHEMA), FactoryMocks.PHYSICAL_DATA_TYPE); ScanTableSource.ScanRuntimeProvider provider = actualDorisSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); @@ -106,7 +107,7 @@ public void testDorisUseOldApi() { OptionUtils.buildDorisOptions(), builder.build(), DorisLookupOptions.builder().build(), - TableSchema.fromResolvedSchema(FactoryMocks.SCHEMA), + TableSchema.fromResolvedSchema(SCHEMA), FactoryMocks.PHYSICAL_DATA_TYPE); ScanTableSource.ScanRuntimeProvider provider = actualDorisSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); @@ -150,7 +151,7 @@ public void testApplyProjection() { OptionUtils.buildDorisOptions(), readOptions, DorisLookupOptions.builder().build(), - TableSchema.fromResolvedSchema(FactoryMocks.SCHEMA), + TableSchema.fromResolvedSchema(SCHEMA), FactoryMocks.PHYSICAL_DATA_TYPE); actualDorisSource.applyProjection(projectionArray, projectionDataType); Assert.assertEquals(readOptions.getReadFields(), "`a`, `c`"); @@ -164,7 +165,7 @@ public void testFilter() { OptionUtils.buildDorisOptions(), readOptions, DorisLookupOptions.builder().build(), - TableSchema.fromResolvedSchema(FactoryMocks.SCHEMA), + TableSchema.fromResolvedSchema(SCHEMA), FactoryMocks.PHYSICAL_DATA_TYPE); ResolvedExpression aRef = new FieldReferenceExpression("a", DataTypes.STRING(), 0, 2); ResolvedExpression aRefCharLength = diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataLookupFunctionTest.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataJdbcLookupFunctionTest.java similarity index 59% rename from flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataLookupFunctionTest.java rename to flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataJdbcLookupFunctionTest.java index 423704fae..d12a06dfc 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataLookupFunctionTest.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataJdbcLookupFunctionTest.java @@ -21,16 +21,20 @@ import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; +import org.apache.flink.table.functions.FunctionContext; import org.apache.flink.table.types.DataType; import org.apache.flink.util.Collector; import com.google.common.cache.Cache; +import org.apache.doris.flink.DorisTestBase; import org.apache.doris.flink.cfg.DorisLookupOptions; import org.apache.doris.flink.cfg.DorisOptions; -import org.apache.doris.flink.cfg.DorisReadOptions; -import org.junit.Ignore; +import org.junit.Before; import org.junit.Test; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.Statement; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -39,10 +43,8 @@ import static org.junit.Assert.assertEquals; -@Ignore -public class DorisRowDataLookupFunctionTest { +public class DorisRowDataJdbcLookupFunctionTest extends DorisTestBase { - private static final String TEST_FENODES = "127.0.0.1:8030"; private static final String LOOKUP_TABLE = "test.t_lookup_table"; private static String[] fieldNames = new String[] {"id1", "id2", "c_string", "c_double"}; @@ -51,17 +53,54 @@ public class DorisRowDataLookupFunctionTest { DataTypes.INT(), DataTypes.STRING(), DataTypes.STRING(), DataTypes.DOUBLE() }; private static String[] lookupKeys = new String[] {"id1", "id2"}; + private static int[] keyIndexs = new int[] {0, 1}; + + @Before + public void setUp() throws Exception { + try (Connection connection = + DriverManager.getConnection( + String.format(URL, DORIS_CONTAINER.getHost()), USERNAME, PASSWORD); + Statement statement = connection.createStatement()) { + statement.execute(String.format("CREATE DATABASE IF NOT EXISTS %s", "test")); + statement.execute(String.format("DROP TABLE IF EXISTS %s", LOOKUP_TABLE)); + statement.execute( + String.format( + "CREATE TABLE %s ( \n" + + "`id1` int,\n" + + "`id2` varchar(128),\n" + + "`c_string` string,\n" + + "`c_double` double\n" + + ") DISTRIBUTED BY HASH(`id1`) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ")\n", + LOOKUP_TABLE)); + statement.execute( + String.format( + "insert into %s values (1,'A','zhangsanA',1.12)," + + "(1,'A','zhangsanA-1',11.12)," + + "(2,'B','zhangsanB',2.12),(4,'D','zhangsanD',4.12)", + LOOKUP_TABLE)); + } + } @Test public void testEval() throws Exception { - - DorisLookupOptions lookupOptions = DorisLookupOptions.builder().build(); - DorisRowDataLookupFunction lookupFunction = buildRowDataLookupFunction(lookupOptions); + DorisLookupOptions lookupOptions = + DorisLookupOptions.builder() + .setJdbcReadBatchQueueSize(16) + .setJdbcReadBatchSize(16) + .setJdbcReadThreadSize(1) + .setMaxRetryTimes(1) + .build(); + DorisRowDataJdbcLookupFunction lookupFunction = + buildRowDataJdbcLookupFunction(lookupOptions); ListOutputCollector collector = new ListOutputCollector(); lookupFunction.setCollector(collector); - lookupFunction.open(null); + FunctionContext context = new FunctionContext(null, null, null); + lookupFunction.open(context); lookupFunction.eval(1, StringData.fromString("A")); lookupFunction.eval(2, StringData.fromString("B")); @@ -81,19 +120,25 @@ public void testEval() throws Exception { @Test public void testEvalWithCache() throws Exception { - long cacheExpireMs = 10000; + long cacheExpireMs = 20000; DorisLookupOptions lookupOptions = DorisLookupOptions.builder() .setCacheExpireMs(cacheExpireMs) .setCacheMaxSize(10) + .setJdbcReadBatchQueueSize(16) + .setJdbcReadBatchSize(16) + .setJdbcReadThreadSize(1) + .setMaxRetryTimes(1) .build(); - DorisRowDataLookupFunction lookupFunction = buildRowDataLookupFunction(lookupOptions); + DorisRowDataJdbcLookupFunction lookupFunction = + buildRowDataJdbcLookupFunction(lookupOptions); ListOutputCollector collector = new ListOutputCollector(); lookupFunction.setCollector(collector); - lookupFunction.open(null); + FunctionContext context = new FunctionContext(null, null, null); + lookupFunction.open(context); lookupFunction.eval(4, StringData.fromString("D")); lookupFunction.eval(5, StringData.fromString("5")); @@ -116,26 +161,25 @@ public void testEvalWithCache() throws Exception { assert cache.getIfPresent(keyRow) == null; } - private DorisRowDataLookupFunction buildRowDataLookupFunction( + private DorisRowDataJdbcLookupFunction buildRowDataJdbcLookupFunction( DorisLookupOptions lookupOptions) { DorisOptions dorisOptions = DorisOptions.builder() - .setFenodes(TEST_FENODES) + .setFenodes(getFenodes()) .setTableIdentifier(LOOKUP_TABLE) - .setUsername("root") - .setPassword("") + .setJdbcUrl(getJdbcUrl()) + .setUsername(USERNAME) + .setPassword(PASSWORD) .build(); - DorisReadOptions readOptions = DorisReadOptions.builder().build(); - - DorisRowDataLookupFunction lookupFunction = - new DorisRowDataLookupFunction( + DorisRowDataJdbcLookupFunction lookupFunction = + new DorisRowDataJdbcLookupFunction( dorisOptions, - readOptions, lookupOptions, fieldNames, fieldDataTypes, - lookupKeys); + lookupKeys, + keyIndexs); return lookupFunction; } diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/CdcMongoSyncDatabaseCase.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/CdcMongoSyncDatabaseCase.java index e247e4298..ef66e8825 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/CdcMongoSyncDatabaseCase.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/CdcMongoSyncDatabaseCase.java @@ -20,7 +20,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.doris.flink.tools.cdc.mongodb.serializer.MongoDBDatabaseSync; +import org.apache.doris.flink.tools.cdc.mongodb.MongoDBDatabaseSync; import java.util.HashMap; import java.util.Map; diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/CdcToolsTest.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/CdcToolsTest.java new file mode 100644 index 000000000..5944ba52a --- /dev/null +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/CdcToolsTest.java @@ -0,0 +1,52 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.flink.tools.cdc; + +import org.apache.flink.api.java.utils.MultipleParameterTool; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +public class CdcToolsTest { + + @Test + public void getConfigMapTest() { + MultipleParameterTool params = + MultipleParameterTool.fromArgs( + new String[] { + "--sink-conf", "fenodes=127.0.0.1:8030", "--sink-conf", "password=" + }); + Map sinkConf = CdcTools.getConfigMap(params, "sink-conf"); + + Map excepted = new HashMap<>(); + excepted.put("password", ""); + excepted.put("fenodes", "127.0.0.1:8030"); + Assert.assertEquals(sinkConf, excepted); + + Map mysqlConf = CdcTools.getConfigMap(params, "--mysql-conf"); + Assert.assertNull(mysqlConf); + + MultipleParameterTool params2 = + MultipleParameterTool.fromArgs(new String[] {"--sink-conf", "fenodes"}); + Map sinkConf2 = CdcTools.getConfigMap(params2, "sink-conf"); + Assert.assertNull(sinkConf2); + } +} diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/DorisDorisE2ECase.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/DorisDorisE2ECase.java index 8f7998c4e..918223ce3 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/DorisDorisE2ECase.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/DorisDorisE2ECase.java @@ -33,6 +33,7 @@ import java.sql.Statement; import java.util.ArrayList; import java.util.List; +import java.util.UUID; /** DorisDorisE2ECase. */ public class DorisDorisE2ECase extends DorisTestBase { @@ -58,10 +59,16 @@ public void testDoris2Doris() throws Exception { + " 'connector' = 'doris'," + " 'fenodes' = '%s'," + " 'table.identifier' = '%s'," + + " 'sink.label-prefix' = '" + + UUID.randomUUID() + + "'," + " 'username' = '%s'," + " 'password' = '%s'" + ")", - getFenodes(), DATABASE_SOURCE + "." + TABLE, USERNAME, PASSWORD); + getFenodes(), + DATABASE_SOURCE + "." + TABLE, + USERNAME, + PASSWORD); tEnv.executeSql(sourceDDL); String sinkDDL = @@ -72,11 +79,17 @@ public void testDoris2Doris() throws Exception { + ") WITH (" + " 'connector' = 'doris'," + " 'fenodes' = '%s'," + + " 'sink.label-prefix' = '" + + UUID.randomUUID() + + "'," + " 'table.identifier' = '%s'," + " 'username' = '%s'," + " 'password' = '%s'" + ")", - getFenodes(), DATABASE_SINK + "." + TABLE, USERNAME, PASSWORD); + getFenodes(), + DATABASE_SINK + "." + TABLE, + USERNAME, + PASSWORD); tEnv.executeSql(sinkDDL); tEnv.executeSql("INSERT INTO doris_sink SELECT * FROM doris_source").await(); diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/MySQLDorisE2ECase.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/MySQLDorisE2ECase.java index 3c1206117..0c3e5966d 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/MySQLDorisE2ECase.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/tools/cdc/MySQLDorisE2ECase.java @@ -90,6 +90,7 @@ public static void stopMySQLContainers() { public void testMySQL2Doris() throws Exception { printClusterStatus(); initializeMySQLTable(); + initializeDorisTable(); JobClient jobClient = submitJob(); // wait 2 times checkpoint Thread.sleep(20000); @@ -242,6 +243,104 @@ public void testAutoAddTable() throws Exception { jobClient.cancel().get(); } + @Test + public void testMySQL2DorisByDefault() throws Exception { + printClusterStatus(); + initializeMySQLTable(); + initializeDorisTable(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRestartStrategy(RestartStrategies.noRestart()); + Map flinkMap = new HashMap<>(); + flinkMap.put("execution.checkpointing.interval", "10s"); + flinkMap.put("pipeline.operator-chaining", "false"); + flinkMap.put("parallelism.default", "1"); + + Configuration configuration = Configuration.fromMap(flinkMap); + env.configure(configuration); + + String database = DATABASE; + Map mysqlConfig = new HashMap<>(); + mysqlConfig.put("database-name", DATABASE); + mysqlConfig.put("hostname", MYSQL_CONTAINER.getHost()); + mysqlConfig.put("port", MYSQL_CONTAINER.getMappedPort(3306) + ""); + mysqlConfig.put("username", MYSQL_USER); + mysqlConfig.put("password", MYSQL_PASSWD); + mysqlConfig.put("server-time-zone", "Asia/Shanghai"); + Configuration config = Configuration.fromMap(mysqlConfig); + + Map sinkConfig = new HashMap<>(); + sinkConfig.put("fenodes", getFenodes()); + sinkConfig.put("username", USERNAME); + sinkConfig.put("password", PASSWORD); + sinkConfig.put("jdbc-url", String.format(DorisTestBase.URL, DORIS_CONTAINER.getHost())); + sinkConfig.put("sink.label-prefix", UUID.randomUUID().toString()); + Configuration sinkConf = Configuration.fromMap(sinkConfig); + + Map tableConfig = new HashMap<>(); + tableConfig.put("replication_num", "1"); + + String includingTables = "tbl1|tbl2|tbl3"; + String excludingTables = ""; + DatabaseSync databaseSync = new MysqlDatabaseSync(); + databaseSync + .setEnv(env) + .setDatabase(database) + .setConfig(config) + .setIncludingTables(includingTables) + .setExcludingTables(excludingTables) + .setIgnoreDefaultValue(false) + .setSinkConfig(sinkConf) + .setTableConfig(tableConfig) + .setCreateTableOnly(false) + .setNewSchemaChange(true) + // no single sink + .setSingleSink(false) + .create(); + databaseSync.build(); + JobClient jobClient = env.executeAsync(); + waitForJobStatus( + jobClient, + Collections.singletonList(RUNNING), + Deadline.fromNow(Duration.ofSeconds(10))); + + // wait 2 times checkpoint + Thread.sleep(20000); + List expected = Arrays.asList("doris_1,1", "doris_2,2", "doris_3,3"); + String sql = + "select * from ( select * from %s.%s union all select * from %s.%s union all select * from %s.%s ) res order by 1"; + String query1 = String.format(sql, DATABASE, TABLE_1, DATABASE, TABLE_2, DATABASE, TABLE_3); + checkResult(expected, query1, 2); + + // add incremental data + try (Connection connection = + DriverManager.getConnection( + MYSQL_CONTAINER.getJdbcUrl(), MYSQL_USER, MYSQL_PASSWD); + Statement statement = connection.createStatement()) { + statement.execute( + String.format("insert into %s.%s values ('doris_1_1',10)", DATABASE, TABLE_1)); + statement.execute( + String.format("insert into %s.%s values ('doris_2_1',11)", DATABASE, TABLE_2)); + statement.execute( + String.format("insert into %s.%s values ('doris_3_1',12)", DATABASE, TABLE_3)); + + statement.execute( + String.format( + "update %s.%s set age=18 where name='doris_1'", DATABASE, TABLE_1)); + statement.execute( + String.format("delete from %s.%s where name='doris_2'", DATABASE, TABLE_2)); + } + + Thread.sleep(20000); + List expected2 = + Arrays.asList( + "doris_1,18", "doris_1_1,10", "doris_2_1,11", "doris_3,3", "doris_3_1,12"); + sql = + "select * from ( select * from %s.%s union all select * from %s.%s union all select * from %s.%s ) res order by 1"; + String query2 = String.format(sql, DATABASE, TABLE_1, DATABASE, TABLE_2, DATABASE, TABLE_3); + checkResult(expected2, query2, 2); + jobClient.cancel().get(); + } + private void initializeDorisTable() throws Exception { try (Connection connection = DriverManager.getConnection( @@ -302,6 +401,7 @@ public JobClient submitJob() throws Exception { .setCreateTableOnly(false) .setNewSchemaChange(true) .setSingleSink(true) + .setIgnoreDefaultValue(true) .create(); databaseSync.build(); JobClient jobClient = env.executeAsync(); @@ -343,6 +443,7 @@ public void initializeMySQLTable() throws Exception { statement.execute(String.format("DROP TABLE IF EXISTS %s.%s", DATABASE, TABLE_1)); statement.execute(String.format("DROP TABLE IF EXISTS %s.%s", DATABASE, TABLE_2)); statement.execute(String.format("DROP TABLE IF EXISTS %s.%s", DATABASE, TABLE_3)); + statement.execute(String.format("DROP TABLE IF EXISTS %s.%s", DATABASE, TABLE_4)); statement.execute( String.format( "CREATE TABLE %s.%s ( \n" From 314e00794b8ea57dd47ec2b5a1fde0ff10535522 Mon Sep 17 00:00:00 2001 From: wudi <676366545@qq.com> Date: Wed, 29 May 2024 20:22:51 +0800 Subject: [PATCH 4/5] update --- .../doris/flink/table/DorisRowDataJdbcLookupFunctionTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataJdbcLookupFunctionTest.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataJdbcLookupFunctionTest.java index d12a06dfc..80a51faa2 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataJdbcLookupFunctionTest.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataJdbcLookupFunctionTest.java @@ -99,7 +99,7 @@ public void testEval() throws Exception { ListOutputCollector collector = new ListOutputCollector(); lookupFunction.setCollector(collector); - FunctionContext context = new FunctionContext(null, null, null); + FunctionContext context = new FunctionContext(null); lookupFunction.open(context); lookupFunction.eval(1, StringData.fromString("A")); @@ -137,7 +137,7 @@ public void testEvalWithCache() throws Exception { ListOutputCollector collector = new ListOutputCollector(); lookupFunction.setCollector(collector); - FunctionContext context = new FunctionContext(null, null, null); + FunctionContext context = new FunctionContext(null); lookupFunction.open(context); lookupFunction.eval(4, StringData.fromString("D")); From 55bdf5db8024e32ba72fc8aaba832ae2230f696b Mon Sep 17 00:00:00 2001 From: wudi <676366545@qq.com> Date: Wed, 29 May 2024 20:33:36 +0800 Subject: [PATCH 5/5] change name --- .../catalog/{DorisCatalogTest.java => DorisCatalogITCase.java} | 2 +- ...ctionTest.java => DorisRowDataJdbcLookupFunctionITCase.java} | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) rename flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/{DorisCatalogTest.java => DorisCatalogITCase.java} (99%) rename flink-doris-connector/src/test/java/org/apache/doris/flink/table/{DorisRowDataJdbcLookupFunctionTest.java => DorisRowDataJdbcLookupFunctionITCase.java} (99%) diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogTest.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogITCase.java similarity index 99% rename from flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogTest.java rename to flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogITCase.java index f4995fe1c..3016f581b 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogTest.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/catalog/DorisCatalogITCase.java @@ -60,7 +60,7 @@ import static org.junit.Assert.assertTrue; /** Class for unit tests to run on catalogs. */ -public class DorisCatalogTest extends DorisTestBase { +public class DorisCatalogITCase extends DorisTestBase { private static final String TEST_CATALOG_NAME = "doris_catalog"; private static final String TEST_FENODES = getFenodes(); private static final String TEST_JDBCURL = getJdbcUrl(); diff --git a/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataJdbcLookupFunctionTest.java b/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataJdbcLookupFunctionITCase.java similarity index 99% rename from flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataJdbcLookupFunctionTest.java rename to flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataJdbcLookupFunctionITCase.java index 80a51faa2..0ad1781ae 100644 --- a/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataJdbcLookupFunctionTest.java +++ b/flink-doris-connector/src/test/java/org/apache/doris/flink/table/DorisRowDataJdbcLookupFunctionITCase.java @@ -43,7 +43,7 @@ import static org.junit.Assert.assertEquals; -public class DorisRowDataJdbcLookupFunctionTest extends DorisTestBase { +public class DorisRowDataJdbcLookupFunctionITCase extends DorisTestBase { private static final String LOOKUP_TABLE = "test.t_lookup_table";