diff --git a/bitsail-base/src/main/java/com/bytedance/bitsail/base/catalog/TableCatalogFactory.java b/bitsail-base/src/main/java/com/bytedance/bitsail/base/catalog/TableCatalogFactory.java new file mode 100644 index 000000000..aae721477 --- /dev/null +++ b/bitsail-base/src/main/java/com/bytedance/bitsail/base/catalog/TableCatalogFactory.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 com.bytedance.bitsail.base.catalog; + +import com.bytedance.bitsail.base.component.ComponentBuilder; +import com.bytedance.bitsail.base.connector.BuilderGroup; +import com.bytedance.bitsail.base.execution.ExecutionEnviron; +import com.bytedance.bitsail.common.catalog.table.TableCatalog; +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; + +import java.io.Serializable; + +/** + * Created 2022/5/23 + */ +public interface TableCatalogFactory extends Serializable, ComponentBuilder { + + /** + * Create a table catalog. + * + * @param executionEnviron execution environment + * @param connectorConfiguration configuration for the reader/writer + */ + TableCatalog createTableCatalog(BuilderGroup builderGroup, + ExecutionEnviron executionEnviron, + BitSailConfiguration connectorConfiguration); + +} diff --git a/bitsail-base/src/main/java/com/bytedance/bitsail/base/catalog/TableCatalogFactoryHelper.java b/bitsail-base/src/main/java/com/bytedance/bitsail/base/catalog/TableCatalogFactoryHelper.java new file mode 100644 index 000000000..c84ecb736 --- /dev/null +++ b/bitsail-base/src/main/java/com/bytedance/bitsail/base/catalog/TableCatalogFactoryHelper.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.base.catalog; + +import com.bytedance.bitsail.base.component.DefaultComponentBuilderLoader; + +import org.apache.commons.lang3.StringUtils; + +public class TableCatalogFactoryHelper { + + public static TableCatalogFactory getTableCatalogFactory(String connectorName) { + DefaultComponentBuilderLoader loader = + new DefaultComponentBuilderLoader<>(TableCatalogFactory.class); + + return loader.loadComponent(StringUtils.lowerCase(connectorName), false); + } +} diff --git a/bitsail-base/src/main/java/com/bytedance/bitsail/base/component/DefaultComponentBuilderLoader.java b/bitsail-base/src/main/java/com/bytedance/bitsail/base/component/DefaultComponentBuilderLoader.java index 9df07b145..35caa25aa 100644 --- a/bitsail-base/src/main/java/com/bytedance/bitsail/base/component/DefaultComponentBuilderLoader.java +++ b/bitsail-base/src/main/java/com/bytedance/bitsail/base/component/DefaultComponentBuilderLoader.java @@ -45,14 +45,21 @@ public DefaultComponentBuilderLoader(Class clazz) { } public T loadComponent(String componentName) { + return this.loadComponent(componentName, true); + } + + public T loadComponent(String componentName, boolean failOnMiss) { if (!loaded) { loadAllComponents(); loaded = true; } componentName = StringUtils.lowerCase(componentName); if (!components.containsKey(componentName)) { - throw new BitSailException(CommonErrorCode.CONFIG_ERROR, - String.format("Component %s not in interface %s support until now.", componentName, clazz)); + if (failOnMiss) { + throw new BitSailException(CommonErrorCode.CONFIG_ERROR, + String.format("Component %s not in interface %s support until now.", componentName, clazz)); + } + return null; } return components.get(componentName); } diff --git a/bitsail-base/src/main/java/com/bytedance/bitsail/base/connector/BuilderGroup.java b/bitsail-base/src/main/java/com/bytedance/bitsail/base/connector/BuilderGroup.java new file mode 100644 index 000000000..40b106036 --- /dev/null +++ b/bitsail-base/src/main/java/com/bytedance/bitsail/base/connector/BuilderGroup.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.base.connector; + +public enum BuilderGroup { + + READER, + WRITER, + TRANSFORMER; +} diff --git a/bitsail-base/src/main/java/com/bytedance/bitsail/base/connector/reader/v1/Source.java b/bitsail-base/src/main/java/com/bytedance/bitsail/base/connector/reader/v1/Source.java index 65b949111..b1d6581f3 100644 --- a/bitsail-base/src/main/java/com/bytedance/bitsail/base/connector/reader/v1/Source.java +++ b/bitsail-base/src/main/java/com/bytedance/bitsail/base/connector/reader/v1/Source.java @@ -20,6 +20,7 @@ package com.bytedance.bitsail.base.connector.reader.v1; import com.bytedance.bitsail.base.execution.ExecutionEnviron; +import com.bytedance.bitsail.base.extension.TypeInfoConverterFactory; import com.bytedance.bitsail.base.serializer.BinarySerializer; import com.bytedance.bitsail.base.serializer.SimpleBinarySerializer; import com.bytedance.bitsail.common.configuration.BitSailConfiguration; @@ -29,7 +30,8 @@ import java.io.IOException; import java.io.Serializable; -public interface Source extends Serializable { +public interface Source + extends Serializable, TypeInfoConverterFactory { /** * Run in client side for source initialize; diff --git a/bitsail-base/src/main/java/com/bytedance/bitsail/base/connector/writer/v1/Sink.java b/bitsail-base/src/main/java/com/bytedance/bitsail/base/connector/writer/v1/Sink.java index 015b719dc..2a00d2697 100644 --- a/bitsail-base/src/main/java/com/bytedance/bitsail/base/connector/writer/v1/Sink.java +++ b/bitsail-base/src/main/java/com/bytedance/bitsail/base/connector/writer/v1/Sink.java @@ -17,6 +17,7 @@ package com.bytedance.bitsail.base.connector.writer.v1; +import com.bytedance.bitsail.base.extension.TypeInfoConverterFactory; import com.bytedance.bitsail.base.serializer.BinarySerializer; import com.bytedance.bitsail.base.serializer.SimpleBinarySerializer; import com.bytedance.bitsail.common.configuration.BitSailConfiguration; @@ -31,7 +32,8 @@ /** * Created 2022/6/10 */ -public interface Sink extends Serializable { +public interface Sink + extends Serializable, TypeInfoConverterFactory { /** * @return The name of writer operation. diff --git a/bitsail-base/src/main/java/com/bytedance/bitsail/base/extension/SchemaAlignmentable.java b/bitsail-base/src/main/java/com/bytedance/bitsail/base/extension/SchemaAlignmentable.java deleted file mode 100644 index 6334c4984..000000000 --- a/bitsail-base/src/main/java/com/bytedance/bitsail/base/extension/SchemaAlignmentable.java +++ /dev/null @@ -1,46 +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 com.bytedance.bitsail.base.extension; - -import com.bytedance.bitsail.base.execution.ExecutionEnviron; -import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.ExternalEngineConnector; - -import java.io.Serializable; - -/** - * Created 2022/5/23 - */ -public interface SchemaAlignmentable extends Serializable { - - /** - * create a connector for operating schema of the reader/writer - * - * @param executionEnviron execution environment - * @param selfConfiguration configuration for the reader/writer - */ - ExternalEngineConnector createExternalEngineConnector(ExecutionEnviron executionEnviron, - BitSailConfiguration selfConfiguration); - - /** - * if the reader/writer support column mapping check when finishing schema alignment - */ - default boolean isSchemaComparable() { - return false; - } -} diff --git a/bitsail-base/src/main/java/com/bytedance/bitsail/base/extension/TypeInfoConverterFactory.java b/bitsail-base/src/main/java/com/bytedance/bitsail/base/extension/TypeInfoConverterFactory.java new file mode 100644 index 000000000..d495b5491 --- /dev/null +++ b/bitsail-base/src/main/java/com/bytedance/bitsail/base/extension/TypeInfoConverterFactory.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.base.extension; + +import com.bytedance.bitsail.common.type.TypeInfoConverter; + +import java.io.Serializable; + +public interface TypeInfoConverterFactory extends Serializable { + + TypeInfoConverter createTypeInfoConverter(); +} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/TableCatalogErrorCode.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/TableCatalogErrorCode.java new file mode 100644 index 000000000..e67600e91 --- /dev/null +++ b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/TableCatalogErrorCode.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 com.bytedance.bitsail.common.catalog; + +import com.bytedance.bitsail.common.exception.ErrorCode; + +public enum TableCatalogErrorCode implements ErrorCode { + + TABLE_CATALOG_TABLE_NOT_EXISTS("Table-Catalog-1", "Catalog table not exists."), + + ; + private final String code; + + private final String description; + + TableCatalogErrorCode(String code, String description) { + this.code = code; + this.description = description; + } + + @Override + public String getCode() { + return code; + } + + @Override + public String getDescription() { + return description; + } +} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/TableCatalogManager.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/TableCatalogManager.java new file mode 100644 index 000000000..dc69f0998 --- /dev/null +++ b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/TableCatalogManager.java @@ -0,0 +1,294 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.common.catalog; + +import com.bytedance.bitsail.common.BitSailException; +import com.bytedance.bitsail.common.catalog.table.CatalogTable; +import com.bytedance.bitsail.common.catalog.table.CatalogTableAlterDefinition; +import com.bytedance.bitsail.common.catalog.table.CatalogTableColumn; +import com.bytedance.bitsail.common.catalog.table.CatalogTableDefinition; +import com.bytedance.bitsail.common.catalog.table.CatalogTableSchema; +import com.bytedance.bitsail.common.catalog.table.TableCatalog; +import com.bytedance.bitsail.common.catalog.table.TableOperation; +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.common.model.ColumnInfo; +import com.bytedance.bitsail.common.option.ReaderOptions; +import com.bytedance.bitsail.common.option.WriterOptions; +import com.bytedance.bitsail.common.type.TypeInfoConverter; +import com.bytedance.bitsail.common.typeinfo.TypeInfo; + +import lombok.Builder; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.compress.utils.Lists; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +public class TableCatalogManager { + + private static final Logger LOG = LoggerFactory.getLogger(TableCatalogManager.class); + + private final TypeInfoConverter readerTypeInfoConverter; + private final TypeInfoConverter writerTypeInfoConverter; + + private final TableCatalog readerTableCatalog; + private final TableCatalog writerTableCatalog; + + private final BitSailConfiguration commonConfiguration; + private final BitSailConfiguration readerConfiguration; + private final BitSailConfiguration writerConfiguration; + + private TableCatalogStrategy tableCatalogStrategy; + private boolean tableCatalogSync; + private boolean ignoreTableCatalogAddSync; + private boolean ignoreTableCatalogDeleteSync; + private boolean ignoreTableCatalogUpdateSync; + private boolean tableCatalogCreateTableNotExists; + private boolean tableCatalogNameCaseInsensitive; + + private CatalogTable readerCatalogTable; + private CatalogTable writerCatalogTable; + + private List finalCatalogColumns; + + @Builder + public TableCatalogManager(TypeInfoConverter readerTypeInfoConverter, + TypeInfoConverter writerTypeInfoConverter, + TableCatalog readerTableCatalog, + TableCatalog writerTableCatalog, + BitSailConfiguration commonConfiguration, + BitSailConfiguration readerConfiguration, + BitSailConfiguration writerConfiguration) { + this.readerTypeInfoConverter = readerTypeInfoConverter; + this.writerTypeInfoConverter = writerTypeInfoConverter; + this.readerTableCatalog = readerTableCatalog; + this.writerTableCatalog = writerTableCatalog; + this.commonConfiguration = commonConfiguration; + this.readerConfiguration = readerConfiguration; + this.writerConfiguration = writerConfiguration; + + prepareCatalogManager(); + } + + private void prepareCatalogManager() { + tableCatalogStrategy = + TableCatalogStrategy.valueOf(StringUtils.upperCase(commonConfiguration + .get(TableCatalogOptions.COLUMN_ALIGN_STRATEGY))); + this.tableCatalogSync = commonConfiguration.get(TableCatalogOptions.SYNC_DDL); + this.ignoreTableCatalogAddSync = commonConfiguration.get(TableCatalogOptions.SYNC_DDL_IGNORE_ADD); + this.ignoreTableCatalogDeleteSync = commonConfiguration.get(TableCatalogOptions.SYNC_DDL_IGNORE_DROP); + this.ignoreTableCatalogUpdateSync = commonConfiguration.get(TableCatalogOptions.SYNC_DDL_IGNORE_UPDATE); + this.tableCatalogCreateTableNotExists = commonConfiguration.get(TableCatalogOptions.SYNC_DDL_CREATE_TABLE); + this.tableCatalogNameCaseInsensitive = commonConfiguration.get(TableCatalogOptions.SYNC_DDL_CASE_INSENSITIVE); + } + + public void alignmentCatalogTable() throws Exception { + if (Objects.isNull(readerTableCatalog) || Objects.isNull(writerTableCatalog)) { + return; + } + + if (TableCatalogStrategy.DISABLE.equals(tableCatalogStrategy)) { + LOG.warn("Ignore table catalog alignment."); + return; + } + //start table column catalog + startTableCatalog(); + + try { + CatalogTableDefinition readerTableDefinition = readerTableCatalog.createCatalogTableDefinition(); + CatalogTableDefinition writerTableDefinition = readerTableCatalog.createCatalogTableDefinition(); + if (!readerTableCatalog.tableExists(readerTableDefinition)) { + throw BitSailException.asBitSailException(TableCatalogErrorCode.TABLE_CATALOG_TABLE_NOT_EXISTS, + String.format("Reader table definition %s not exists.", readerTableDefinition)); + } + + // get reader catalog table. + readerCatalogTable = readerTableCatalog.getCatalogTable(readerTableDefinition); + + if (!writerTableCatalog.tableExists(writerTableDefinition)) { + + if (!tableCatalogCreateTableNotExists) { + throw BitSailException.asBitSailException(TableCatalogErrorCode.TABLE_CATALOG_TABLE_NOT_EXISTS, + String.format("Writer table definition %s not exists.", writerTableDefinition)); + } + // try to create table when not exists. + writerTableCatalog.createTable(writerTableDefinition, readerCatalogTable); + } + + // get writer catalog table. + writerCatalogTable = writerTableCatalog.getCatalogTable(writerTableDefinition); + + // get base table schema. + CatalogTableSchema catalogTableSchema = tableCatalogStrategy + .apply(readerCatalogTable, writerCatalogTable); + + LOG.info("Base catalog table schema {}.", catalogTableSchema); + + if (tableCatalogSync) { + // get need changed columns. + CatalogTableAlterDefinition catalogTableAlterDefinition = + calNecessaryCatalogSchema(catalogTableSchema); + alterCatalogSchema(catalogTableAlterDefinition); + } else { + // directly use base table schema. + finalCatalogColumns.addAll(catalogTableSchema.getColumns()); + } + + List finalReaderColumnInfos = transform(finalCatalogColumns, readerTypeInfoConverter); + List finalWriterColumnInfos = transform(finalCatalogColumns, writerTypeInfoConverter); + + readerConfiguration.set(ReaderOptions.BaseReaderOptions.COLUMNS, finalReaderColumnInfos); + LOG.info("Final reader's columns: {}", finalReaderColumnInfos); + + writerConfiguration.set(WriterOptions.BaseWriterOptions.COLUMNS, finalWriterColumnInfos); + LOG.info("Final writer's columns: {}", finalWriterColumnInfos); + + } finally { + // close table catalog connection in finally. + closeTableCatalog(); + } + + } + + private List transform(List catalogTableColumns, + TypeInfoConverter typeInfoConverter) { + List columnInfos = Lists.newArrayList(); + for (CatalogTableColumn catalogTableColumn : catalogTableColumns) { + columnInfos.add(ColumnInfo + .builder() + .name(catalogTableColumn.getName()) + .type(typeInfoConverter.fromTypeInfo(catalogTableColumn.getType())) + .build()); + } + return columnInfos; + } + + private void alterCatalogSchema(CatalogTableAlterDefinition catalogTableAlterDefinition) { + if (!catalogTableAlterDefinition.isNotEmpty()) { + return; + } + if (!ignoreTableCatalogAddSync && + CollectionUtils.isNotEmpty(catalogTableAlterDefinition.getPendingAddColumns())) { + LOG.info("Writer catalog table {} try to add column: {}.", writerCatalogTable, + catalogTableAlterDefinition.getPendingAddColumns()); + writerTableCatalog.alterTableColumns( + TableOperation.ALTER_COLUMNS_ADD, + catalogTableAlterDefinition.getPendingAddColumns() + ); + } + + if (!ignoreTableCatalogUpdateSync && + CollectionUtils.isNotEmpty(catalogTableAlterDefinition.getPendingUpdateColumns())) { + LOG.info("Writer catalog table {} try to update column: {}.", writerCatalogTable, + catalogTableAlterDefinition.getPendingUpdateColumns()); + writerTableCatalog.alterTableColumns( + TableOperation.ALTER_COLUMNS_UPDATE, + catalogTableAlterDefinition.getPendingUpdateColumns() + ); + } + + if (!ignoreTableCatalogDeleteSync && + CollectionUtils.isNotEmpty(catalogTableAlterDefinition.getPendingDeleteColumns())) { + LOG.info("Writer catalog table {} try to delete column: {}.", writerCatalogTable, + catalogTableAlterDefinition.getPendingDeleteColumns()); + writerTableCatalog.alterTableColumns( + TableOperation.ALTER_COLUMNS_DELETE, + catalogTableAlterDefinition.getPendingDeleteColumns() + ); + } + + } + + private CatalogTableAlterDefinition calNecessaryCatalogSchema(CatalogTableSchema baseCatalogTableSchema) { + CatalogTableSchema writeCatalogTableSchema = writerCatalogTable.getCatalogTableSchema(); + + Map> writeTableColumnMapping = writeCatalogTableSchema + .getColumns() + .stream() + .collect(Collectors.toMap( + column -> tableCatalogNameCaseInsensitive ? + StringUtils.lowerCase(column.getName()) : + column.getName(), + CatalogTableColumn::getType)); + + List writePrimaryTableColumnMapping = Lists.newArrayList(); + if (CollectionUtils.isNotEmpty(writeCatalogTableSchema.getPrimaryKeys())) { + writePrimaryTableColumnMapping.addAll(writeCatalogTableSchema + .getPrimaryKeys() + .stream() + .map(column -> tableCatalogNameCaseInsensitive ? + StringUtils.lowerCase(column.getName()) : + column.getName()) + .collect(Collectors.toList())); + } + + finalCatalogColumns = Lists.newArrayList(); + + List pendingAddTableColumns = Lists.newArrayList(); + List pendingUpdateTableColumns = Lists.newArrayList(); + List pendingDeleteTableColumns = Lists.newArrayList(); + for (CatalogTableColumn catalogTableColumn : baseCatalogTableSchema.getColumns()) { + + String baseCatalogColumnName = tableCatalogNameCaseInsensitive ? + StringUtils.lowerCase(catalogTableColumn.getName()) : + catalogTableColumn.getName(); + + if (writePrimaryTableColumnMapping.contains(baseCatalogColumnName)) { + finalCatalogColumns.add(catalogTableColumn); + continue; + } + + if (writeTableColumnMapping.containsKey(baseCatalogColumnName)) { + TypeInfo writerTypeInfo = writeTableColumnMapping.get(baseCatalogColumnName); + TypeInfo baseTypeInfo = catalogTableColumn.getType(); + + finalCatalogColumns.add(catalogTableColumn); + if (!writerTableCatalog.compareTypeCompatible(writerTypeInfo, baseTypeInfo)) { + pendingUpdateTableColumns.add(catalogTableColumn); + } + } else { + finalCatalogColumns.add(catalogTableColumn); + pendingAddTableColumns.add(catalogTableColumn); + } + } + + return CatalogTableAlterDefinition.builder() + .pendingDeleteColumns(pendingDeleteTableColumns) + .pendingAddColumns(pendingAddTableColumns) + .pendingUpdateColumns(pendingUpdateTableColumns) + .build(); + } + + private void startTableCatalog() { + readerTableCatalog.open(readerTypeInfoConverter); + writerTableCatalog.open(writerTypeInfoConverter); + } + + private void closeTableCatalog() { + readerTableCatalog.close(); + writerTableCatalog.close(); + } + +} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/TableCatalogOptions.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/TableCatalogOptions.java new file mode 100644 index 000000000..3a843c078 --- /dev/null +++ b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/TableCatalogOptions.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.common.catalog; + +import com.bytedance.bitsail.common.option.CommonOptions; +import com.bytedance.bitsail.common.option.ConfigOption; + +import com.google.common.annotations.Beta; + +import static com.bytedance.bitsail.common.option.ConfigOptions.key; + +public interface TableCatalogOptions extends CommonOptions { + + ConfigOption COLUMN_ALIGN_STRATEGY = + key(COMMON_PREFIX + "column_align_strategy") + .defaultValue("disable"); + + /** + * Whether enable the ddl sync feature. + */ + ConfigOption SYNC_DDL = + key(COMMON_PREFIX + "sync_ddl") + .defaultValue(false); + + ConfigOption SYNC_DDL_SKIP_ERROR_COLUMNS = + key(COMMON_PREFIX + "sync_ddl_skip_error_columns") + .defaultValue(true); + + ConfigOption SYNC_DDL_PRE_EXECUTE = + key(COMMON_PREFIX + "sync_ddl_pre_execute") + .defaultValue(false); + + /** + * Ignore ddl delete fields. + */ + ConfigOption SYNC_DDL_IGNORE_DROP = + key(COMMON_PREFIX + "sync_ddl_ignore_drop") + .defaultValue(true); + /** + * Ignore ddl new added fields. + */ + ConfigOption SYNC_DDL_IGNORE_ADD = + key(COMMON_PREFIX + "sync_ddl_ignore_add") + .defaultValue(false); + + /** + * Ignore ddl updated fields. + */ + ConfigOption SYNC_DDL_IGNORE_UPDATE = + key(COMMON_PREFIX + "sync_ddl_ignore_update") + .defaultValue(false); + + /** + * Only test in develop + */ + @Beta + ConfigOption SYNC_DDL_CREATE_TABLE = + key(COMMON_PREFIX + "sync_ddl_create_table") + .defaultValue(false); + + ConfigOption SYNC_DDL_CASE_INSENSITIVE = + key(COMMON_PREFIX + "sync_ddl_case_insensitive") + .defaultValue(true); +} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/TableCatalogStrategy.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/TableCatalogStrategy.java new file mode 100644 index 000000000..75cfe93e2 --- /dev/null +++ b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/TableCatalogStrategy.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.common.catalog; + +import com.bytedance.bitsail.common.catalog.table.CatalogTable; +import com.bytedance.bitsail.common.catalog.table.CatalogTableColumn; +import com.bytedance.bitsail.common.catalog.table.CatalogTableSchema; + +import com.google.common.collect.Sets; +import org.apache.commons.compress.utils.Lists; + +import java.io.Serializable; +import java.util.List; +import java.util.Set; + +public enum TableCatalogStrategy implements Serializable { + + DISABLE, + + INTERSECT { + @Override + public CatalogTableSchema apply(CatalogTable reader, CatalogTable writer) { + CatalogTableSchema readerCatalogTableSchema = reader.getCatalogTableSchema(); + CatalogTableSchema writerCatalogTableSchema = writer.getCatalogTableSchema(); + + List readerColumns = readerCatalogTableSchema.getColumns(); + List writerColumns = writerCatalogTableSchema.getColumns(); + + Set intersect = Sets.intersection( + Sets.newHashSet(readerColumns), + Sets.newHashSet(writerColumns)) + .copyInto(Sets.newHashSet()); + + return new CatalogTableSchema(Lists.newArrayList(intersect.iterator())); + } + }, + + SOURCE_ONLY { + @Override + public CatalogTableSchema apply(CatalogTable reader, CatalogTable writer) { + return reader.getCatalogTableSchema(); + } + }; + + public CatalogTableSchema apply(CatalogTable reader, + CatalogTable writer) { + throw new UnsupportedOperationException(); + } + +} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/CatalogTable.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/CatalogTable.java new file mode 100644 index 000000000..a1b16a417 --- /dev/null +++ b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/CatalogTable.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.common.catalog.table; + +import lombok.Builder; +import lombok.Getter; + +import java.io.Serializable; + +@Getter +@Builder +public class CatalogTable implements Serializable { + + private CatalogTableDefinition catalogTableDefinition; + + private final CatalogTableSchema catalogTableSchema; + + private final String comment; + + public CatalogTable(CatalogTableDefinition catalogTableDefinition, + CatalogTableSchema catalogTableSchema) { + this(catalogTableDefinition, catalogTableSchema, null); + } + + public CatalogTable(CatalogTableDefinition catalogTableDefinition, + CatalogTableSchema catalogTableSchema, + String comment) { + this.catalogTableDefinition = catalogTableDefinition; + this.catalogTableSchema = catalogTableSchema; + this.comment = comment; + } + + @Override + public String toString() { + return "CatalogTable{" + + "catalogTableDefinition=" + catalogTableDefinition + + ", catalogTableSchema=" + catalogTableSchema + + ", comment='" + comment + '\'' + + '}'; + } +} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/CatalogTableAlterDefinition.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/CatalogTableAlterDefinition.java new file mode 100644 index 000000000..3f306f98f --- /dev/null +++ b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/CatalogTableAlterDefinition.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.common.catalog.table; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Getter; +import org.apache.commons.collections.CollectionUtils; + +import java.io.Serializable; +import java.util.List; + +@Builder +@AllArgsConstructor +@Getter +public class CatalogTableAlterDefinition implements Serializable { + + private List pendingAddColumns; + + private List pendingUpdateColumns; + + private List pendingDeleteColumns; + + public boolean isNotEmpty() { + return CollectionUtils.isNotEmpty(pendingAddColumns) || + CollectionUtils.isNotEmpty(pendingUpdateColumns) || + CollectionUtils.isNotEmpty(pendingDeleteColumns); + } +} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/CatalogTableColumn.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/CatalogTableColumn.java new file mode 100644 index 000000000..834ce1fdc --- /dev/null +++ b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/CatalogTableColumn.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.common.catalog.table; + +import com.bytedance.bitsail.common.typeinfo.TypeInfo; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.EqualsAndHashCode; +import lombok.Getter; + +import java.io.Serializable; + +@Getter +@Builder +@AllArgsConstructor +@EqualsAndHashCode(of = {"name", "type"}) +public class CatalogTableColumn implements Serializable { + + private final String name; + + private final TypeInfo type; + + private String comment; + + public CatalogTableColumn(String name, TypeInfo type) { + this.name = name; + this.type = type; + } + + @Override + public String toString() { + return "CatalogTableColumn{" + + "name='" + name + '\'' + + ", type=" + type + + ", comment='" + comment + '\'' + + '}'; + } +} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/CatalogTableDefinition.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/CatalogTableDefinition.java new file mode 100644 index 000000000..343198fb7 --- /dev/null +++ b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/CatalogTableDefinition.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.common.catalog.table; + +import lombok.Builder; +import lombok.Getter; + +import java.io.Serializable; + +@Getter +@Builder +public class CatalogTableDefinition implements Serializable { + + private final String database; + private final String schema; + private final String table; + + public CatalogTableDefinition(String database, String schema, String table) { + this.database = database; + this.schema = schema; + this.table = table; + } + + @Override + public String toString() { + return "CatalogTableDefinition{" + + "database='" + database + '\'' + + ", schema='" + schema + '\'' + + ", table='" + table + '\'' + + '}'; + } +} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/CatalogTableSchema.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/CatalogTableSchema.java new file mode 100644 index 000000000..fd62a2330 --- /dev/null +++ b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/CatalogTableSchema.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.common.catalog.table; + +import lombok.Builder; +import lombok.Getter; + +import java.io.Serializable; +import java.util.Collections; +import java.util.List; + +@Getter +@Builder +public class CatalogTableSchema implements Serializable { + + private final List columns; + + private final List primaryKeys; + + public CatalogTableSchema(List columns) { + this(columns, Collections.emptyList()); + } + + public CatalogTableSchema(List columns, + List primaryKeys) { + this.columns = columns; + this.primaryKeys = primaryKeys; + } + + @Override + public String toString() { + return "CatalogTableSchema{" + + "columns=" + columns + + ", primaryKeys=" + primaryKeys + + '}'; + } +} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/TableCatalog.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/TableCatalog.java new file mode 100644 index 000000000..ddbd46bf1 --- /dev/null +++ b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/TableCatalog.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.common.catalog.table; + +import com.bytedance.bitsail.common.model.ColumnInfo; +import com.bytedance.bitsail.common.type.TypeInfoConverter; +import com.bytedance.bitsail.common.typeinfo.TypeInfo; + +import com.google.common.collect.Lists; + +import java.io.Serializable; +import java.util.List; + +/** + * Table catalog only for the signal table for now. + */ +public interface TableCatalog extends Serializable { + + /** + * Open Table catalog + */ + void open(TypeInfoConverter typeInfoConverter); + + /** + * Close table catalog + */ + void close(); + + /** + * Get the reference table for the table catalog. + */ + CatalogTableDefinition createCatalogTableDefinition(); + + /** + * Check the table exits or not. + */ + boolean tableExists(CatalogTableDefinition catalogTableDefinition); + + /** + * Acquire catalog table by the table definition. + */ + CatalogTable getCatalogTable(CatalogTableDefinition catalogTableDefinition); + + /** + * Create table + */ + void createTable(CatalogTableDefinition catalogTableDefinition, + CatalogTable catalogTable); + + /** + * Alter table + */ + void alterTable(TableOperation tableOperation, + CatalogTable table); + + /** + * Alter table columns. + */ + void alterTableColumns(TableOperation tableOperation, + List catalogTableColumns); + + boolean compareTypeCompatible(TypeInfo original, TypeInfo compared); + + default List convertTableColumn(TypeInfoConverter typeInfoConverter, + List columnInfos) { + List tableColumns = Lists.newArrayList(); + for (ColumnInfo columnInfo : columnInfos) { + tableColumns.add( + CatalogTableColumn.builder() + .name(columnInfo.getName()) + .type(typeInfoConverter.fromTypeString(columnInfo.getType())) + .build() + ); + } + return tableColumns; + } + +} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/TableOperation.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/TableOperation.java new file mode 100644 index 000000000..e96b8753b --- /dev/null +++ b/bitsail-common/src/main/java/com/bytedance/bitsail/common/catalog/table/TableOperation.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.common.catalog.table; + +import java.io.Serializable; + +public enum TableOperation implements Serializable { + CHANGE_TBL_PROPS, + CHANGE_SERDE_PROPS, + CHANGE_FILE_FORMAT, + CHANGE_LOCATION, + ALTER_COLUMNS_ADD, + ALTER_COLUMNS_DELETE, + ALTER_COLUMNS_UPDATE +} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/DdlSyncManager.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/DdlSyncManager.java deleted file mode 100644 index 0aa48a7fc..000000000 --- a/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/DdlSyncManager.java +++ /dev/null @@ -1,423 +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 com.bytedance.bitsail.common.ddl; - -import com.bytedance.bitsail.common.BitSailException; -import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.sink.SinkEngineConnector; -import com.bytedance.bitsail.common.ddl.source.SourceEngineConnector; -import com.bytedance.bitsail.common.exception.CommonErrorCode; -import com.bytedance.bitsail.common.model.ColumnInfo; -import com.bytedance.bitsail.common.option.CommonOptions; -import com.bytedance.bitsail.common.option.ReaderOptions; -import com.bytedance.bitsail.common.option.WriterOptions; -import com.bytedance.bitsail.common.type.TypeInfoConverter; -import com.bytedance.bitsail.common.typeinfo.TypeInfo; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; -import lombok.Getter; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.time.StopWatch; - -import java.util.Arrays; -import java.util.LinkedHashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.stream.Collectors; - -@Slf4j -public class DdlSyncManager { - private static final char[] INVALID_CHARS = {'+', '-', '*', '/', '.', ','}; - - private final SourceEngineConnector readerExternalEngineConnector; - private final SinkEngineConnector writerExternalEngineConnector; - private final BitSailConfiguration commonConf; - private final BitSailConfiguration readerConf; - private final BitSailConfiguration writerConf; - private final boolean ddlSyncSkipErrorColumns; - private final boolean ddlSyncPreExecute; - private final boolean ddlSyncIgnoreAdd; - private final boolean ddlSyncIgnoreUpdate; - private final boolean ddlSyncIgnoreDrop; - @Getter - private final List alignedReaderColumns; - @Getter - private final List alignedWriterColumns; - private BitSailConfiguration globalConfiguration; - @Getter - private List pendingAddColumns; - @Getter - private List pendingUpdateColumns; - @Getter - private List pendingDeleteColumns; - - public DdlSyncManager(SourceEngineConnector readerExternalEngineConnector, - SinkEngineConnector writerExternalEngineConnector, - BitSailConfiguration commonConf, - BitSailConfiguration readerConf, - BitSailConfiguration writerConf) { - - this.readerExternalEngineConnector = readerExternalEngineConnector; - this.writerExternalEngineConnector = writerExternalEngineConnector; - this.commonConf = commonConf; - this.readerConf = readerConf; - this.writerConf = writerConf; - - this.ddlSyncSkipErrorColumns = commonConf.get(CommonOptions.SYNC_DDL_SKIP_ERROR_COLUMNS); - this.ddlSyncPreExecute = commonConf.get(CommonOptions.SYNC_DDL_PRE_EXECUTE); - this.ddlSyncIgnoreAdd = commonConf.get(CommonOptions.SYNC_DDL_IGNORE_ADD); - this.ddlSyncIgnoreUpdate = commonConf.get(CommonOptions.SYNC_DDL_IGNORE_UPDATE); - this.ddlSyncIgnoreDrop = commonConf.get(CommonOptions.SYNC_DDL_IGNORE_DROP); - - this.alignedReaderColumns = Lists.newLinkedList(); - this.alignedWriterColumns = Lists.newLinkedList(); - this.pendingAddColumns = new LinkedList<>(); - this.pendingDeleteColumns = new LinkedList<>(); - this.pendingUpdateColumns = new LinkedList<>(); - } - - private static boolean containsInvalidChar(String fieldName) { - for (char c : INVALID_CHARS) { - if (StringUtils.contains(fieldName, c)) { - log.warn("Reader field name {} contains invalid character {}.", fieldName, c); - return true; - } - } - return false; - } - - public void doColumnAlignment(boolean supportSchemaCheck) throws Exception { - //1. acquire alignment strategy - SchemaColumnAligner.ColumnAlignmentStrategy alignmentStrategy = getAlignmentStrategy(); - - //2. get pending align columns by the strategy - doColumnConfAlignment(alignmentStrategy); - - //3. do the real align between reader & writer. - doExternalColumnAlignment(alignmentStrategy); - - //4. final check. - checkColumnsAlignment(readerExternalEngineConnector, supportSchemaCheck); - } - - @VisibleForTesting - SchemaColumnAligner.ColumnAlignmentStrategy getAlignmentStrategy() { - String columnAlignmentStrategy = commonConf - .get(CommonOptions.COLUMN_ALIGN_STRATEGY) - .toLowerCase().trim(); - SchemaColumnAligner.ColumnAlignmentStrategy strategy; - try { - strategy = SchemaColumnAligner.ColumnAlignmentStrategy.valueOf(columnAlignmentStrategy); - } catch (Exception e) { - throw BitSailException.asBitSailException(CommonErrorCode.CONFIG_ERROR, - String.format("Strategy for the column align %s is not supported, support values are = %s", - columnAlignmentStrategy, Arrays.toString(SchemaColumnAligner.ColumnAlignmentStrategy.values()))); - } - return strategy; - } - - public void doColumnConfAlignment(SchemaColumnAligner.ColumnAlignmentStrategy strategy) throws Exception { - switch (strategy) { - case disable: - log.info("auto_get_columns is disable!"); - return; - case intersect: - doIntersectAlignment(); - return; - case source_only: - doSourceOnlyAlignment(); - return; - default: - throw BitSailException.asBitSailException(CommonErrorCode.CONFIG_ERROR, - String.format("Unsupported AutoGetColumnMode mode:[%s]. Allowed modes are %s, please check your configuration.", - strategy, Arrays.toString(SchemaColumnAligner.ColumnAlignmentStrategy.values()))); - } - } - - void getChangeableColumns(SchemaColumnAligner.ColumnAlignmentStrategy strategy) throws Exception { - - List readerColumnInfos = readerConf - .getNecessaryOption(ReaderOptions.BaseReaderOptions.COLUMNS, CommonErrorCode.CONFIG_ERROR); - - List writerColumnInfos; - if (strategy == SchemaColumnAligner.ColumnAlignmentStrategy.intersect) { - writerColumnInfos = writerConf - .getNecessaryOption(WriterOptions.BaseWriterOptions.COLUMNS, CommonErrorCode.CONFIG_ERROR); - } else { - writerColumnInfos = writerExternalEngineConnector.getExternalColumnInfos(); - } - - Map writerColumnMapping = writerColumnInfos.stream() - .collect(Collectors.toMap(ColumnInfo::getName, ColumnInfo::getType)); - - List writerExcludedColumnNames = writerExternalEngineConnector.getExcludedColumnInfos() - .stream() - .map(StringUtils::lowerCase) - .collect(Collectors.toList()); - - log.info("Reader Columns are {}, and size: {}.", readerColumnInfos, CollectionUtils.size(readerColumnInfos)); - log.info("Writer Columns are {}, and size: {}.", writerColumnInfos, CollectionUtils.size(writerColumnInfos)); - - List columnsToAdd = Lists.newLinkedList(); - List columnsToUpdate = Lists.newLinkedList(); - - TypeInfoConverter readerTypeInfoConverter = readerExternalEngineConnector.createTypeInfoConverter(); - TypeInfoConverter writerTypeInfoConverter = writerExternalEngineConnector.createTypeInfoConverter(); - - for (ColumnInfo readerColumnInfo : readerColumnInfos) { - String readerColumnName = StringUtils.lowerCase(readerColumnInfo.getName()); - String readerColumnType = StringUtils.lowerCase(readerColumnInfo.getType()); - - if (containsInvalidChar(readerColumnInfo.getName())) { - if (ddlSyncSkipErrorColumns) { - log.info("Here have an column name: {} which contains invalid chars, skip it do column alignment", - readerColumnName); - continue; - } - throw BitSailException.asBitSailException(CommonErrorCode.PLUGIN_ERROR, - String.format("Invalid column name:[%s]. Please validate your columns names!", readerColumnName)); - } - - if (writerExcludedColumnNames.contains(readerColumnName)) { - log.info("Ignore the excluded column: {}.", readerColumnName); - alignedReaderColumns.add(new ColumnInfo(readerColumnName, readerColumnType)); - //todo if writer columns hasn't the source name. - alignedWriterColumns.add(new ColumnInfo(readerColumnName, writerColumnMapping.get(readerColumnName))); - continue; - } - - if (writerColumnMapping.containsKey(readerColumnName)) { - //Update the columns - columnsToUpdate(readerTypeInfoConverter, - writerTypeInfoConverter, - readerColumnName, - readerColumnType, - columnsToUpdate, - writerColumnMapping); - } else { - //Adding the columns. - columnToAdding(readerTypeInfoConverter, - writerTypeInfoConverter, - readerColumnName, - readerColumnType, - columnsToAdd); - } - } - - if (CollectionUtils.isNotEmpty(columnsToAdd) && !ddlSyncIgnoreAdd) { - pendingAddColumns = columnsToAdd; - log.info("There are some fields need to be added to sink storage: {}", pendingAddColumns); - } else { - log.info("There is no fields need to be added to sink storage"); - } - - if (CollectionUtils.isNotEmpty(columnsToUpdate) && !ddlSyncIgnoreUpdate) { - pendingUpdateColumns = columnsToUpdate; - log.info("There may be some fields need to be modified in sink storage: {}", pendingUpdateColumns); - } else { - log.info("There is no fields need to be modified in sink storage"); - } - //todo Current now, we didn't support column deletion action. - //pendingDeleteColumns = null; - } - - private void columnToAdding(TypeInfoConverter readerTypeInfoConverter, - TypeInfoConverter writerTypeInfoConverter, - String sourceColumnName, - String sourceColumnType, - List columnsToAdd) { - TypeInfo readerTypeInfo = readerTypeInfoConverter.fromTypeString(sourceColumnType); - String writerExternalEngineTypeName = writerTypeInfoConverter.fromTypeInfo(readerTypeInfo); - - log.info("Adding column for reader column name: {}, " + - "reader column type: {}, reader column type info: {}.", sourceColumnName, sourceColumnType, readerTypeInfo); - if (Objects.nonNull(writerExternalEngineTypeName)) { - columnsToAdd.add(new ColumnInfo(sourceColumnName, writerExternalEngineTypeName)); - alignedReaderColumns.add(new ColumnInfo(sourceColumnName, sourceColumnType)); - alignedWriterColumns.add(new ColumnInfo(sourceColumnName, writerExternalEngineTypeName)); - return; - } - - if (!ddlSyncSkipErrorColumns) { - log.info("Cannot get sink type from source type: [{}] , continue...", sourceColumnType); - return; - } - throw BitSailException.asBitSailException(CommonErrorCode.PLUGIN_ERROR, "Column name: " + sourceColumnName + - " Cannot get sink type from source type: " + sourceColumnType + ", Please validate your columns type!"); - } - - private void columnsToUpdate(TypeInfoConverter readerTypeInfoConverter, - TypeInfoConverter writerTypeInfoConverter, - String sourceColumnName, - String sourceColumnType, - List columnsToUpdate, - Map writerColumnMapping) { - String originalWriterColumnType = writerColumnMapping.get(sourceColumnName); - - TypeInfo readerTypeInfo = readerTypeInfoConverter.fromTypeString(sourceColumnType); - String newWriterColumnType = writerTypeInfoConverter.fromTypeInfo(readerTypeInfo); - log.info("Updating column for column name: {}, reader column type: {}, " + - "reader column type info: {}, writer column name: {}.", - sourceColumnName, sourceColumnType, readerTypeInfo, newWriterColumnType); - - if (Objects.nonNull(newWriterColumnType)) { - alignedReaderColumns.add(new ColumnInfo(sourceColumnName, sourceColumnType)); - - if (writerExternalEngineConnector.isTypeCompatible(newWriterColumnType, originalWriterColumnType)) { - alignedWriterColumns.add(new ColumnInfo(sourceColumnName, originalWriterColumnType)); - - } else { - alignedWriterColumns.add(new ColumnInfo(sourceColumnName, newWriterColumnType)); - columnsToUpdate.add(new ColumnInfo(sourceColumnName, newWriterColumnType)); - } - - return; - } - //if writer engine not support this column type - if (ddlSyncSkipErrorColumns) { - log.warn("Sink column is [{}], cannot get sink type from source type:[{}], this column will not be transmitted.", - sourceColumnName, sourceColumnType); - } else { - throw BitSailException.asBitSailException(CommonErrorCode.PLUGIN_ERROR, - String.format("Sink column is: [%s] ,cannot get sink type from source type: [%s], Please validate your columns type!", - sourceColumnName, sourceColumnType)); - } - } - - @SuppressWarnings("checkstyle:MagicNumber") - public void invokeExternalColumnAlignment(SchemaColumnAligner.ColumnAlignmentStrategy strategy) throws Exception { - StopWatch stopWatch = new StopWatch(); - stopWatch.start(); - - try { - log.info("Starting alignment source external system and sink external system..."); - if (this.readerExternalEngineConnector == null || this.writerExternalEngineConnector == null) { - throw BitSailException.asBitSailException(CommonErrorCode.CONFIG_ERROR, - "source or sink isn't support schema manager, switch off the job.common.sync_ddl option!"); - } - getChangeableColumns(strategy); - - if (!ddlSyncIgnoreAdd) { - this.writerExternalEngineConnector.addColumns(this.pendingAddColumns); - } - - if (!ddlSyncIgnoreUpdate) { - this.writerExternalEngineConnector.updateColumns(this.pendingUpdateColumns); - } - - if (!ddlSyncIgnoreDrop) { - this.writerExternalEngineConnector.deleteColumns(this.pendingDeleteColumns); - } - } finally { - stopWatch.stop(); - } - log.info("sync ddl finished, cost: {}s", stopWatch.getTime() / 1000); - } - - /** - * @throws Exception - */ - private void doIntersectAlignment() throws Exception { - if (null == readerExternalEngineConnector || null == writerExternalEngineConnector) { - throw BitSailException.asBitSailException(CommonErrorCode.CONFIG_ERROR, - "The option of auto_get_schema is intersect, but the source or sink does not support schema manage now!"); - } - SchemaColumnAligner schemaColumnAligner = new SchemaColumnAligner(readerExternalEngineConnector, writerExternalEngineConnector); - SchemaHelper.SchemaIntersectionResponse schemaResponse = schemaColumnAligner.doIntersectStrategy(); - List readerCols = schemaResponse.getReaderColumns(); - List writerCols = schemaResponse.getWriterColumns(); - - readerConf.set(ReaderOptions.BaseReaderOptions.COLUMNS, readerCols); - writerConf.set(WriterOptions.BaseWriterOptions.COLUMNS, writerCols); - - if (globalConfiguration != null) { - globalConfiguration.set(ReaderOptions.BaseReaderOptions.COLUMNS, readerCols); - globalConfiguration.set(WriterOptions.BaseWriterOptions.COLUMNS, writerCols); - } - log.info("auto_get_columns is intersect, get columns from DB finished, final reader columns {}, writer columns {}.", - readerCols, writerCols); - } - - private void doSourceOnlyAlignment() throws Exception { - if (readerExternalEngineConnector == null) { - throw BitSailException.asBitSailException(CommonErrorCode.CONFIG_ERROR, - "The option of auto_get_schema is source_only, but the source is not support schema manage now!"); - } - SchemaColumnAligner schemaColumnAligner = new SchemaColumnAligner(readerExternalEngineConnector, writerExternalEngineConnector); - List columnInfos = schemaColumnAligner.doSourceOnlyStrategy(); - - readerConf.set(ReaderOptions.BaseReaderOptions.COLUMNS, columnInfos); - if (globalConfiguration != null) { - globalConfiguration.set(ReaderOptions.BaseReaderOptions.COLUMNS, columnInfos); - } - log.info("auto_get_columns is source_only, get columns from DB finished, final reader columns {}.", columnInfos); - } - - /** - * @throws Exception - */ - public void doExternalColumnAlignment(SchemaColumnAligner.ColumnAlignmentStrategy strategy) throws Exception { - if (!commonConf.get(CommonOptions.SYNC_DDL)) { - log.info("sync_ddl switch is off, will not auto sync ddl."); - return; - } - invokeExternalColumnAlignment(strategy); - - readerConf.set(ReaderOptions.BaseReaderOptions.COLUMNS, alignedReaderColumns); - log.info("Final reader's columns: {}", alignedReaderColumns); - - writerConf.set(WriterOptions.BaseWriterOptions.COLUMNS, alignedWriterColumns); - log.info("Final writer's columns: {}", alignedWriterColumns); - - if (globalConfiguration != null) { - globalConfiguration.set(ReaderOptions.BaseReaderOptions.COLUMNS, alignedReaderColumns); - globalConfiguration.set(WriterOptions.BaseWriterOptions.COLUMNS, alignedWriterColumns); - } - } - - private void checkColumnsAlignment(SourceEngineConnector sourceSchemaManager, - boolean supportSchemaCheck) { - Boolean ddlSync = commonConf.get(CommonOptions.SYNC_DDL); - if (ddlSync != null && ddlSync) { - log.info("The DDL sync mode is enabled. Columns mapping check will be ignored!"); - return; - } - String autoGetColumns = commonConf.get(CommonOptions.COLUMN_ALIGN_STRATEGY); - if (StringUtils.equalsIgnoreCase(SchemaColumnAligner.ColumnAlignmentStrategy.intersect.toString(), autoGetColumns)) { - log.info("The auto_get_columns mode is 'intersect'. Columns mapping check will be ignored!"); - return; - } - try { - LinkedHashMap readerCols = - SchemaHelper.getLinkedMapColumnsFromConfColumns(readerConf.get(ReaderOptions.BaseReaderOptions.COLUMNS), false); - if (supportSchemaCheck) { - LinkedHashMap sourceCols = SchemaHelper.getLinkedMapColumnsFromConfColumns(sourceSchemaManager.getExternalColumnInfos(), false); - if (SchemaHelper.isColumnsSizeEqual(sourceCols, readerCols, "source", "reader")) { - SchemaHelper.compareColumnsName(sourceCols, readerCols, "source", "reader"); - } - } - } catch (Exception e) { - log.error("Columns mapping check failed."); - } - } -} \ No newline at end of file diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/ExternalEngineConnector.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/ExternalEngineConnector.java deleted file mode 100644 index ca3b76e3a..000000000 --- a/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/ExternalEngineConnector.java +++ /dev/null @@ -1,42 +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 com.bytedance.bitsail.common.ddl; - -import com.bytedance.bitsail.common.model.ColumnInfo; -import com.bytedance.bitsail.common.type.TypeInfoConverter; - -import java.io.Serializable; -import java.util.List; - -public interface ExternalEngineConnector extends Serializable { - - /** - * Get external engine system's name for type system. - */ - String getExternalEngineName(); - - /** - * Acquire all external columns by connect to external system. - */ - List getExternalColumnInfos() throws Exception; - - /** - * Create converter for the external engine. - */ - TypeInfoConverter createTypeInfoConverter(); -} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/SchemaColumnAligner.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/SchemaColumnAligner.java deleted file mode 100644 index bdc52fa8e..000000000 --- a/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/SchemaColumnAligner.java +++ /dev/null @@ -1,74 +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 com.bytedance.bitsail.common.ddl; - -import com.bytedance.bitsail.common.ddl.sink.SinkEngineConnector; -import com.bytedance.bitsail.common.ddl.source.SourceEngineConnector; -import com.bytedance.bitsail.common.model.ColumnInfo; - -import lombok.extern.slf4j.Slf4j; - -import java.util.List; - -@Slf4j -public class SchemaColumnAligner { - - private final SourceEngineConnector sourceEngineConnector; - private final SinkEngineConnector sinkEngineConnector; - - public SchemaColumnAligner(SourceEngineConnector sourceEngineConnector, - SinkEngineConnector sinkEngineConnector) { - - this.sourceEngineConnector = sourceEngineConnector; - this.sinkEngineConnector = sinkEngineConnector; - } - - public SchemaHelper.SchemaIntersectionResponse doIntersectStrategy() throws Exception { - List sourceExternalColumnInfos = sourceEngineConnector - .getExternalColumnInfos(); - List sinkExternalColumnInfos = sinkEngineConnector - .getExternalColumnInfos(); - return intersectColumns(sourceExternalColumnInfos, sinkExternalColumnInfos); - } - - public List doSourceOnlyStrategy() throws Exception { - return sourceEngineConnector.getExternalColumnInfos(); - } - - SchemaHelper.SchemaIntersectionResponse intersectColumns(List readerColumns, - List writerColumns) { - return SchemaHelper.intersectColumns(readerColumns, writerColumns); - } - - public enum ColumnAlignmentStrategy { - /** - * Disable column alignment action - */ - disable, - - /** - * Intersect source & sink columns - */ - intersect, - - /** - * Use source columns instead of sink columns - */ - source_only - } -} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/SchemaHelper.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/SchemaHelper.java deleted file mode 100644 index 79a169299..000000000 --- a/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/SchemaHelper.java +++ /dev/null @@ -1,211 +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 com.bytedance.bitsail.common.ddl; - -import com.bytedance.bitsail.common.BitSailException; -import com.bytedance.bitsail.common.exception.CommonErrorCode; -import com.bytedance.bitsail.common.model.ColumnInfo; - -import com.google.common.collect.Sets; -import lombok.AllArgsConstructor; -import lombok.Getter; -import lombok.extern.slf4j.Slf4j; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.function.Function; -import java.util.stream.Collectors; - -import static java.util.stream.Collectors.toList; -import static java.util.stream.Collectors.toSet; - -@Slf4j -public class SchemaHelper { - - public static LinkedHashMap getLinkedMapColumnsFromConfColumns(List columns, boolean lowerCaseType) { - if (null == columns || columns.size() == 0) { - return new LinkedHashMap<>(); - } - LinkedHashMap columnsMap = new LinkedHashMap<>(); - for (ColumnInfo columnInfo : columns) { - String name = columnInfo.getName(); - String type = columnInfo.getType(); - if (lowerCaseType) { - type = type.toLowerCase(); - } - columnsMap.put(name, type); - } - return columnsMap; - } - - public static SchemaIntersectionResponse intersectColumns(List readerColumns, List writerColumns) { - final Set readerColumnNames = toColumnNamesSet(readerColumns); - final Set writerColumnNames = toColumnNamesSet(writerColumns); - final Set intersectionColumnNames = Sets.intersection(readerColumnNames, writerColumnNames); - - if (intersectionColumnNames.size() == 0) { - throw BitSailException.asBitSailException(CommonErrorCode.PLUGIN_ERROR, "The source columns and sink columns have no intersections, " + - "and the source columns are " + readerColumns + " the sink columns are " + writerColumns + - ". Please make sure source columns and sink columns have intersections"); - } - - readerColumns = filterColumnsByNames(readerColumns, intersectionColumnNames); - writerColumns = filterColumnsByNames(writerColumns, intersectionColumnNames); - - writerColumns = sortColumnsByOtherColumnNames(writerColumns, readerColumns); - return new SchemaIntersectionResponse(readerColumns, writerColumns); - } - - /** - * if cols1.size>cols2.size return cols1 - cols2 else return cols2 - cols1 - * - * @param cols1 - * @param cols2 - * @return - */ - public static List getComplementaryColumns(LinkedHashMap cols1, LinkedHashMap cols2) { - LinkedHashMap temp; - LinkedHashMap cols2Copy = new LinkedHashMap(); - List complementaryColumns = new ArrayList<>(); - if (cols1.size() < cols2.size()) { - temp = cols1; - cols1 = cols2; - cols2 = temp; - } - cols2.forEach((x, y) -> cols2Copy.put(convert2ColumnName(x), y)); - Iterator it1 = cols1.entrySet().iterator(); - while (it1.hasNext()) { - Map.Entry entry1 = (Map.Entry) it1.next(); - String colName1 = entry1.getKey(); - String colType1 = entry1.getValue(); - if (!cols2Copy.containsKey(convert2ColumnName(colName1))) { - complementaryColumns.add(getColumn(colName1, colType1)); - } - } - return complementaryColumns; - } - - public static void compareColumnsName(LinkedHashMap cols1, LinkedHashMap cols2, String cols1Name, String cols2Name) { - SchemaDifferenceResponse diffColumns = getDifferentColumns(cols1, cols2); - if (diffColumns.size() > 0) { - log.warn("Columns mapping warning: " + cols1Name + " columns are not aligned to " + cols2Name + " columns. " + - "Different " + cols1Name + " columns are:{}, " + cols2Name + " columns are:{}", - diffColumns.getCols1Diff(), diffColumns.getCols2Diff()); - } - } - - public static boolean isColumnsSizeEqual(LinkedHashMap cols1, - LinkedHashMap cols2, - String cols1Name, - String cols2Name) { - if (cols1.size() != cols2.size()) { - log.warn("Columns mapping warning: " + cols1Name + " columns and " + cols2Name + " columns size are unequal. " + - cols1Name + " columns size:{} " + cols2Name + " columns size:{}, complementary Columns set are:{}", - cols1.size(), cols2.size(), getComplementaryColumns(cols1, cols2)); - return false; - } - return true; - } - - private static boolean compareColumnName(String left, String right) { - if (null == left || null == right) { - return false; - } - left = convert2ColumnName(left); - right = convert2ColumnName(right); - return left.equals(right); - } - - public static SchemaDifferenceResponse getDifferentColumns(LinkedHashMap left, LinkedHashMap right) { - List cols1Diff = new ArrayList<>(); - List cols2Diff = new ArrayList<>(); - if (left.size() != right.size()) { - log.info("Columns sizes are not equal, cols1 size:{} cols2 size:{},skip different columns check.", left.size(), right.size()); - return new SchemaDifferenceResponse(cols1Diff, cols2Diff); - } - Iterator> it1 = left.entrySet().iterator(); - Iterator> it2 = right.entrySet().iterator(); - while (it1.hasNext()) { - Map.Entry entry1 = it1.next(); - Map.Entry entry2 = it2.next(); - String colName1 = entry1.getKey(); - String colType1 = entry1.getValue(); - String colName2 = entry2.getKey(); - String colType2 = entry2.getValue(); - if (!compareColumnName(colName1, colName2)) { - cols1Diff.add(getColumn(colName1, colType1)); - cols2Diff.add(getColumn(colName2, colType2)); - } - } - return new SchemaDifferenceResponse(cols1Diff, cols2Diff); - } - - private static String convert2ColumnName(String colName) { - String[] split = colName.toLowerCase().replace("_", "").split("\\."); - colName = split.length > 0 ? split[split.length - 1] : null; - return colName; - } - - private static ColumnInfo getColumn(String name, String type) { - return new ColumnInfo(name, type); - } - - private static Set toColumnNamesSet(List columns) { - return columns.stream().map(SchemaHelper::getUpperCaseColName).collect(toSet()); - } - - private static List sortColumnsByOtherColumnNames(List writerCols, List readerCols) { - // Map - final Map writerColsMap = writerCols.stream().collect( - Collectors.toMap(SchemaHelper::getUpperCaseColName, Function.identity())); - - return readerCols.stream().map( - readerCol -> writerColsMap.get(getUpperCaseColName(readerCol))).collect(toList()); - } - - private static String getUpperCaseColName(ColumnInfo columnInf) { - return columnInf.getName().toUpperCase(); - } - - private static List filterColumnsByNames(List readerCols, Set intersectionColumnNames) { - return readerCols.stream() - .filter(m -> intersectionColumnNames.contains(getUpperCaseColName(m))).collect(Collectors.toList()); - } - - @AllArgsConstructor - @Getter - public static class SchemaIntersectionResponse { - private List readerColumns; - private List writerColumns; - } - - @AllArgsConstructor - @Getter - public static class SchemaDifferenceResponse { - private List cols1Diff; - private List cols2Diff; - - public int size() { - return cols1Diff.size(); - } - } -} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/sink/SinkEngineConnector.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/sink/SinkEngineConnector.java deleted file mode 100644 index a6172d458..000000000 --- a/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/sink/SinkEngineConnector.java +++ /dev/null @@ -1,39 +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 com.bytedance.bitsail.common.ddl.sink; - -import com.bytedance.bitsail.common.ddl.ExternalEngineConnector; -import com.bytedance.bitsail.common.model.ColumnInfo; - -import java.util.List; - -public interface SinkEngineConnector extends ExternalEngineConnector { - - void addColumns(List columnsToAdd) throws Exception; - - void deleteColumns(List columnsToDelete) throws Exception; - - void updateColumns(List columnsToUpdate) throws Exception; - - boolean isTypeCompatible(String newer, String older); - - /** - * get columns which in the excluded list. - */ - List getExcludedColumnInfos() throws Exception; -} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/sink/SinkEngineConnectorBase.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/sink/SinkEngineConnectorBase.java deleted file mode 100644 index fc73c67a5..000000000 --- a/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/sink/SinkEngineConnectorBase.java +++ /dev/null @@ -1,37 +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 com.bytedance.bitsail.common.ddl.sink; - -import com.bytedance.bitsail.common.configuration.BitSailConfiguration; - -import lombok.Getter; -import lombok.extern.slf4j.Slf4j; - -@Getter -@Slf4j -public abstract class SinkEngineConnectorBase implements SinkEngineConnector { - - protected BitSailConfiguration commonConfiguration; - protected BitSailConfiguration writerConfiguration; - - public SinkEngineConnectorBase(BitSailConfiguration commonConfiguration, - BitSailConfiguration writerConfiguration) { - this.commonConfiguration = commonConfiguration; - this.writerConfiguration = writerConfiguration; - } -} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/source/SourceEngineConnector.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/source/SourceEngineConnector.java deleted file mode 100644 index acb79ab06..000000000 --- a/bitsail-common/src/main/java/com/bytedance/bitsail/common/ddl/source/SourceEngineConnector.java +++ /dev/null @@ -1,34 +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 com.bytedance.bitsail.common.ddl.source; - -import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.ExternalEngineConnector; - -public abstract class SourceEngineConnector implements ExternalEngineConnector { - - protected BitSailConfiguration commonConfiguration; - - protected BitSailConfiguration readerConfiguration; - - public SourceEngineConnector(BitSailConfiguration commonConfiguration, - BitSailConfiguration readerConfiguration) { - this.commonConfiguration = commonConfiguration; - this.readerConfiguration = readerConfiguration; - } -} diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/option/CommonOptions.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/option/CommonOptions.java index 14a4d1896..55be94b75 100644 --- a/bitsail-common/src/main/java/com/bytedance/bitsail/common/option/CommonOptions.java +++ b/bitsail-common/src/main/java/com/bytedance/bitsail/common/option/CommonOptions.java @@ -182,45 +182,6 @@ public interface CommonOptions { key(COMMON_PREFIX + "low_volume_test_count_threshold") .defaultValue(-1L); - ConfigOption COLUMN_ALIGN_STRATEGY = - key(COMMON_PREFIX + "column_align_strategy") - .defaultValue("disable"); - - /** - * Whether enable the ddl sync feature. - */ - ConfigOption SYNC_DDL = - key(COMMON_PREFIX + "sync_ddl") - .defaultValue(false); - - ConfigOption SYNC_DDL_SKIP_ERROR_COLUMNS = - key(COMMON_PREFIX + "sync_ddl_skip_error_columns") - .defaultValue(true); - - ConfigOption SYNC_DDL_PRE_EXECUTE = - key(COMMON_PREFIX + "sync_ddl_pre_execute") - .defaultValue(false); - - /** - * Ignore ddl delete fields. - */ - ConfigOption SYNC_DDL_IGNORE_DROP = - key(COMMON_PREFIX + "sync_ddl_ignore_drop") - .defaultValue(true); - /** - * Ignore ddl new added fields. - */ - ConfigOption SYNC_DDL_IGNORE_ADD = - key(COMMON_PREFIX + "sync_ddl_ignore_add") - .defaultValue(false); - - /** - * Ignore ddl updated fields. - */ - ConfigOption SYNC_DDL_IGNORE_UPDATE = - key(COMMON_PREFIX + "sync_ddl_ignore_update") - .defaultValue(false); - ConfigOption DRY_RUN = key(COMMON_PREFIX + "dry_run") .defaultValue(false); diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/type/BitSailTypeParser.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/type/BitSailTypeParser.java index eec409f03..6e3a7d0d0 100644 --- a/bitsail-common/src/main/java/com/bytedance/bitsail/common/type/BitSailTypeParser.java +++ b/bitsail-common/src/main/java/com/bytedance/bitsail/common/type/BitSailTypeParser.java @@ -44,7 +44,7 @@ public class BitSailTypeParser { private static final String SPLIT_TOKEN = ","; public static String fromTypeInfo(TypeInfo typeInfo) { - throw new UnsupportedOperationException(); + return TypeInfoBridge.bridgeTypes(typeInfo); } public static List fromTypePropertyString(String typePropertyString) { diff --git a/bitsail-common/src/main/java/com/bytedance/bitsail/common/typeinfo/TypeInfoBridge.java b/bitsail-common/src/main/java/com/bytedance/bitsail/common/typeinfo/TypeInfoBridge.java index 60cbed311..428973755 100644 --- a/bitsail-common/src/main/java/com/bytedance/bitsail/common/typeinfo/TypeInfoBridge.java +++ b/bitsail-common/src/main/java/com/bytedance/bitsail/common/typeinfo/TypeInfoBridge.java @@ -19,10 +19,14 @@ package com.bytedance.bitsail.common.typeinfo; +import com.bytedance.bitsail.common.BitSailException; +import com.bytedance.bitsail.common.exception.CommonErrorCode; + import com.google.common.collect.Maps; import org.apache.commons.lang3.StringUtils; import java.util.Map; +import java.util.Objects; public class TypeInfoBridge { @@ -32,6 +36,9 @@ public class TypeInfoBridge { public static final Map> TYPE_INFO_NAME_MAPPING = Maps.newHashMap(); + public static final Map, Types> TYPE_INFO_TYPES_MAPPING = + Maps.newHashMap(); + static { TYPE_INFO_MAPPING.put(Types.VOID, TypeInfos.VOID_TYPE_INFO); TYPE_INFO_MAPPING.put(Types.SHORT, TypeInfos.SHORT_TYPE_INFO); @@ -60,6 +67,7 @@ public class TypeInfoBridge { TYPE_INFO_NAME_MAPPING.put(StringUtils.upperCase(type.getTypeStringNickName()), TYPE_INFO_MAPPING.get(type)); } + TYPE_INFO_TYPES_MAPPING.put(TYPE_INFO_MAPPING.get(type).getTypeClass(), type); } } @@ -67,4 +75,17 @@ public static TypeInfo bridgeTypeInfo(String typeString) { return TYPE_INFO_NAME_MAPPING.get(typeString); } + public static String bridgeTypes(TypeInfo typeInfo) { + Class typeClass = typeInfo.getTypeClass(); + + Types types = TYPE_INFO_TYPES_MAPPING.get(typeClass); + if (Objects.nonNull(types)) { + return StringUtils.isNotEmpty(types.getTypeStringNickName()) ? + types.getTypeStringNickName() : + types.name().toLowerCase(); + } + throw BitSailException.asBitSailException(CommonErrorCode.INTERNAL_ERROR, + String.format("Not support bridge complex type info %s.", typeInfo)); + } + } diff --git a/bitsail-common/src/test/java/com/bytedance/bitsail/common/catalog/TableCatalogManagerTest.java b/bitsail-common/src/test/java/com/bytedance/bitsail/common/catalog/TableCatalogManagerTest.java new file mode 100644 index 000000000..2fa3962ea --- /dev/null +++ b/bitsail-common/src/test/java/com/bytedance/bitsail/common/catalog/TableCatalogManagerTest.java @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.common.catalog; + +import com.bytedance.bitsail.common.catalog.fake.FakeTableCatalog; +import com.bytedance.bitsail.common.catalog.table.CatalogTableColumn; +import com.bytedance.bitsail.common.catalog.table.CatalogTableDefinition; +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.common.model.ColumnInfo; +import com.bytedance.bitsail.common.option.ReaderOptions; +import com.bytedance.bitsail.common.option.WriterOptions; +import com.bytedance.bitsail.common.type.BitSailTypeInfoConverter; + +import com.google.common.collect.Lists; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; + +public class TableCatalogManagerTest { + + private BitSailConfiguration commonConfiguration; + private BitSailConfiguration readerConfiguration; + private BitSailConfiguration writerConfiguration; + + @Before + public void before() { + commonConfiguration = BitSailConfiguration.newDefault(); + readerConfiguration = BitSailConfiguration.newDefault(); + writerConfiguration = BitSailConfiguration.newDefault(); + } + + @Test + public void testTableCatalogAlignmentIntersect() throws Exception { + List readerColumns = Lists.newArrayList(); + List writerColumns = Lists.newArrayList(); + + readerColumns.add(ColumnInfo.builder() + .name("name") + .type("string") + .build()); + + readerColumns.add(ColumnInfo.builder() + .name("id") + .type("int") + .build()); + + writerColumns.add(ColumnInfo.builder() + .name("name") + .type("string") + .build()); + + commonConfiguration.set(TableCatalogOptions.COLUMN_ALIGN_STRATEGY, + TableCatalogStrategy.INTERSECT.name()); + commonConfiguration.set(TableCatalogOptions.SYNC_DDL, true); + FakeTableCatalog readerFakeTableCatalog = new FakeTableCatalog(readerColumns, CatalogTableDefinition.builder() + .database("a") + .table("b").build()); + FakeTableCatalog writerFakeTableCatalog = new FakeTableCatalog(writerColumns, CatalogTableDefinition.builder() + .database("a") + .table("c").build()); + + TableCatalogManager tableCatalogManager = TableCatalogManager.builder() + .readerTableCatalog(readerFakeTableCatalog) + .writerTableCatalog(writerFakeTableCatalog) + .writerTypeInfoConverter(new BitSailTypeInfoConverter()) + .readerTypeInfoConverter(new BitSailTypeInfoConverter()) + .commonConfiguration(commonConfiguration) + .readerConfiguration(readerConfiguration) + .writerConfiguration(writerConfiguration) + .build(); + + tableCatalogManager.alignmentCatalogTable(); + + List finalReaderColumns = readerConfiguration.get(ReaderOptions.BaseReaderOptions.COLUMNS); + List finalWriterColumns = writerConfiguration.get(WriterOptions.BaseWriterOptions.COLUMNS); + + Assert.assertEquals(finalReaderColumns.size(), finalWriterColumns.size()); + } + + @Test + public void testTableCatalogAlignmentSourceOnly() throws Exception { + List readerColumns = Lists.newArrayList(); + List writerColumns = Lists.newArrayList(); + + readerColumns.add(ColumnInfo.builder() + .name("name") + .type("string") + .build()); + + readerColumns.add(ColumnInfo.builder() + .name("id") + .type("int") + .build()); + + writerColumns.add(ColumnInfo.builder() + .name("name") + .type("string") + .build()); + + commonConfiguration.set(TableCatalogOptions.COLUMN_ALIGN_STRATEGY, + TableCatalogStrategy.SOURCE_ONLY.name()); + commonConfiguration.set(TableCatalogOptions.SYNC_DDL, true); + + FakeTableCatalog readerFakeTableCatalog = new FakeTableCatalog(readerColumns, CatalogTableDefinition.builder() + .database("a") + .table("b").build()); + FakeTableCatalog writerFakeTableCatalog = new FakeTableCatalog(writerColumns, CatalogTableDefinition.builder() + .database("a") + .table("c").build()); + + TableCatalogManager tableCatalogManager = TableCatalogManager.builder() + .readerTableCatalog(readerFakeTableCatalog) + .writerTableCatalog(writerFakeTableCatalog) + .writerTypeInfoConverter(new BitSailTypeInfoConverter()) + .readerTypeInfoConverter(new BitSailTypeInfoConverter()) + .commonConfiguration(commonConfiguration) + .readerConfiguration(readerConfiguration) + .writerConfiguration(writerConfiguration) + .build(); + + tableCatalogManager.alignmentCatalogTable(); + + List finalReaderColumns = readerConfiguration.get(ReaderOptions.BaseReaderOptions.COLUMNS); + List finalWriterColumns = writerConfiguration.get(WriterOptions.BaseWriterOptions.COLUMNS); + + Assert.assertEquals(finalReaderColumns.size(), finalWriterColumns.size()); + List addedTableColumns = writerFakeTableCatalog.getAddedTableColumns(); + Assert.assertEquals(addedTableColumns.size(), 1); + } + + @Test + public void testColumnAlignmentUpdate() throws Exception { + List readerColumns = Lists.newArrayList(); + List writerColumns = Lists.newArrayList(); + + readerColumns.add(ColumnInfo.builder() + .name("name") + .type("int") + .build()); + + readerColumns.add(ColumnInfo.builder() + .name("id") + .type("int") + .build()); + + writerColumns.add(ColumnInfo.builder() + .name("name") + .type("string") + .build()); + + commonConfiguration.set(TableCatalogOptions.COLUMN_ALIGN_STRATEGY, TableCatalogStrategy.SOURCE_ONLY.name()); + commonConfiguration.set(TableCatalogOptions.SYNC_DDL, true); + + FakeTableCatalog readerFakeTableCatalog = new FakeTableCatalog(readerColumns, CatalogTableDefinition.builder() + .database("a") + .table("b").build()); + FakeTableCatalog writerFakeTableCatalog = new FakeTableCatalog(writerColumns, CatalogTableDefinition.builder() + .database("a") + .table("c").build()); + + TableCatalogManager tableCatalogManager = TableCatalogManager.builder() + .readerTableCatalog(readerFakeTableCatalog) + .writerTableCatalog(writerFakeTableCatalog) + .writerTypeInfoConverter(new BitSailTypeInfoConverter()) + .readerTypeInfoConverter(new BitSailTypeInfoConverter()) + .commonConfiguration(commonConfiguration) + .readerConfiguration(readerConfiguration) + .writerConfiguration(writerConfiguration) + .build(); + + tableCatalogManager.alignmentCatalogTable(); + + List finalReaderColumns = readerConfiguration.get(ReaderOptions.BaseReaderOptions.COLUMNS); + List finalWriterColumns = writerConfiguration.get(WriterOptions.BaseWriterOptions.COLUMNS); + + Assert.assertEquals(finalReaderColumns.size(), finalWriterColumns.size()); + List addedTableColumns = writerFakeTableCatalog.getAddedTableColumns(); + Assert.assertEquals(addedTableColumns.size(), 1); + Assert.assertEquals(writerFakeTableCatalog.getUpdatedTableColumns().size(), 1); + } +} \ No newline at end of file diff --git a/bitsail-common/src/test/java/com/bytedance/bitsail/common/catalog/fake/FakeTableCatalog.java b/bitsail-common/src/test/java/com/bytedance/bitsail/common/catalog/fake/FakeTableCatalog.java new file mode 100644 index 000000000..c914b3974 --- /dev/null +++ b/bitsail-common/src/test/java/com/bytedance/bitsail/common/catalog/fake/FakeTableCatalog.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.common.catalog.fake; + +import com.bytedance.bitsail.common.catalog.table.CatalogTable; +import com.bytedance.bitsail.common.catalog.table.CatalogTableColumn; +import com.bytedance.bitsail.common.catalog.table.CatalogTableDefinition; +import com.bytedance.bitsail.common.catalog.table.CatalogTableSchema; +import com.bytedance.bitsail.common.catalog.table.TableCatalog; +import com.bytedance.bitsail.common.catalog.table.TableOperation; +import com.bytedance.bitsail.common.model.ColumnInfo; +import com.bytedance.bitsail.common.type.BitSailTypeInfoConverter; +import com.bytedance.bitsail.common.type.TypeInfoConverter; +import com.bytedance.bitsail.common.typeinfo.TypeInfo; + +import lombok.Getter; +import org.apache.commons.compress.utils.Lists; + +import java.util.List; +import java.util.stream.Collectors; + +public class FakeTableCatalog implements TableCatalog { + + private final List columnInfos; + + private final CatalogTableDefinition tableDefinition; + + private final TypeInfoConverter typeInfoConverter; + + private final List catalogTableColumns; + + @Getter + private final List addedTableColumns; + @Getter + private final List updatedTableColumns; + @Getter + private final List deletedTableColumns; + + public FakeTableCatalog(List columnInfos, + CatalogTableDefinition tableDefinition) { + this.columnInfos = columnInfos; + this.tableDefinition = tableDefinition; + this.typeInfoConverter = new BitSailTypeInfoConverter(); + this.catalogTableColumns = columnInfos.stream() + .map(column -> + CatalogTableColumn.builder() + .name(column.getName()) + .type(typeInfoConverter.fromTypeString(column.getType())) + .build() + ).collect(Collectors.toList()); + addedTableColumns = Lists.newArrayList(); + updatedTableColumns = Lists.newArrayList(); + deletedTableColumns = Lists.newArrayList(); + } + + @Override + public void open(TypeInfoConverter typeInfoConverter) { + + } + + @Override + public void close() { + + } + + @Override + public CatalogTableDefinition createCatalogTableDefinition() { + return tableDefinition; + } + + @Override + public boolean tableExists(CatalogTableDefinition catalogTableDefinition) { + return true; + } + + @Override + public CatalogTable getCatalogTable(CatalogTableDefinition catalogTableDefinition) { + CatalogTableSchema tableSchema = CatalogTableSchema.builder() + .columns(catalogTableColumns) + .primaryKeys(null) + .build(); + return CatalogTable.builder() + .catalogTableDefinition(catalogTableDefinition) + .catalogTableSchema(tableSchema) + .build(); + } + + @Override + public void createTable(CatalogTableDefinition catalogTableDefinition, CatalogTable catalogTable) { + + } + + @Override + public void alterTable(TableOperation tableOperation, CatalogTable table) { + + } + + @Override + public void alterTableColumns(TableOperation tableOperation, List catalogTableColumns) { + if (TableOperation.ALTER_COLUMNS_ADD.equals(tableOperation)) { + addedTableColumns.addAll(catalogTableColumns); + } + if (TableOperation.ALTER_COLUMNS_UPDATE.equals(tableOperation)) { + updatedTableColumns.addAll(catalogTableColumns); + } + if (TableOperation.ALTER_COLUMNS_DELETE.equals(tableOperation)) { + deletedTableColumns.addAll(catalogTableColumns); + } + } + + @Override + public boolean compareTypeCompatible(TypeInfo original, TypeInfo compared) { + return original.getTypeClass() == compared.getTypeClass(); + } + + @Override + public List convertTableColumn(TypeInfoConverter typeInfoConverter, List columnInfos) { + return TableCatalog.super.convertTableColumn(typeInfoConverter, columnInfos); + } +} diff --git a/bitsail-common/src/test/java/com/bytedance/bitsail/common/ddl/DdlSyncManagerTest.java b/bitsail-common/src/test/java/com/bytedance/bitsail/common/ddl/DdlSyncManagerTest.java deleted file mode 100644 index 5eab2b6c8..000000000 --- a/bitsail-common/src/test/java/com/bytedance/bitsail/common/ddl/DdlSyncManagerTest.java +++ /dev/null @@ -1,163 +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 com.bytedance.bitsail.common.ddl; - -import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.external.FakeSinkEngineConnector; -import com.bytedance.bitsail.common.ddl.external.FakeSourceEngineConnector; -import com.bytedance.bitsail.common.model.ColumnInfo; -import com.bytedance.bitsail.common.option.CommonOptions; -import com.bytedance.bitsail.common.option.ReaderOptions; -import com.bytedance.bitsail.common.option.WriterOptions; - -import com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.List; - -/** - * Created 2022/8/23 - */ -public class DdlSyncManagerTest { - private BitSailConfiguration commonConfiguration; - private BitSailConfiguration readerConfiguration; - private BitSailConfiguration writerConfiguration; - - private FakeSourceEngineConnector sourceEngineConnector; - private FakeSinkEngineConnector sinkEngineConnector; - - private DdlSyncManager syncManager; - - @Before - public void before() { - commonConfiguration = BitSailConfiguration.newDefault(); - readerConfiguration = BitSailConfiguration.newDefault(); - writerConfiguration = BitSailConfiguration.newDefault(); - } - - @Test - public void testColumnAlignment() throws Exception { - List readerColumns = Lists.newArrayList(); - List writerColumns = Lists.newArrayList(); - - readerColumns.add(ColumnInfo.builder() - .name("name") - .type("string") - .build()); - - readerColumns.add(ColumnInfo.builder() - .name("id") - .type("int") - .build()); - - writerColumns.add(ColumnInfo.builder() - .name("name") - .type("string") - .build()); - - commonConfiguration.set(CommonOptions.COLUMN_ALIGN_STRATEGY, SchemaColumnAligner.ColumnAlignmentStrategy.intersect.name()); - commonConfiguration.set(CommonOptions.SYNC_DDL, true); - sourceEngineConnector = new FakeSourceEngineConnector(readerColumns, commonConfiguration, readerConfiguration); - sinkEngineConnector = new FakeSinkEngineConnector(writerColumns, commonConfiguration, readerConfiguration); - - syncManager = new DdlSyncManager(sourceEngineConnector, sinkEngineConnector, commonConfiguration, readerConfiguration, writerConfiguration); - syncManager.doColumnAlignment(false); - - List finalReaderColumns = readerConfiguration.get(ReaderOptions.BaseReaderOptions.COLUMNS); - List finalWriterColumns = writerConfiguration.get(WriterOptions.BaseWriterOptions.COLUMNS); - - Assert.assertEquals(finalReaderColumns.size(), finalWriterColumns.size()); - } - - @Test - public void testColumnAlignmentSourceOnly() throws Exception { - List readerColumns = Lists.newArrayList(); - List writerColumns = Lists.newArrayList(); - - readerColumns.add(ColumnInfo.builder() - .name("name") - .type("string") - .build()); - - readerColumns.add(ColumnInfo.builder() - .name("id") - .type("int") - .build()); - - writerColumns.add(ColumnInfo.builder() - .name("name") - .type("string") - .build()); - - commonConfiguration.set(CommonOptions.COLUMN_ALIGN_STRATEGY, SchemaColumnAligner.ColumnAlignmentStrategy.source_only.name()); - commonConfiguration.set(CommonOptions.SYNC_DDL, true); - - sourceEngineConnector = new FakeSourceEngineConnector(readerColumns, commonConfiguration, readerConfiguration); - sinkEngineConnector = new FakeSinkEngineConnector(writerColumns, commonConfiguration, readerConfiguration); - - syncManager = new DdlSyncManager(sourceEngineConnector, sinkEngineConnector, commonConfiguration, readerConfiguration, writerConfiguration); - syncManager.doColumnAlignment(false); - - List finalReaderColumns = readerConfiguration.get(ReaderOptions.BaseReaderOptions.COLUMNS); - List finalWriterColumns = writerConfiguration.get(WriterOptions.BaseWriterOptions.COLUMNS); - - Assert.assertEquals(finalReaderColumns.size(), finalWriterColumns.size()); - List addedColumns = sinkEngineConnector.getAddedColumns(); - Assert.assertEquals(addedColumns.size(), 1); - } - - @Test - public void testColumnAlignmentUpdate() throws Exception { - List readerColumns = Lists.newArrayList(); - List writerColumns = Lists.newArrayList(); - - readerColumns.add(ColumnInfo.builder() - .name("name") - .type("int") - .build()); - - readerColumns.add(ColumnInfo.builder() - .name("id") - .type("int") - .build()); - - writerColumns.add(ColumnInfo.builder() - .name("name") - .type("string") - .build()); - - commonConfiguration.set(CommonOptions.COLUMN_ALIGN_STRATEGY, SchemaColumnAligner.ColumnAlignmentStrategy.source_only.name()); - commonConfiguration.set(CommonOptions.SYNC_DDL, true); - - sourceEngineConnector = new FakeSourceEngineConnector(readerColumns, commonConfiguration, readerConfiguration); - sinkEngineConnector = new FakeSinkEngineConnector(writerColumns, commonConfiguration, readerConfiguration); - - syncManager = new DdlSyncManager(sourceEngineConnector, sinkEngineConnector, commonConfiguration, readerConfiguration, writerConfiguration); - syncManager.doColumnAlignment(false); - - List finalReaderColumns = readerConfiguration.get(ReaderOptions.BaseReaderOptions.COLUMNS); - List finalWriterColumns = writerConfiguration.get(WriterOptions.BaseWriterOptions.COLUMNS); - - Assert.assertEquals(finalReaderColumns.size(), finalWriterColumns.size()); - List addedColumns = sinkEngineConnector.getAddedColumns(); - Assert.assertEquals(addedColumns.size(), 1); - Assert.assertEquals(sinkEngineConnector.getUpdatedColumns().size(), 1); - } -} \ No newline at end of file diff --git a/bitsail-common/src/test/java/com/bytedance/bitsail/common/ddl/external/FakeSinkEngineConnector.java b/bitsail-common/src/test/java/com/bytedance/bitsail/common/ddl/external/FakeSinkEngineConnector.java deleted file mode 100644 index 2352c511f..000000000 --- a/bitsail-common/src/test/java/com/bytedance/bitsail/common/ddl/external/FakeSinkEngineConnector.java +++ /dev/null @@ -1,91 +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 com.bytedance.bitsail.common.ddl.external; - -import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.sink.SinkEngineConnectorBase; -import com.bytedance.bitsail.common.model.ColumnInfo; -import com.bytedance.bitsail.common.type.TypeInfoConverter; -import com.bytedance.bitsail.common.type.filemapping.FileMappingTypeInfoConverter; - -import com.google.common.collect.Lists; -import lombok.Getter; - -import java.util.List; - -/** - * Created 2022/8/23 - */ -@Getter -public class FakeSinkEngineConnector extends SinkEngineConnectorBase { - private List columnInfos; - - private List updatedColumns; - private List deletedColumns; - private List addedColumns; - - public FakeSinkEngineConnector(List columnInfos, - BitSailConfiguration commonConfiguration, - BitSailConfiguration writerConfiguration) { - super(commonConfiguration, writerConfiguration); - this.columnInfos = columnInfos; - updatedColumns = Lists.newArrayList(); - deletedColumns = Lists.newArrayList(); - addedColumns = Lists.newArrayList(); - } - - @Override - public String getExternalEngineName() { - return "fake"; - } - - @Override - public List getExternalColumnInfos() throws Exception { - return columnInfos; - } - - @Override - public TypeInfoConverter createTypeInfoConverter() { - return new FileMappingTypeInfoConverter(getExternalEngineName()); - } - - @Override - public void addColumns(List columnsToAdd) throws Exception { - addedColumns = columnsToAdd; - } - - @Override - public void deleteColumns(List columnsToDelete) throws Exception { - deletedColumns = columnsToDelete; - } - - @Override - public void updateColumns(List columnsToUpdate) throws Exception { - updatedColumns = columnsToUpdate; - } - - @Override - public boolean isTypeCompatible(String newer, String older) { - return false; - } - - @Override - public List getExcludedColumnInfos() throws Exception { - return Lists.newArrayList(); - } -} diff --git a/bitsail-common/src/test/java/com/bytedance/bitsail/common/ddl/external/FakeSourceEngineConnector.java b/bitsail-common/src/test/java/com/bytedance/bitsail/common/ddl/external/FakeSourceEngineConnector.java deleted file mode 100644 index dd0665115..000000000 --- a/bitsail-common/src/test/java/com/bytedance/bitsail/common/ddl/external/FakeSourceEngineConnector.java +++ /dev/null @@ -1,56 +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 com.bytedance.bitsail.common.ddl.external; - -import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.source.SourceEngineConnector; -import com.bytedance.bitsail.common.model.ColumnInfo; -import com.bytedance.bitsail.common.type.TypeInfoConverter; -import com.bytedance.bitsail.common.type.filemapping.FileMappingTypeInfoConverter; - -import java.util.List; - -/** - * Created 2022/8/23 - */ -public class FakeSourceEngineConnector extends SourceEngineConnector { - - private List columnInfos; - - public FakeSourceEngineConnector(List columnInfos, - BitSailConfiguration commonConfiguration, - BitSailConfiguration readerConfiguration) { - super(commonConfiguration, readerConfiguration); - this.columnInfos = columnInfos; - } - - @Override - public List getExternalColumnInfos() throws Exception { - return columnInfos; - } - - @Override - public TypeInfoConverter createTypeInfoConverter() { - return new FileMappingTypeInfoConverter(getExternalEngineName()); - } - - @Override - public String getExternalEngineName() { - return "fake"; - } -} diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-fake/src/main/java/com/bytedance/bitsail/connector/legacy/fake/source/FakeSource.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-fake/src/main/java/com/bytedance/bitsail/connector/legacy/fake/source/FakeSource.java index e771b45c3..47d2ed412 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-fake/src/main/java/com/bytedance/bitsail/connector/legacy/fake/source/FakeSource.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-fake/src/main/java/com/bytedance/bitsail/connector/legacy/fake/source/FakeSource.java @@ -26,6 +26,7 @@ import com.bytedance.bitsail.common.model.ColumnInfo; import com.bytedance.bitsail.common.option.ReaderOptions; import com.bytedance.bitsail.common.type.BitSailTypeInfoConverter; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.connector.legacy.fake.option.FakeReaderOptions; import com.bytedance.bitsail.flink.core.legacy.connector.InputFormatPlugin; import com.bytedance.bitsail.flink.core.typeinfo.PrimitiveColumnTypeInfo; @@ -206,7 +207,7 @@ public void initPlugin() throws Exception { rate = inputSliceConfig.get(FakeReaderOptions.RATE); randomNullInt = (int) Math.floor(inputSliceConfig.get(FakeReaderOptions.RANDOM_NULL_RATE) * 10); this.columnInfos = inputSliceConfig.get(ReaderOptions.BaseReaderOptions.COLUMNS); - this.rowTypeInfo = ColumnFlinkTypeInfoUtil.getRowTypeInformation(new BitSailTypeInfoConverter(), columnInfos); + this.rowTypeInfo = ColumnFlinkTypeInfoUtil.getRowTypeInformation(createTypeInfoConverter(), columnInfos); this.uniqueFields = initUniqueFieldsMapping(inputSliceConfig.get(FakeReaderOptions.UNIQUE_FIELDS)); this.useBitSailType = inputSliceConfig.get(FakeReaderOptions.USE_BITSAIL_TYPE); @@ -269,6 +270,11 @@ public TypeInformation getProducedType() { return rowTypeInfo; } + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new BitSailTypeInfoConverter(); + } + private static class FakeInputSplit implements InputSplit { @Override diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-ftp/src/main/java/com/bytedance/bitsail/connector/legacy/ftp/source/FtpInputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-ftp/src/main/java/com/bytedance/bitsail/connector/legacy/ftp/source/FtpInputFormat.java index 177822e10..997c5443e 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-ftp/src/main/java/com/bytedance/bitsail/connector/legacy/ftp/source/FtpInputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-ftp/src/main/java/com/bytedance/bitsail/connector/legacy/ftp/source/FtpInputFormat.java @@ -21,6 +21,7 @@ import com.bytedance.bitsail.common.BitSailException; import com.bytedance.bitsail.common.exception.CommonErrorCode; import com.bytedance.bitsail.common.model.ColumnInfo; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.common.type.filemapping.FileMappingTypeInfoConverter; import com.bytedance.bitsail.component.format.api.RowBuilder; import com.bytedance.bitsail.connector.legacy.ftp.client.FtpHandlerFactory; @@ -73,7 +74,7 @@ public void initPlugin() throws Exception { this.rowBuilder = new TextRowBuilder(inputSliceConfig); List columnInfos = inputSliceConfig.getNecessaryOption(FtpReaderOptions.COLUMNS, FtpInputFormatErrorCode.REQUIRED_VALUE); - this.rowTypeInfo = ColumnFlinkTypeInfoUtil.getRowTypeInformation(new FileMappingTypeInfoConverter(StringUtils.lowerCase(getType())), columnInfos); + this.rowTypeInfo = ColumnFlinkTypeInfoUtil.getRowTypeInformation(createTypeInfoConverter(), columnInfos); log.info("Row Type Info: " + rowTypeInfo); } @@ -249,4 +250,9 @@ private void initFtpConfig() { this.ftpConfig.setProtocol(FtpConfig.Protocol.valueOf(protocol.toUpperCase())); this.ftpConfig.setConnectPattern(FtpConfig.ConnectPattern.valueOf(connectPattern.toUpperCase())); } + + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new FileMappingTypeInfoConverter(StringUtils.lowerCase(getType())); + } } diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hadoop/src/main/java/com/bytedance/bitsail/connector/hadoop/source/HadoopInputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hadoop/src/main/java/com/bytedance/bitsail/connector/hadoop/source/HadoopInputFormat.java index 540e3a2bd..915e6cff0 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hadoop/src/main/java/com/bytedance/bitsail/connector/hadoop/source/HadoopInputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hadoop/src/main/java/com/bytedance/bitsail/connector/hadoop/source/HadoopInputFormat.java @@ -21,6 +21,7 @@ import com.bytedance.bitsail.common.BitSailException; import com.bytedance.bitsail.common.model.ColumnInfo; import com.bytedance.bitsail.common.type.BitSailTypeInfoConverter; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.component.format.api.RowBuilder; import com.bytedance.bitsail.connector.hadoop.common.TextInputFormatErrorCode; import com.bytedance.bitsail.connector.hadoop.option.HadoopReaderOptions; @@ -90,7 +91,7 @@ public void initPlugin() throws Exception { } List columnInfos = inputSliceConfig.getNecessaryOption(HadoopReaderOptions.COLUMNS, TextInputFormatErrorCode.REQUIRED_VALUE); - this.rowTypeInfo = ColumnFlinkTypeInfoUtil.getRowTypeInformation(new BitSailTypeInfoConverter(), columnInfos); + this.rowTypeInfo = ColumnFlinkTypeInfoUtil.getRowTypeInformation(createTypeInfoConverter(), columnInfos); this.fieldIndex = createFieldIndexes(columnInfos); LOG.info("Row Type Info: " + rowTypeInfo); @@ -112,4 +113,9 @@ public Row buildRow(Row reuse, String mandatoryEncoding) throws BitSailException public TypeInformation getProducedType() { return rowTypeInfo; } + + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new BitSailTypeInfoConverter(); + } } diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hadoop/src/main/java/com/bytedance/bitsail/connector/hadoop/source/format/TextInputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hadoop/src/main/java/com/bytedance/bitsail/connector/hadoop/source/format/TextInputFormat.java index e4a8ef8b0..2ee21546d 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hadoop/src/main/java/com/bytedance/bitsail/connector/hadoop/source/format/TextInputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hadoop/src/main/java/com/bytedance/bitsail/connector/hadoop/source/format/TextInputFormat.java @@ -29,6 +29,7 @@ import com.bytedance.bitsail.common.configuration.BitSailConfiguration; import com.bytedance.bitsail.common.model.ColumnInfo; import com.bytedance.bitsail.common.type.BitSailTypeInfoConverter; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.connector.hadoop.common.TextInputFormatErrorCode; import com.bytedance.bitsail.connector.hadoop.option.HadoopReaderOptions; import com.bytedance.bitsail.flink.core.parser.BytesParser; @@ -117,7 +118,7 @@ public void initPlugin() throws Exception { List columnInfos = inputSliceConfig .getNecessaryOption(HadoopReaderOptions.COLUMNS, TextInputFormatErrorCode.REQUIRED_VALUE); - this.rowTypeInfo = ColumnFlinkTypeInfoUtil.getRowTypeInformation(new BitSailTypeInfoConverter(), columnInfos); + this.rowTypeInfo = ColumnFlinkTypeInfoUtil.getRowTypeInformation(createTypeInfoConverter(), columnInfos); LOG.info("Row type info: " + rowTypeInfo); } @@ -160,6 +161,11 @@ public TypeInformation getProducedType() { return rowTypeInfo; } + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new BitSailTypeInfoConverter(); + } + public static class TextInputFormatBuilder { private final TextInputFormat format; diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hbase/src/main/java/com/bytedance/bitsail/connector/hbase/sink/HBaseOutputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hbase/src/main/java/com/bytedance/bitsail/connector/hbase/sink/HBaseOutputFormat.java index fb7b6f1a5..0eca4427d 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hbase/src/main/java/com/bytedance/bitsail/connector/hbase/sink/HBaseOutputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hbase/src/main/java/com/bytedance/bitsail/connector/hbase/sink/HBaseOutputFormat.java @@ -21,6 +21,7 @@ import com.bytedance.bitsail.common.column.Column; import com.bytedance.bitsail.common.exception.CommonErrorCode; import com.bytedance.bitsail.common.model.ColumnInfo; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.common.type.filemapping.FileMappingTypeInfoConverter; import com.bytedance.bitsail.connector.hbase.HBaseHelper; import com.bytedance.bitsail.connector.hbase.auth.KerberosAuthenticator; @@ -393,7 +394,7 @@ public void initPlugin() throws Exception { List columns = outputSliceConfig.getNecessaryOption(HBaseWriterOptions.COLUMNS, HBasePluginErrorCode.REQUIRED_VALUE); columnNames = columns.stream().map(ColumnInfo::getName).collect(Collectors.toList()); columnTypes = columns.stream().map(ColumnInfo::getType).collect(Collectors.toList()); - rowTypeInfo = NativeFlinkTypeInfoUtil.getRowTypeInformation(columns, new FileMappingTypeInfoConverter(StringUtils.lowerCase(getType()))); + rowTypeInfo = NativeFlinkTypeInfoUtil.getRowTypeInformation(columns, createTypeInfoConverter()); } @Override @@ -448,4 +449,8 @@ private String buildRowKeyExpress(Object rowKeyInfo) { return expressBuilder.toString(); } + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new FileMappingTypeInfoConverter(StringUtils.lowerCase(getType())); + } } diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hbase/src/main/java/com/bytedance/bitsail/connector/hbase/source/HBaseInputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hbase/src/main/java/com/bytedance/bitsail/connector/hbase/source/HBaseInputFormat.java index 017590a0f..01ca6ef58 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hbase/src/main/java/com/bytedance/bitsail/connector/hbase/source/HBaseInputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hbase/src/main/java/com/bytedance/bitsail/connector/hbase/source/HBaseInputFormat.java @@ -26,6 +26,7 @@ import com.bytedance.bitsail.common.constants.Constants; import com.bytedance.bitsail.common.model.ColumnInfo; import com.bytedance.bitsail.common.option.ReaderOptions; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.common.type.filemapping.FileMappingTypeInfoConverter; import com.bytedance.bitsail.common.typeinfo.TypeInfo; import com.bytedance.bitsail.common.typeinfo.TypeInfoUtils; @@ -159,7 +160,7 @@ public void initPlugin() throws Exception { List columnInfos = inputSliceConfig.getNecessaryOption( HBaseReaderOptions.COLUMNS, HBasePluginErrorCode.REQUIRED_VALUE); typeInfos = - TypeInfoUtils.getTypeInfos(new FileMappingTypeInfoConverter(StringUtils.lowerCase(getType())), columnInfos); + TypeInfoUtils.getTypeInfos(createTypeInfoConverter(), columnInfos); columnNames = columnInfos.stream().map(ColumnInfo::getName) .collect(Collectors.toList()); @@ -362,4 +363,9 @@ public JobConf getConf() { hbaseConfig.forEach((key, value) -> jobConf.set(key, value.toString())); return jobConf; } + + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new FileMappingTypeInfoConverter(StringUtils.lowerCase(getType())); + } } diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/common/HiveSinkEngineConnector.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/common/HiveSinkEngineConnector.java deleted file mode 100644 index 423c6aee5..000000000 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/common/HiveSinkEngineConnector.java +++ /dev/null @@ -1,163 +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 com.bytedance.bitsail.connector.legacy.hive.common; - -import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.sink.SinkEngineConnectorBase; -import com.bytedance.bitsail.common.model.ColumnInfo; -import com.bytedance.bitsail.common.type.TypeInfoConverter; -import com.bytedance.bitsail.common.type.filemapping.HiveTypeInfoConverter; -import com.bytedance.bitsail.common.util.JsonSerializer; -import com.bytedance.bitsail.common.util.TypeConvertUtil.StorageEngine; -import com.bytedance.bitsail.connector.legacy.hive.option.HiveWriterOptions; - -import com.bytedance.bitsail.shaded.hive.client.HiveMetaClientUtil; - -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.hive.conf.HiveConf; - -import java.util.ArrayList; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -@Slf4j -public class HiveSinkEngineConnector extends SinkEngineConnectorBase { - - private static final int DECIMAL_DEFAULT_PRECISION = 28; - private static final int DECIMAL_DEFAULT_SCALE = 10; - private static final String DECIMAL = "decimal"; - private final String database; - private final String table; - private final List columnInfos; - private final Map columnMappings; - private transient HiveConf hiveConf; - - public HiveSinkEngineConnector(BitSailConfiguration commonConfiguration, - BitSailConfiguration writerConfiguration) { - super(commonConfiguration, writerConfiguration); - database = writerConfiguration.get(HiveWriterOptions.DB_NAME); - table = writerConfiguration.get(HiveWriterOptions.TABLE_NAME); - hiveConf = HiveMetaClientUtil.getHiveConf(JsonSerializer - .parseToMap(writerConfiguration.get(HiveWriterOptions.HIVE_METASTORE_PROPERTIES))); - try { - columnInfos = HiveMetaClientUtil.getColumnInfo(hiveConf, database, table); - columnMappings = columnInfos.stream() - .collect(Collectors.toMap(ColumnInfo::getName, columnInfo -> columnInfo)); - } catch (Exception e) { - throw new IllegalStateException(e); - } - } - - @Override - public void addColumns(List columnsToAdd) throws Exception { - if (CollectionUtils.isEmpty(columnsToAdd)) { - log.info("columns to add is empty, will not execute column add operation."); - return; - } - log.info("start to add columns to hive..., columns to add is : {}", columnsToAdd); - List columnInfosToAdd = formatColumnInfos(columnsToAdd, columnMappings, true); - HiveMetaClientUtil.addColumnInfos(hiveConf, database, table, columnInfosToAdd); - log.info("finished add columns to hive."); - } - - @Override - public void updateColumns(List columnsToUpdate) throws Exception { - if (CollectionUtils.isEmpty(columnsToUpdate)) { - log.info("columns to update is empty, will not execute column update operation."); - return; - } - log.info("start to add columns to hive..., columns to update is : {}", columnsToUpdate); - List columnInfosToUpdate = formatColumnInfos(columnsToUpdate, columnMappings, false); - HiveMetaClientUtil.updateColumnInfos(hiveConf, database, table, columnInfosToUpdate); - log.info("finished update columns to hive."); - } - - @Override - public void deleteColumns(List columnsToDelete) { - log.info("delete operation is configured to be ignored for now! So these column delete operations will be ignored: {}", columnsToDelete); - } - - @Override - public boolean isTypeCompatible(String newType, String oldType) { - //todo support the decimal(m,n) - if (oldType.contains(DECIMAL)) { - oldType = DECIMAL; - } - if (newType.contains(DECIMAL)) { - newType = DECIMAL; - } - return oldType.equalsIgnoreCase(newType); - } - - @Override - public List getExcludedColumnInfos() throws Exception { - return new ArrayList<>(HiveMetaClientUtil - .getPartitionKeys(hiveConf, database, table).keySet()); - } - - @Override - public List getExternalColumnInfos() throws Exception { - return columnInfos; - } - - @Override - public String getExternalEngineName() { - return StorageEngine.hive.name(); - } - - String addDecimalPrecision(String type) { - if (type.equalsIgnoreCase(DECIMAL)) { - return String.format("%s(%d,%d)", DECIMAL, DECIMAL_DEFAULT_PRECISION, DECIMAL_DEFAULT_SCALE); - } - return type; - } - - public List formatColumnInfos(List columns, - Map originColumnInfo, - boolean addMode) { - if (CollectionUtils.isEmpty(columns)) { - return new LinkedList<>(); - } - final String addColumnComment = "BitSail ddl sync add column"; - List columnInfos = new LinkedList<>(); - for (ColumnInfo column : columns) { - String type = addDecimalPrecision(column.getType()); - String commentInfo; - if (addMode) { - commentInfo = addColumnComment; - } else { - ColumnInfo columnInfo = originColumnInfo.get(column.getName()); - String comment = StringUtils.isEmpty(columnInfo.getComment()) ? "Origin comment is null." - : columnInfo.getComment(); - commentInfo = String.format("%s BitSail ddl sync update column type from %s to %s", - comment, columnInfo.getType(), type); - } - columnInfos.add(new ColumnInfo(column.getName(), type, commentInfo)); - } - return columnInfos; - } - - @Override - public TypeInfoConverter createTypeInfoConverter() { - return new HiveTypeInfoConverter(); - } -} diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/common/HiveSourceEngineConnector.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/common/HiveSourceEngineConnector.java deleted file mode 100644 index c01fc4939..000000000 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/common/HiveSourceEngineConnector.java +++ /dev/null @@ -1,69 +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 com.bytedance.bitsail.connector.legacy.hive.common; - -import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.source.SourceEngineConnector; -import com.bytedance.bitsail.common.exception.FrameworkErrorCode; -import com.bytedance.bitsail.common.model.ColumnInfo; -import com.bytedance.bitsail.common.type.TypeInfoConverter; -import com.bytedance.bitsail.common.type.filemapping.HiveTypeInfoConverter; -import com.bytedance.bitsail.common.util.TypeConvertUtil.StorageEngine; -import com.bytedance.bitsail.connector.legacy.hive.option.HiveReaderOptions; - -import com.bytedance.bitsail.shaded.hive.client.HiveMetaClientUtil; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.List; - -public class HiveSourceEngineConnector extends SourceEngineConnector { - private static final Logger LOG = LoggerFactory.getLogger(HiveSourceEngineConnector.class); - - private final String database; - private final String table; - private final HiveConf hiveConf; - - public HiveSourceEngineConnector(BitSailConfiguration commonConfiguration, - BitSailConfiguration readerConfiguration, - HiveConf hiveConf) { - super(commonConfiguration, readerConfiguration); - this.database = readerConfiguration.getNecessaryOption(HiveReaderOptions.DB_NAME, FrameworkErrorCode.REQUIRED_VALUE); - this.table = readerConfiguration.getNecessaryOption(HiveReaderOptions.TABLE_NAME, FrameworkErrorCode.REQUIRED_VALUE); - this.hiveConf = hiveConf; - - LOG.info("Hive metastore engine source connector database: {}, table: {}.", database, table); - } - - @Override - public List getExternalColumnInfos() throws Exception { - return HiveMetaClientUtil.getColumnInfo(hiveConf, database, table); - } - - @Override - public TypeInfoConverter createTypeInfoConverter() { - return new HiveTypeInfoConverter(); - } - - @Override - public String getExternalEngineName() { - return StorageEngine.hive.name(); - } -} diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/common/HiveTableCatalog.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/common/HiveTableCatalog.java new file mode 100644 index 000000000..e72cb5a45 --- /dev/null +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/common/HiveTableCatalog.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.connector.legacy.hive.common; + +import com.bytedance.bitsail.common.catalog.table.CatalogTable; +import com.bytedance.bitsail.common.catalog.table.CatalogTableColumn; +import com.bytedance.bitsail.common.catalog.table.CatalogTableDefinition; +import com.bytedance.bitsail.common.catalog.table.CatalogTableSchema; +import com.bytedance.bitsail.common.catalog.table.TableCatalog; +import com.bytedance.bitsail.common.catalog.table.TableOperation; +import com.bytedance.bitsail.common.model.ColumnInfo; +import com.bytedance.bitsail.common.type.TypeInfoConverter; +import com.bytedance.bitsail.common.typeinfo.TypeInfo; + +import com.bytedance.bitsail.shaded.hive.client.HiveMetaClientUtil; + +import lombok.Builder; +import org.apache.hadoop.hive.conf.HiveConf; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +@Builder +public class HiveTableCatalog implements TableCatalog { + private static final Logger LOG = LoggerFactory.getLogger(HiveTableCatalog.class); + + private final String namespace; + private final String database; + private final String table; + private final HiveConf hiveConf; + + private TypeInfoConverter typeInfoConverter; + + @Override + public void open(TypeInfoConverter typeInfoConverter) { + this.typeInfoConverter = typeInfoConverter; + //ignore + } + + @Override + public void close() { + //ignore + } + + @Override + public CatalogTableDefinition createCatalogTableDefinition() { + return CatalogTableDefinition + .builder() + .database(database) + .table(table) + .build(); + } + + @Override + public boolean tableExists(CatalogTableDefinition catalogTableDefinition) { + //todo real check. + return true; + } + + @Override + public CatalogTable getCatalogTable(CatalogTableDefinition catalogTableDefinition) { + try { + List columnInfo = HiveMetaClientUtil.getColumnInfo(hiveConf, + catalogTableDefinition.getDatabase(), + catalogTableDefinition.getTable()); + + return CatalogTable.builder() + .catalogTableDefinition(catalogTableDefinition) + .catalogTableSchema(getCatalogTableSchema(columnInfo)) + .build(); + } catch (Exception e) { + LOG.error("Acquire hive catalog table failed", e); + throw new IllegalStateException(); + } + } + + @Override + public void createTable(CatalogTableDefinition catalogTableDefinition, CatalogTable catalogTable) { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTable(TableOperation tableOperation, CatalogTable table) { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTableColumns(TableOperation tableOperation, + List catalogTableColumns) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean compareTypeCompatible(TypeInfo original, TypeInfo compared) { + return original.getTypeClass() == compared.getTypeClass(); + } + + private CatalogTableSchema getCatalogTableSchema(List columnInfos) { + List tableColumns = convertTableColumn(typeInfoConverter, columnInfos); + return CatalogTableSchema.builder() + .columns(tableColumns) + .build(); + } +} diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/common/HiveTableCatalogFactory.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/common/HiveTableCatalogFactory.java new file mode 100644 index 000000000..9d7aecd93 --- /dev/null +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/common/HiveTableCatalogFactory.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.connector.legacy.hive.common; + +import com.bytedance.bitsail.base.catalog.TableCatalogFactory; +import com.bytedance.bitsail.base.connector.BuilderGroup; +import com.bytedance.bitsail.base.execution.ExecutionEnviron; +import com.bytedance.bitsail.common.catalog.table.TableCatalog; +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.common.exception.FrameworkErrorCode; +import com.bytedance.bitsail.connector.legacy.hive.option.HiveReaderOptions; +import com.bytedance.bitsail.connector.legacy.hive.option.HiveWriterOptions; +import com.bytedance.bitsail.connector.legacy.hive.util.HiveConfUtils; + +public class HiveTableCatalogFactory implements TableCatalogFactory { + + @Override + public TableCatalog createTableCatalog(BuilderGroup builderGroup, + ExecutionEnviron executionEnviron, + BitSailConfiguration connectorConfiguration) { + if (BuilderGroup.READER.equals(builderGroup)) { + String database = connectorConfiguration + .getNecessaryOption(HiveReaderOptions.DB_NAME, FrameworkErrorCode.REQUIRED_VALUE); + String table = connectorConfiguration + .getNecessaryOption(HiveReaderOptions.TABLE_NAME, FrameworkErrorCode.REQUIRED_VALUE); + return HiveTableCatalog + .builder() + .database(database) + .table(table) + .namespace(null) + .hiveConf(HiveConfUtils.fromJsonProperties( + connectorConfiguration.get(HiveReaderOptions.HIVE_METASTORE_PROPERTIES))) + .build(); + } else { + String database = connectorConfiguration + .getNecessaryOption(HiveWriterOptions.DB_NAME, FrameworkErrorCode.REQUIRED_VALUE); + String table = connectorConfiguration + .getNecessaryOption(HiveWriterOptions.TABLE_NAME, FrameworkErrorCode.REQUIRED_VALUE); + return HiveTableCatalog + .builder() + .database(database) + .table(table) + .namespace(null) + .hiveConf(HiveConfUtils.fromJsonProperties( + connectorConfiguration.get(HiveWriterOptions.HIVE_METASTORE_PROPERTIES))) + .build(); + } + } + + @Override + public String getComponentName() { + return "hive"; + } +} diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/sink/HiveOutputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/sink/HiveOutputFormat.java index 0a805822f..9c6151e81 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/sink/HiveOutputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/sink/HiveOutputFormat.java @@ -19,18 +19,17 @@ import com.bytedance.bitsail.base.execution.ProcessResult; import com.bytedance.bitsail.common.BitSailException; -import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.sink.SinkEngineConnector; import com.bytedance.bitsail.common.exception.CommonErrorCode; import com.bytedance.bitsail.common.model.ColumnInfo; import com.bytedance.bitsail.common.option.CommonOptions; import com.bytedance.bitsail.common.option.WriterOptions; +import com.bytedance.bitsail.common.type.TypeInfoConverter; +import com.bytedance.bitsail.common.type.filemapping.HiveTypeInfoConverter; import com.bytedance.bitsail.common.util.JsonSerializer; import com.bytedance.bitsail.common.util.Pair; import com.bytedance.bitsail.connector.hadoop.sink.FileOutputFormatPlugin; import com.bytedance.bitsail.connector.hadoop.util.HdfsUtils; import com.bytedance.bitsail.connector.legacy.hive.common.HiveParqueFormatErrorCode; -import com.bytedance.bitsail.connector.legacy.hive.common.HiveSinkEngineConnector; import com.bytedance.bitsail.connector.legacy.hive.option.HiveWriterOptions; import com.bytedance.bitsail.conversion.hive.ConvertToHiveObjectOptions; import com.bytedance.bitsail.conversion.hive.extractor.GeneralWritableExtractor; @@ -369,6 +368,11 @@ public void tryCleanupOnError() { } } + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new HiveTypeInfoConverter(); + } + @SuppressWarnings("checkstyle:MagicNumber") @Override public void initPlugin() throws TException, IOException { @@ -408,11 +412,6 @@ public void initPlugin() throws TException, IOException { hiveSerdeParameter); } - @Override - public SinkEngineConnector initSinkSchemaManager(BitSailConfiguration commonConf, BitSailConfiguration writerConf) throws Exception { - return new HiveSinkEngineConnector(commonConf, writerConf); - } - /** * When add partition in Default HiveOutputFormat, we used the partition location path as the location. * for example, the partition whole location is "hdfs://test-db/test_table/date=20220101", we only use diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/source/HiveInputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/source/HiveInputFormat.java index ed3836341..852345fc4 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/source/HiveInputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/source/HiveInputFormat.java @@ -20,17 +20,16 @@ import com.bytedance.bitsail.base.execution.ProcessResult; import com.bytedance.bitsail.common.BitSailException; import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.source.SourceEngineConnector; import com.bytedance.bitsail.common.exception.CommonErrorCode; import com.bytedance.bitsail.common.exception.FrameworkErrorCode; import com.bytedance.bitsail.common.model.ColumnInfo; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.common.type.filemapping.HiveTypeInfoConverter; import com.bytedance.bitsail.common.util.JsonSerializer; import com.bytedance.bitsail.common.util.Pair; import com.bytedance.bitsail.component.format.api.RowBuilder; import com.bytedance.bitsail.component.format.hive.HiveGeneralRowBuilder; import com.bytedance.bitsail.connector.hadoop.source.HadoopInputFormatBasePlugin; -import com.bytedance.bitsail.connector.legacy.hive.common.HiveSourceEngineConnector; import com.bytedance.bitsail.connector.legacy.hive.option.HiveReaderOptions; import com.bytedance.bitsail.flink.core.typeutils.ColumnFlinkTypeInfoUtil; @@ -127,6 +126,11 @@ public void initPlugin() throws Exception { } } + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new HiveTypeInfoConverter(); + } + @Override public TypeInformation getProducedType() { return rowTypeInfo; @@ -142,11 +146,6 @@ public String getType() { return "Hive"; } - @Override - public SourceEngineConnector initSourceSchemaManager(BitSailConfiguration commonConf, BitSailConfiguration readerConf) { - return new HiveSourceEngineConnector(commonConf, readerConf, getHiveConf(readerConf)); - } - private InputFormat getMapredInputFormat(BitSailConfiguration commonConfig, BitSailConfiguration inputConf) throws Exception { StorageDescriptor storageDescriptor = HiveMetaClientUtil.getTableFormat( getHiveConf(inputConf), db, table); diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/util/HiveConfUtils.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/util/HiveConfUtils.java new file mode 100644 index 000000000..1b83c5ce2 --- /dev/null +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/java/com/bytedance/bitsail/connector/legacy/hive/util/HiveConfUtils.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.connector.legacy.hive.util; + +import com.bytedance.bitsail.common.util.JsonSerializer; + +import com.bytedance.bitsail.shaded.hive.client.HiveMetaClientUtil; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; + +import java.util.Map; + +public class HiveConfUtils { + + public static HiveConf fromJsonProperties(String jsonProperties) { + Map hiveProperties = + JsonSerializer.parseToMap(jsonProperties); + return HiveMetaClientUtil.getHiveConf(hiveProperties); + } + + public HiveConf fromHiveConfPath(String location) { + HiveConf hiveConf = new HiveConf(); + hiveConf.addResource(new Path(location)); + return hiveConf; + } +} diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/resources/META-INF/services/com.bytedance.bitsail.base.catalog.TableCatalogFactory b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/resources/META-INF/services/com.bytedance.bitsail.base.catalog.TableCatalogFactory new file mode 100644 index 000000000..d47bc8cd3 --- /dev/null +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/main/resources/META-INF/services/com.bytedance.bitsail.base.catalog.TableCatalogFactory @@ -0,0 +1,20 @@ +# +# 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. +# + +com.bytedance.bitsail.connector.legacy.hive.common.HiveTableCatalogFactory \ No newline at end of file diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/test/java/com/bytedance/bitsail/connector/legacy/hive/sink/HiveParquetOutputFailoverTest.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/test/java/com/bytedance/bitsail/connector/legacy/hive/sink/HiveParquetOutputFailoverTest.java index f5e28f337..20dea084f 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/test/java/com/bytedance/bitsail/connector/legacy/hive/sink/HiveParquetOutputFailoverTest.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-hive/src/test/java/com/bytedance/bitsail/connector/legacy/hive/sink/HiveParquetOutputFailoverTest.java @@ -142,8 +142,7 @@ public BitSailConfiguration getCommonConf() { return BitSailConfiguration.newDefault() .set(CommonOptions.INTERNAL_INSTANCE_ID, getInstanceId()) .set(CommonOptions.USER_NAME, "root") - .set(CommonOptions.JOB_ID, getJobId()) - .set(CommonOptions.SYNC_DDL, false); + .set(CommonOptions.JOB_ID, getJobId()); } protected String getInstanceId() { diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/catalog/MySQLTableCatalog.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/catalog/MySQLTableCatalog.java new file mode 100644 index 000000000..b1ab3bbeb --- /dev/null +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/catalog/MySQLTableCatalog.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.connector.legacy.jdbc.catalog; + +import com.bytedance.bitsail.common.BitSailException; +import com.bytedance.bitsail.common.catalog.table.CatalogTable; +import com.bytedance.bitsail.common.catalog.table.CatalogTableColumn; +import com.bytedance.bitsail.common.catalog.table.CatalogTableDefinition; +import com.bytedance.bitsail.common.catalog.table.CatalogTableSchema; +import com.bytedance.bitsail.common.catalog.table.TableCatalog; +import com.bytedance.bitsail.common.catalog.table.TableOperation; +import com.bytedance.bitsail.common.type.TypeInfoConverter; +import com.bytedance.bitsail.common.typeinfo.TypeInfo; +import com.bytedance.bitsail.connector.legacy.jdbc.exception.JDBCPluginErrorCode; +import com.bytedance.bitsail.connector.legacy.jdbc.model.TableInfo; +import com.bytedance.bitsail.connector.legacy.jdbc.utils.MysqlUtil; + +import lombok.Builder; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +public class MySQLTableCatalog implements TableCatalog { + private static final Logger LOG = LoggerFactory.getLogger(MySQLTableCatalog.class); + + private final String database; + private final String table; + private final String schema; + private final String username; + private final String password; + private final String url; + private final String customizedSQL; + private final boolean useCustomizedSQL; + private TypeInfoConverter typeInfoConverter; + + @Builder + public MySQLTableCatalog(String database, + String table, + String schema, + String username, + String password, + String url, + String customizedSQL) { + this.database = database; + this.table = table; + this.schema = schema; + this.username = username; + this.password = password; + this.url = url; + this.customizedSQL = customizedSQL; + this.useCustomizedSQL = StringUtils.isNotEmpty(customizedSQL); + } + + @Override + public void open(TypeInfoConverter typeInfoConverter) { + this.typeInfoConverter = typeInfoConverter; + } + + @Override + public void close() { + //ignore + } + + @Override + public CatalogTableDefinition createCatalogTableDefinition() { + return CatalogTableDefinition + .builder() + .database(database) + .table(table) + .build(); + } + + @Override + public boolean tableExists(CatalogTableDefinition catalogTableDefinition) { + //todo doesn't check + return true; + } + + @Override + public CatalogTable getCatalogTable(CatalogTableDefinition catalogTableDefinition) { + TableInfo tableInfo; + try { + if (useCustomizedSQL) { + tableInfo = MysqlUtil.getInstance() + .getCustomizedSQLTableInfo(url, username, password, database, customizedSQL); + } else { + tableInfo = MysqlUtil.getInstance() + .getTableInfo(url, username, password, database, schema, table, null, null); + } + } catch (Exception e) { + LOG.error("Failed to get table info by the definition {}.", catalogTableDefinition); + throw BitSailException.asBitSailException(JDBCPluginErrorCode.INTERNAL_ERROR, e); + } + return CatalogTable + .builder() + .catalogTableDefinition(catalogTableDefinition) + .catalogTableSchema(CatalogTableSchema.builder() + .columns(convertTableColumn( + typeInfoConverter, + tableInfo.getColumnInfoList()) + ).build() + ).build(); + } + + @Override + public void createTable(CatalogTableDefinition catalogTableDefinition, CatalogTable catalogTable) { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTable(TableOperation tableOperation, + CatalogTable table) { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTableColumns(TableOperation tableOperation, + List catalogTableColumns) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean compareTypeCompatible(TypeInfo original, + TypeInfo compared) { + return original.getTypeClass() == compared.getTypeClass(); + } +} diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/catalog/MySQLTableCatalogFactory.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/catalog/MySQLTableCatalogFactory.java new file mode 100644 index 000000000..aad6baac7 --- /dev/null +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/catalog/MySQLTableCatalogFactory.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.connector.legacy.jdbc.catalog; + +import com.bytedance.bitsail.base.catalog.TableCatalogFactory; +import com.bytedance.bitsail.base.connector.BuilderGroup; +import com.bytedance.bitsail.base.execution.ExecutionEnviron; +import com.bytedance.bitsail.common.BitSailException; +import com.bytedance.bitsail.common.catalog.table.TableCatalog; +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.common.exception.CommonErrorCode; +import com.bytedance.bitsail.common.exception.FrameworkErrorCode; +import com.bytedance.bitsail.connector.legacy.jdbc.exception.JDBCPluginErrorCode; +import com.bytedance.bitsail.connector.legacy.jdbc.model.ClusterInfo; +import com.bytedance.bitsail.connector.legacy.jdbc.model.ConnectionInfo; +import com.bytedance.bitsail.connector.legacy.jdbc.options.JdbcReaderOptions; +import com.bytedance.bitsail.connector.legacy.jdbc.options.JdbcWriterOptions; + +import org.apache.commons.collections.CollectionUtils; + +import java.util.List; + +public class MySQLTableCatalogFactory implements TableCatalogFactory { + + @Override + public TableCatalog createTableCatalog(BuilderGroup builderGroup, + ExecutionEnviron executionEnviron, + BitSailConfiguration connectorConfiguration) { + if (BuilderGroup.READER.equals(builderGroup)) { + + List connections = connectorConfiguration + .getNecessaryOption(JdbcReaderOptions.CONNECTIONS, + FrameworkErrorCode.REQUIRED_VALUE); + + return MySQLTableCatalog + .builder() + .database(connectorConfiguration.get(JdbcReaderOptions.DB_NAME)) + .table(connectorConfiguration.get(JdbcReaderOptions.TABLE_NAME)) + .schema(connectorConfiguration.get(JdbcReaderOptions.TABLE_SCHEMA)) + .username(connectorConfiguration.getNecessaryOption(JdbcReaderOptions.USER_NAME, + CommonErrorCode.CONFIG_ERROR)) + .password(connectorConfiguration.getNecessaryOption(JdbcReaderOptions.PASSWORD, + CommonErrorCode.CONFIG_ERROR)) + .customizedSQL(connectorConfiguration.get(JdbcReaderOptions.CUSTOMIZED_SQL)) + .url(getClusterUrl(connections)) + .build(); + } + return MySQLTableCatalog + .builder() + .username(connectorConfiguration.get(JdbcWriterOptions.USER_NAME)) + .password(connectorConfiguration.get(JdbcWriterOptions.PASSWORD)) + .schema(connectorConfiguration.get(JdbcWriterOptions.USER_NAME)) + .table(connectorConfiguration.get(JdbcWriterOptions.TABLE_NAME)) + .database(connectorConfiguration.get(JdbcWriterOptions.DB_NAME)) + .url(connectorConfiguration.get(JdbcWriterOptions.CONNECTIONS) + .get(0).getUrl()) + .build(); + } + + private static String getClusterUrl(List connections) { + if (CollectionUtils.isEmpty(connections)) { + throw BitSailException.asBitSailException( + JDBCPluginErrorCode.REQUIRED_VALUE, + "Connection can't be empty."); + } + ClusterInfo clusterInfo = connections.get(0); + ConnectionInfo connectionInfo = clusterInfo.getSlaves().get(0); + return connectionInfo.getUrl(); + } + + @Override + public String getComponentName() { + return "JDBC"; + } +} diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/options/JdbcWriterOptions.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/options/JdbcWriterOptions.java index b2afaab65..a7efb81ee 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/options/JdbcWriterOptions.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/options/JdbcWriterOptions.java @@ -20,6 +20,7 @@ import com.bytedance.bitsail.common.annotation.Essential; import com.bytedance.bitsail.common.option.ConfigOption; import com.bytedance.bitsail.common.option.WriterOptions; +import com.bytedance.bitsail.connector.legacy.jdbc.model.ConnectionInfo; import com.alibaba.fastjson.TypeReference; @@ -60,10 +61,9 @@ public interface JdbcWriterOptions extends WriterOptions.BaseWriterOptions { }); @Essential - ConfigOption>> CONNECTIONS = + ConfigOption> CONNECTIONS = key(WRITER_PREFIX + "connections") - .onlyReference(new TypeReference>>() { - }); + .onlyReference(new TypeReference>(){}); ConfigOption CONNECTION_PARAMETERS = key(WRITER_PREFIX + "connection_parameters") diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/sink/JDBCOutputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/sink/JDBCOutputFormat.java index 86e6502b0..262462494 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/sink/JDBCOutputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/sink/JDBCOutputFormat.java @@ -27,10 +27,10 @@ import com.bytedance.bitsail.common.type.filemapping.JdbcTypeInfoConverter; import com.bytedance.bitsail.common.util.Pair; import com.bytedance.bitsail.common.util.TypeConvertUtil.StorageEngine; -import com.bytedance.bitsail.connector.legacy.jdbc.constants.Key; import com.bytedance.bitsail.connector.legacy.jdbc.constants.WriteModeProxy; import com.bytedance.bitsail.connector.legacy.jdbc.exception.JDBCPluginErrorCode; import com.bytedance.bitsail.connector.legacy.jdbc.extension.DatabaseInterface; +import com.bytedance.bitsail.connector.legacy.jdbc.model.ConnectionInfo; import com.bytedance.bitsail.connector.legacy.jdbc.model.SqlType; import com.bytedance.bitsail.connector.legacy.jdbc.options.JdbcWriterOptions; import com.bytedance.bitsail.connector.legacy.jdbc.utils.JDBCConnHolder; @@ -42,7 +42,6 @@ import com.bytedance.bitsail.flink.core.legacy.connector.OutputFormatPlugin; import com.bytedance.bitsail.flink.core.typeutils.NativeFlinkTypeInfoUtil; -import com.alibaba.fastjson.JSONObject; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NoArgsConstructor; @@ -144,10 +143,9 @@ public void initPlugin() throws IOException { deleteThreshold = outputSliceConfig.get(JdbcWriterOptions.DELETE_THRESHOLD); deleteIntervalMs = outputSliceConfig.get(JdbcWriterOptions.DELETE_INTERVAL_MS); - List> connectionList = outputSliceConfig.getNecessaryOption(JdbcWriterOptions.CONNECTIONS, + List connections = outputSliceConfig.getNecessaryOption(JdbcWriterOptions.CONNECTIONS, JDBCPluginErrorCode.REQUIRED_VALUE); - List connections = connectionList.stream().map(JSONObject::new).collect(Collectors.toList()); - dbURL = (connections.get(0)).getString(Key.DB_URL); + dbURL = connections.get(0).getUrl(); String connectionParameters = outputSliceConfig.getUnNecessaryOption(JdbcWriterOptions.CONNECTION_PARAMETERS, null); dbURL = getDbURL(dbURL, connectionParameters); @@ -183,7 +181,7 @@ public void initPlugin() throws IOException { writeModeProxy = buildWriteModeProxy(writeMode); writeModeProxy.prepareOnClient(); - rowTypeInfo = NativeFlinkTypeInfoUtil.getRowTypeInformation(columns, getTypeConverter()); + rowTypeInfo = NativeFlinkTypeInfoUtil.getRowTypeInformation(columns, createTypeInfoConverter()); log.info("Output Row Type Info: " + rowTypeInfo); // generated values @@ -193,10 +191,6 @@ public void initPlugin() throws IOException { log.info("Validate plugin configuration parameters finished."); } - public TypeInfoConverter getTypeConverter() { - return new JdbcTypeInfoConverter(getStorageEngine().name()); - } - public String getDbURL(String dbURL, String connectionParameters) { if (StringUtils.isEmpty(connectionParameters)) { return dbURL; @@ -631,6 +625,11 @@ void preQuery() { } } + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new JdbcTypeInfoConverter(getStorageEngine().name()); + } + @Override public RowTypeInfo getProducedType() { return rowTypeInfo; diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/source/JDBCInputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/source/JDBCInputFormat.java index 16acbe2af..1cc3aa988 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/source/JDBCInputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/source/JDBCInputFormat.java @@ -19,9 +19,9 @@ import com.bytedance.bitsail.common.BitSailException; import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.source.SourceEngineConnector; import com.bytedance.bitsail.common.exception.CommonErrorCode; import com.bytedance.bitsail.common.model.ColumnInfo; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.common.type.filemapping.FileMappingTypeInfoConverter; import com.bytedance.bitsail.common.util.TypeConvertUtil.StorageEngine; import com.bytedance.bitsail.connector.legacy.jdbc.converter.JdbcValueConverter; @@ -143,6 +143,11 @@ private static void fillClusterInfo(List clusterInfos, String table } } + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new FileMappingTypeInfoConverter(getStorageEngine().name()); + } + @Override public RowTypeInfo getProducedType() { return rowTypeInfo; @@ -526,7 +531,7 @@ private void validateParameters() throws ExecutionException, InterruptedExceptio this.queryTemplateFormat = sqlTemplateFormat; this.splitParameterInfo = paramProvider.getParameterValues(); this.fetchSize = fetchSize; - rowTypeInfo = NativeFlinkTypeInfoUtil.getRowTypeInformation(columns, new FileMappingTypeInfoConverter(getStorageEngine().name())); + rowTypeInfo = NativeFlinkTypeInfoUtil.getRowTypeInformation(columns, createTypeInfoConverter()); LOG.info("Row Type Info: " + rowTypeInfo); LOG.info("Validate plugin configuration parameters finished."); @@ -613,11 +618,6 @@ String genSqlTemplate(String splitPK, List columns, String filter) { return sql.toString(); } - @Override - public SourceEngineConnector initSourceSchemaManager(BitSailConfiguration commonConf, BitSailConfiguration readerConf) throws Exception { - return new JDBCSourceEngineConnectorBase(commonConf, readerConf); - } - /** * In mariadb mysql java client, it enables streaming result set by `setFetchSize({any positive integer})`.
* In official mysql jdbc client, it enables streaming result set by `setFetchSize(INT.MIN_VALUE)`, and fetch one row once.
diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/source/JDBCSourceEngineConnectorBase.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/source/JDBCSourceEngineConnectorBase.java deleted file mode 100644 index b9315ac10..000000000 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/source/JDBCSourceEngineConnectorBase.java +++ /dev/null @@ -1,115 +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 com.bytedance.bitsail.connector.legacy.jdbc.source; - -import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.source.SourceEngineConnector; -import com.bytedance.bitsail.common.exception.CommonErrorCode; -import com.bytedance.bitsail.common.exception.FrameworkErrorCode; -import com.bytedance.bitsail.common.model.ColumnInfo; -import com.bytedance.bitsail.common.type.TypeInfoConverter; -import com.bytedance.bitsail.common.type.filemapping.JdbcTypeInfoConverter; -import com.bytedance.bitsail.common.util.TypeConvertUtil.StorageEngine; -import com.bytedance.bitsail.connector.legacy.jdbc.model.ClusterInfo; -import com.bytedance.bitsail.connector.legacy.jdbc.model.TableInfo; -import com.bytedance.bitsail.connector.legacy.jdbc.options.JdbcReaderOptions; -import com.bytedance.bitsail.connector.legacy.jdbc.utils.AbstractJdbcUtil; -import com.bytedance.bitsail.connector.legacy.jdbc.utils.MysqlUtil; - -import org.apache.commons.lang3.StringUtils; - -import java.util.List; - -public class JDBCSourceEngineConnectorBase extends SourceEngineConnector { - - protected String database; - protected String table; - protected String schema; - protected String username; - protected String password; - protected String url; - protected String customizedSQL; - protected boolean useCustomizedSQL; - - public JDBCSourceEngineConnectorBase(BitSailConfiguration commonConfiguration, BitSailConfiguration readerConfiguration) { - super(commonConfiguration, readerConfiguration); - username = readerConfiguration.getNecessaryOption(JdbcReaderOptions.USER_NAME, - CommonErrorCode.CONFIG_ERROR); - password = readerConfiguration.getNecessaryOption(JdbcReaderOptions.PASSWORD, - FrameworkErrorCode.REQUIRED_VALUE); - database = readerConfiguration.get(JdbcReaderOptions.DB_NAME); - schema = readerConfiguration.get(JdbcReaderOptions.TABLE_SCHEMA); - customizedSQL = readerConfiguration.get(JdbcReaderOptions.CUSTOMIZED_SQL); - useCustomizedSQL = StringUtils.isNotEmpty(customizedSQL); - - table = StringUtils.isNotBlank(customizedSQL) ? null - : readerConfiguration.getUnNecessaryOption(JdbcReaderOptions.TABLE_NAME, ""); - - List connections = readerConfiguration - .getNecessaryOption(JdbcReaderOptions.CONNECTIONS, - FrameworkErrorCode.REQUIRED_VALUE); - - ClusterInfo clusterInfo = connections.get(0); - table = getSingleTable(table, customizedSQL, clusterInfo); - url = clusterInfo.getSlaves() - .get(0).getUrl(); - } - - private static String getSingleTable(String orginTable, - String customizedSQL, - ClusterInfo clusterInfo) { - if (StringUtils.isNotEmpty(customizedSQL)) { - return null; - } - if (StringUtils.isNotEmpty(clusterInfo.getTableNames())) { - return StringUtils.split(clusterInfo.getTableNames(), ",")[0]; - } - if (StringUtils.isEmpty(orginTable)) { - throw new IllegalArgumentException("Can not found any table info in connections"); - } - return StringUtils.split(orginTable, ",")[0]; - } - - @Override - public String getExternalEngineName() { - return StorageEngine.mysql.name(); - } - - public AbstractJdbcUtil getJdbcUtil() { - return MysqlUtil.getInstance(); - } - - @Override - public TypeInfoConverter createTypeInfoConverter() { - return new JdbcTypeInfoConverter(getExternalEngineName()); - } - - @Override - public List getExternalColumnInfos() throws Exception { - // Generate from conf params - AbstractJdbcUtil jdbcUtil = getJdbcUtil(); - TableInfo tableInfo = null; - if (useCustomizedSQL) { - tableInfo = jdbcUtil.getCustomizedSQLTableInfo(url, username, password, database, customizedSQL); - } else { - tableInfo = jdbcUtil.getTableInfo(url, username, password, database, schema, table, null, null); - } - - return tableInfo.getColumnInfoList(); - } -} diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/source/OracleInputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/source/OracleInputFormat.java index 53f2cae87..7fecfaec6 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/source/OracleInputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/source/OracleInputFormat.java @@ -17,8 +17,6 @@ package com.bytedance.bitsail.connector.legacy.jdbc.source; -import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.source.SourceEngineConnector; import com.bytedance.bitsail.common.model.ColumnInfo; import com.bytedance.bitsail.common.util.TypeConvertUtil.StorageEngine; import com.bytedance.bitsail.connector.legacy.jdbc.converter.JdbcValueConverter; @@ -98,11 +96,6 @@ public StorageEngine getStorageEngine() { return StorageEngine.oracle; } - @Override - public SourceEngineConnector initSourceSchemaManager(BitSailConfiguration commonConf, BitSailConfiguration readerConf) throws Exception { - return new OracleSourceEngineConnector(commonConf, readerConf); - } - @Override protected JdbcValueConverter createJdbcValueConverter() { return new OracleValueConverter(OracleValueConverter.IntervalHandlingMode.parse(this.intervalHandlingMode)); diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/source/OracleSourceEngineConnector.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/source/OracleSourceEngineConnector.java deleted file mode 100644 index 368fc2089..000000000 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/java/com/bytedance/bitsail/connector/legacy/jdbc/source/OracleSourceEngineConnector.java +++ /dev/null @@ -1,87 +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 com.bytedance.bitsail.connector.legacy.jdbc.source; - -import com.bytedance.bitsail.common.BitSailException; -import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.source.SourceEngineConnector; -import com.bytedance.bitsail.common.exception.CommonErrorCode; -import com.bytedance.bitsail.common.model.ColumnInfo; -import com.bytedance.bitsail.common.type.TypeInfoConverter; -import com.bytedance.bitsail.common.type.filemapping.JdbcTypeInfoConverter; -import com.bytedance.bitsail.common.util.JsonSerializer; -import com.bytedance.bitsail.connector.legacy.jdbc.model.ClusterInfo; -import com.bytedance.bitsail.connector.legacy.jdbc.model.ConnectionInfo; -import com.bytedance.bitsail.connector.legacy.jdbc.model.TableInfo; -import com.bytedance.bitsail.connector.legacy.jdbc.options.JdbcReaderOptions; -import com.bytedance.bitsail.connector.legacy.jdbc.options.OracleReaderOptions; -import com.bytedance.bitsail.connector.legacy.jdbc.utils.OracleUtil; - -import com.google.common.base.Strings; -import org.apache.commons.collections.CollectionUtils; - -import java.util.List; - -public class OracleSourceEngineConnector extends SourceEngineConnector { - private final String url; - private final String schemaName; - private final String tableName; - private final String userName; - private final String password; - private final String initSql; - - public OracleSourceEngineConnector(BitSailConfiguration commonConfiguration, - BitSailConfiguration readerConfiguration) { - super(commonConfiguration, readerConfiguration); - List clusterInfos = readerConfiguration.get(JdbcReaderOptions.CONNECTIONS); - List slaves = clusterInfos.get(0).getSlaves(); - if (CollectionUtils.isEmpty(slaves)) { - throw BitSailException.asBitSailException(CommonErrorCode.CONFIG_ERROR, - "Get DB information error, the slaves info is null! Connection Json string is " + JsonSerializer.serialize(clusterInfos)); - } - url = slaves.get(0).getUrl(); - schemaName = readerConfiguration.get(OracleReaderOptions.TABLE_SCHEMA); - tableName = readerConfiguration.get(OracleReaderOptions.TABLE_NAME); - userName = readerConfiguration.get(OracleReaderOptions.USER_NAME); - password = readerConfiguration.get(OracleReaderOptions.PASSWORD); - initSql = readerConfiguration.get(OracleReaderOptions.INIT_SQL); - } - - @Override - public List getExternalColumnInfos() throws Exception { - OracleUtil oracleUtil = new OracleUtil(); - TableInfo tableInfo; - if (!Strings.isNullOrEmpty(tableName) && tableName.toUpperCase().split("\\.").length == 2) { - // TODO The following line supports backward compatibility of signature. Format is 'Schema.table'. Can remove this line after Processor upgrade is complete. - tableInfo = oracleUtil.getTableInfo(url, userName, password, tableName, initSql); - } else { - tableInfo = oracleUtil.getTableInfo(url, userName, password, null, schemaName, tableName, initSql, null); - } - return tableInfo.getColumnInfoList(); - } - - @Override - public TypeInfoConverter createTypeInfoConverter() { - return new JdbcTypeInfoConverter(getExternalEngineName()); - } - - @Override - public String getExternalEngineName() { - return "oracle"; - } -} diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/resources/META-INF/services/com.bytedance.bitsail.base.catalog.TableCatalogFactory b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/resources/META-INF/services/com.bytedance.bitsail.base.catalog.TableCatalogFactory new file mode 100644 index 000000000..539fde940 --- /dev/null +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/main/resources/META-INF/services/com.bytedance.bitsail.base.catalog.TableCatalogFactory @@ -0,0 +1,20 @@ +# +# 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. +# + +com.bytedance.bitsail.connector.legacy.jdbc.catalog.MySQLTableCatalogFactory \ No newline at end of file diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/test/java/com/bytedance/bitsail/connector/legacy/jdbc/catalog/MySQLTableCatalogITCase.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/test/java/com/bytedance/bitsail/connector/legacy/jdbc/catalog/MySQLTableCatalogITCase.java new file mode 100644 index 000000000..bbd822bef --- /dev/null +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/test/java/com/bytedance/bitsail/connector/legacy/jdbc/catalog/MySQLTableCatalogITCase.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.bytedance.bitsail.connector.legacy.jdbc.catalog; + +import com.bytedance.bitsail.common.catalog.table.CatalogTable; +import com.bytedance.bitsail.common.catalog.table.CatalogTableDefinition; +import com.bytedance.bitsail.common.type.filemapping.JdbcTypeInfoConverter; +import com.bytedance.bitsail.connector.legacy.jdbc.container.MySQLContainerMariadbAdapter; + +import org.apache.commons.collections.CollectionUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.MySQLContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerImageName; + +import java.util.stream.Stream; + +public class MySQLTableCatalogITCase { + private static final Logger LOG = LoggerFactory.getLogger(MySQLTableCatalogITCase.class); + + private static final String MYSQL_DOCKER_IMAGER = "mysql:8.0.29"; + + private MySQLContainer container; + + private static final String TABLE = "jdbc_dynamic_table"; + private static final String DATABASE = "test"; + + @Before + public void before() { + container = new MySQLContainerMariadbAdapter<>(DockerImageName.parse(MYSQL_DOCKER_IMAGER)) + .withUrlParam("permitMysqlScheme", null) + .withInitScript("scripts/fake_to_jdbc_sink.sql") + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + Startables.deepStart(Stream.of(container)).join(); + } + + @After + public void after() { + container.close(); + } + + @Test + public void testGetCatalogTable() { + MySQLTableCatalog catalog = MySQLTableCatalog + .builder() + .url(container.getJdbcUrl()) + .table(TABLE) + .database(DATABASE) + .username(container.getUsername()) + .password(container.getPassword()) + .build(); + + catalog.open(new JdbcTypeInfoConverter("mysql")); + + CatalogTableDefinition catalogTableDefinition = catalog.createCatalogTableDefinition(); + CatalogTable catalogTable = catalog.getCatalogTable(catalogTableDefinition); + + Assert.assertNotNull(catalogTable.getCatalogTableSchema()); + Assert.assertEquals(CollectionUtils.size(catalogTable.getCatalogTableSchema().getColumns()), 21); + } +} \ No newline at end of file diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/test/java/com/bytedance/bitsail/connector/legacy/jdbc/sink/JdbcSinkITCase.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/test/java/com/bytedance/bitsail/connector/legacy/jdbc/sink/JdbcSinkITCase.java index 922d76da6..8adf22342 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/test/java/com/bytedance/bitsail/connector/legacy/jdbc/sink/JdbcSinkITCase.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/test/java/com/bytedance/bitsail/connector/legacy/jdbc/sink/JdbcSinkITCase.java @@ -19,12 +19,12 @@ import com.bytedance.bitsail.common.configuration.BitSailConfiguration; import com.bytedance.bitsail.connector.legacy.jdbc.container.MySQLContainerMariadbAdapter; +import com.bytedance.bitsail.connector.legacy.jdbc.model.ConnectionInfo; import com.bytedance.bitsail.connector.legacy.jdbc.options.JdbcWriterOptions; import com.bytedance.bitsail.test.connector.test.EmbeddedFlinkCluster; import com.bytedance.bitsail.test.connector.test.utils.JobConfUtils; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -35,7 +35,6 @@ import org.testcontainers.lifecycle.Startables; import org.testcontainers.utility.DockerImageName; -import java.util.Map; import java.util.stream.Stream; /** @@ -67,11 +66,11 @@ public void after() { public void testInsertModeMySQL() throws Exception { BitSailConfiguration globalConfiguration = JobConfUtils.fromClasspath("scripts/fake_to_jdbc_sink.json"); - Map connection = Maps.newHashMap(); - connection.put("db_url", container.getJdbcUrl()); - connection.put("host", container.getHost()); - connection.put("port", container.getFirstMappedPort()); - globalConfiguration.set(JdbcWriterOptions.CONNECTIONS, Lists.newArrayList(connection)); + ConnectionInfo connectionInfo = new ConnectionInfo(); + connectionInfo.setHost(container.getHost()); + connectionInfo.setPort(container.getFirstMappedPort()); + connectionInfo.setUrl(container.getJdbcUrl()); + globalConfiguration.set(JdbcWriterOptions.CONNECTIONS, Lists.newArrayList(connectionInfo)); EmbeddedFlinkCluster.submitJob(globalConfiguration); } } \ No newline at end of file diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/test/java/com/bytedance/bitsail/connector/legacy/jdbc/sink/OracleSinkITCase.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/test/java/com/bytedance/bitsail/connector/legacy/jdbc/sink/OracleSinkITCase.java index 3cc9e911a..71efdc90f 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/test/java/com/bytedance/bitsail/connector/legacy/jdbc/sink/OracleSinkITCase.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-jdbc/src/test/java/com/bytedance/bitsail/connector/legacy/jdbc/sink/OracleSinkITCase.java @@ -18,12 +18,12 @@ package com.bytedance.bitsail.connector.legacy.jdbc.sink; import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.connector.legacy.jdbc.model.ConnectionInfo; import com.bytedance.bitsail.connector.legacy.jdbc.options.JdbcWriterOptions; import com.bytedance.bitsail.test.connector.test.EmbeddedFlinkCluster; import com.bytedance.bitsail.test.connector.test.utils.JobConfUtils; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -33,7 +33,6 @@ import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.lifecycle.Startables; -import java.util.Map; import java.util.stream.Stream; /** @@ -71,11 +70,11 @@ public void after() { public void testInsertModeOracle() throws Exception { BitSailConfiguration globalConfiguration = JobConfUtils.fromClasspath("scripts/fake_to_oracle_sink.json"); - Map connection = Maps.newHashMap(); - connection.put("db_url", container.getJdbcUrl()); - connection.put("host", container.getHost()); - connection.put("port", container.getFirstMappedPort()); - globalConfiguration.set(JdbcWriterOptions.CONNECTIONS, Lists.newArrayList(connection)); + ConnectionInfo connectionInfo = new ConnectionInfo(); + connectionInfo.setHost(container.getHost()); + connectionInfo.setPort(container.getFirstMappedPort()); + connectionInfo.setUrl(container.getJdbcUrl()); + globalConfiguration.set(JdbcWriterOptions.CONNECTIONS, Lists.newArrayList(connectionInfo)); EmbeddedFlinkCluster.submitJob(globalConfiguration); } } \ No newline at end of file diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-kafka/src/main/java/com/bytedance/bitsail/connector/legacy/kafka/sink/KafkaOutputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-kafka/src/main/java/com/bytedance/bitsail/connector/legacy/kafka/sink/KafkaOutputFormat.java index 37ae21b9c..e5f22342f 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-kafka/src/main/java/com/bytedance/bitsail/connector/legacy/kafka/sink/KafkaOutputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-kafka/src/main/java/com/bytedance/bitsail/connector/legacy/kafka/sink/KafkaOutputFormat.java @@ -19,6 +19,8 @@ import com.bytedance.bitsail.common.model.ColumnInfo; import com.bytedance.bitsail.common.option.CommonOptions; +import com.bytedance.bitsail.common.type.BitSailTypeInfoConverter; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.connector.legacy.kafka.common.KafkaFormatErrorCode; import com.bytedance.bitsail.connector.legacy.kafka.option.KafkaWriterOptions; import com.bytedance.bitsail.flink.core.constants.TypeSystem; @@ -94,7 +96,7 @@ public void initPlugin() { partitionFieldsIndices = getPartitionFieldsIndices(columns, partitionFieldsNames); } - this.rowTypeInfo = NativeFlinkTypeInfoUtil.getRowTypeInformation(columns); + this.rowTypeInfo = NativeFlinkTypeInfoUtil.getRowTypeInformation(columns, createTypeInfoConverter()); log.info("Output Row Type Info: " + rowTypeInfo); } @@ -256,4 +258,9 @@ private void closeProducer() { kafkaProducer.close(); } } + + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new BitSailTypeInfoConverter(); + } } diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-larksheet/src/main/java/com/bytedance/bitsail/connector/legacy/larksheet/source/LarkSheetInputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-larksheet/src/main/java/com/bytedance/bitsail/connector/legacy/larksheet/source/LarkSheetInputFormat.java index 1ff7bced3..04375beba 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-larksheet/src/main/java/com/bytedance/bitsail/connector/legacy/larksheet/source/LarkSheetInputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-larksheet/src/main/java/com/bytedance/bitsail/connector/legacy/larksheet/source/LarkSheetInputFormat.java @@ -21,6 +21,8 @@ import com.bytedance.bitsail.common.column.Column; import com.bytedance.bitsail.common.column.StringColumn; import com.bytedance.bitsail.common.model.ColumnInfo; +import com.bytedance.bitsail.common.type.BitSailTypeInfoConverter; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.connector.legacy.larksheet.api.SheetConfig; import com.bytedance.bitsail.connector.legacy.larksheet.api.TokenHolder; import com.bytedance.bitsail.connector.legacy.larksheet.error.LarkSheetFormatErrorCode; @@ -30,7 +32,7 @@ import com.bytedance.bitsail.connector.legacy.larksheet.option.LarkSheetReaderOptions; import com.bytedance.bitsail.connector.legacy.larksheet.util.LarkSheetUtil; import com.bytedance.bitsail.flink.core.legacy.connector.InputFormatPlugin; -import com.bytedance.bitsail.flink.core.typeinfo.PrimitiveColumnTypeInfo; +import com.bytedance.bitsail.flink.core.typeutils.ColumnFlinkTypeInfoUtil; import com.google.common.collect.Maps; import org.apache.commons.collections.CollectionUtils; @@ -106,6 +108,11 @@ public class LarkSheetInputFormat extends InputFormatPlugin */ private List sheetInfoList; + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new BitSailTypeInfoConverter(); + } + /** * Some features when transforming data. */ @@ -303,18 +310,7 @@ public TypeInformation getProducedType() { * @param readerColumns Columns defined in job configuration. */ private RowTypeInfo buildRowTypeInfo(List readerColumns) { - int size = readerColumns.size(); - PrimitiveColumnTypeInfo[] typeInfos = new PrimitiveColumnTypeInfo[size]; - String[] names = new String[size]; - - for (int i = 0; i < size; i++) { - typeInfos[i] = PrimitiveColumnTypeInfo.STRING_COLUMN_TYPE_INFO; - names[i] = readerColumns.get(i).getName(); - } - - RowTypeInfo rowTypeInfo = new RowTypeInfo(typeInfos, names); - LOG.info("Row type info: {}", this.rowTypeInfo); - return rowTypeInfo; + return ColumnFlinkTypeInfoUtil.getRowTypeInformation(createTypeInfoConverter(), readerColumns); } /** diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-mongodb/src/main/java/com/bytedance/bitsail/connector/legacy/mongodb/sink/MongoDBOutputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-mongodb/src/main/java/com/bytedance/bitsail/connector/legacy/mongodb/sink/MongoDBOutputFormat.java index 4ce4cb590..241075f33 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-mongodb/src/main/java/com/bytedance/bitsail/connector/legacy/mongodb/sink/MongoDBOutputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-mongodb/src/main/java/com/bytedance/bitsail/connector/legacy/mongodb/sink/MongoDBOutputFormat.java @@ -20,6 +20,7 @@ import com.bytedance.bitsail.common.configuration.BitSailConfiguration; import com.bytedance.bitsail.common.exception.CommonErrorCode; import com.bytedance.bitsail.common.model.ColumnInfo; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.common.type.filemapping.MongoTypeInfoConverter; import com.bytedance.bitsail.connector.legacy.mongodb.common.MongoConnConfig; import com.bytedance.bitsail.connector.legacy.mongodb.common.MongoConnOptions; @@ -111,7 +112,7 @@ public void initPlugin() throws Exception { String uniqueKeysStr = outputSliceConfig.getNecessaryOption(MongoDBWriterOptions.UNIQUE_KEY, MongoDBPluginsErrorCode.REQUIRED_VALUE); this.uniqueKeyList = Arrays.asList(StringUtils.split(uniqueKeysStr, ",")); } - this.rowTypeInfo = NativeFlinkTypeInfoUtil.getRowTypeInformation(columnInfos, new MongoTypeInfoConverter()); + this.rowTypeInfo = NativeFlinkTypeInfoUtil.getRowTypeInformation(columnInfos, createTypeInfoConverter()); LOG.info("Output Row Type Info: " + rowTypeInfo); } @@ -345,4 +346,9 @@ private MongoConnOptions getOptions() { build.writeConcern(outputSliceConfig.get(MongoDBWriterOptions.WRITE_CONCERN)); return build.build(); } + + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new MongoTypeInfoConverter(); + } } diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-mongodb/src/main/java/com/bytedance/bitsail/connector/legacy/mongodb/source/MongoDBInputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-mongodb/src/main/java/com/bytedance/bitsail/connector/legacy/mongodb/source/MongoDBInputFormat.java index 5cf184e1d..6db5ddcbb 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-mongodb/src/main/java/com/bytedance/bitsail/connector/legacy/mongodb/source/MongoDBInputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-mongodb/src/main/java/com/bytedance/bitsail/connector/legacy/mongodb/source/MongoDBInputFormat.java @@ -20,6 +20,7 @@ import com.bytedance.bitsail.common.BitSailException; import com.bytedance.bitsail.common.exception.CommonErrorCode; import com.bytedance.bitsail.common.model.ColumnInfo; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.common.type.filemapping.MongoTypeInfoConverter; import com.bytedance.bitsail.connector.legacy.mongodb.common.MongoConnConfig; import com.bytedance.bitsail.connector.legacy.mongodb.error.MongoDBPluginsErrorCode; @@ -399,7 +400,7 @@ public void initPlugin() { this.taskGroupInfo = splitter.getTaskGroupInfo(); this.totalSplitNum = splitter.getTotalSplitNum(); - this.rowTypeInfo = NativeFlinkTypeInfoUtil.getRowTypeInformation(columnInfos, new MongoTypeInfoConverter()); + this.rowTypeInfo = NativeFlinkTypeInfoUtil.getRowTypeInformation(columnInfos, createTypeInfoConverter()); LOG.info("Row Type Info: " + rowTypeInfo); } @@ -518,4 +519,9 @@ public void close() { public String getType() { return "mongodb"; } + + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new MongoTypeInfoConverter(); + } } diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-print/src/main/java/com/bytedance/bitsail/connector/legacy/print/sink/PrintSink.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-print/src/main/java/com/bytedance/bitsail/connector/legacy/print/sink/PrintSink.java index 304ea0da4..3201373a1 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-print/src/main/java/com/bytedance/bitsail/connector/legacy/print/sink/PrintSink.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-print/src/main/java/com/bytedance/bitsail/connector/legacy/print/sink/PrintSink.java @@ -17,6 +17,8 @@ package com.bytedance.bitsail.connector.legacy.print.sink; +import com.bytedance.bitsail.common.type.BitSailTypeInfoConverter; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.flink.core.legacy.connector.OutputFormatPlugin; import org.apache.flink.api.common.functions.util.PrintSinkOutputWriter; @@ -59,4 +61,9 @@ public String getType() { public void tryCleanupOnError() throws Exception { } + + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new BitSailTypeInfoConverter(); + } } diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-redis/src/main/java/com/bytedance/bitsail/connector/legacy/redis/sink/RedisOutputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-redis/src/main/java/com/bytedance/bitsail/connector/legacy/redis/sink/RedisOutputFormat.java index fd86d322e..d91d3940b 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-redis/src/main/java/com/bytedance/bitsail/connector/legacy/redis/sink/RedisOutputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-redis/src/main/java/com/bytedance/bitsail/connector/legacy/redis/sink/RedisOutputFormat.java @@ -19,6 +19,7 @@ import com.bytedance.bitsail.common.BitSailException; import com.bytedance.bitsail.common.model.ColumnInfo; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.connector.legacy.redis.RedisPipelineProcessor; import com.bytedance.bitsail.connector.legacy.redis.constant.RedisConstants; import com.bytedance.bitsail.connector.legacy.redis.core.TtlType; @@ -144,5 +145,10 @@ public String getType() { public int getMaxParallelism() { return MAX_PARALLELISM_OUTPUT_REDIS; } + + @Override + public TypeInfoConverter createTypeInfoConverter() { + return null; + } } diff --git a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-rocketmq/src/main/java/com/bytedance/bitsail/connector/legacy/rocketmq/sink/RocketMQOutputFormat.java b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-rocketmq/src/main/java/com/bytedance/bitsail/connector/legacy/rocketmq/sink/RocketMQOutputFormat.java index c8cecc1a6..8aad1d754 100644 --- a/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-rocketmq/src/main/java/com/bytedance/bitsail/connector/legacy/rocketmq/sink/RocketMQOutputFormat.java +++ b/bitsail-connectors/bitsail-connectors-legacy/bitsail-connector-rocketmq/src/main/java/com/bytedance/bitsail/connector/legacy/rocketmq/sink/RocketMQOutputFormat.java @@ -18,6 +18,8 @@ package com.bytedance.bitsail.connector.legacy.rocketmq.sink; import com.bytedance.bitsail.common.model.ColumnInfo; +import com.bytedance.bitsail.common.type.BitSailTypeInfoConverter; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.common.util.Preconditions; import com.bytedance.bitsail.connector.legacy.rocketmq.config.RocketMQSinkConfig; import com.bytedance.bitsail.connector.legacy.rocketmq.constant.RocketMQConstants; @@ -202,5 +204,10 @@ private Message prepareMessage(Row row) { return msg; } + + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new BitSailTypeInfoConverter(); + } } diff --git a/bitsail-cores/bitsail-core-entry/src/test/java/com/bytedance/bitsail/core/reader/MockInputFormatPlugin.java b/bitsail-cores/bitsail-core-entry/src/test/java/com/bytedance/bitsail/core/reader/MockInputFormatPlugin.java index 68a141803..c1adcc2f8 100644 --- a/bitsail-cores/bitsail-core-entry/src/test/java/com/bytedance/bitsail/core/reader/MockInputFormatPlugin.java +++ b/bitsail-cores/bitsail-core-entry/src/test/java/com/bytedance/bitsail/core/reader/MockInputFormatPlugin.java @@ -20,6 +20,8 @@ package com.bytedance.bitsail.core.reader; import com.bytedance.bitsail.common.BitSailException; +import com.bytedance.bitsail.common.type.BitSailTypeInfoConverter; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.flink.core.legacy.connector.InputFormatPlugin; import org.apache.flink.api.common.io.statistics.BaseStatistics; @@ -79,4 +81,9 @@ public Object nextRecord(Object o) throws IOException { public void close() throws IOException { } + + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new BitSailTypeInfoConverter(); + } } diff --git a/bitsail-cores/bitsail-core-entry/src/test/java/com/bytedance/bitsail/core/writer/MockOutputFormatPlugin.java b/bitsail-cores/bitsail-core-entry/src/test/java/com/bytedance/bitsail/core/writer/MockOutputFormatPlugin.java index 3231ecee6..eb7a77960 100644 --- a/bitsail-cores/bitsail-core-entry/src/test/java/com/bytedance/bitsail/core/writer/MockOutputFormatPlugin.java +++ b/bitsail-cores/bitsail-core-entry/src/test/java/com/bytedance/bitsail/core/writer/MockOutputFormatPlugin.java @@ -19,6 +19,8 @@ package com.bytedance.bitsail.core.writer; +import com.bytedance.bitsail.common.type.BitSailTypeInfoConverter; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.flink.core.legacy.connector.OutputFormatPlugin; import org.apache.flink.types.Row; @@ -61,4 +63,9 @@ public int getMaxParallelism() { public void tryCleanupOnError() throws Exception { } + + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new BitSailTypeInfoConverter(); + } } diff --git a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/execution/FlinkExecutionEnviron.java b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/execution/FlinkExecutionEnviron.java index e9d67c92e..511e4a915 100644 --- a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/execution/FlinkExecutionEnviron.java +++ b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/execution/FlinkExecutionEnviron.java @@ -31,7 +31,7 @@ import com.bytedance.bitsail.common.option.CommonOptions; import com.bytedance.bitsail.flink.core.FlinkJobMode; import com.bytedance.bitsail.flink.core.execution.configurer.BitSailRuntimePluginConfigurer; -import com.bytedance.bitsail.flink.core.execution.configurer.FlinkDAGBuilderConfigurer; +import com.bytedance.bitsail.flink.core.execution.configurer.FlinkDAGBuilderInterceptor; import com.bytedance.bitsail.flink.core.execution.configurer.StreamExecutionEnvironmentConfigurer; import com.bytedance.bitsail.flink.core.execution.utils.ExecutionUtils; import com.bytedance.bitsail.flink.core.parallelism.FlinkParallelismAdvisor; @@ -131,9 +131,8 @@ public void configure(List readerBuilders, }); /* try to do schema alignment and configure each DAG builder */ - FlinkDAGBuilderConfigurer dagBuilderConfigurer = new FlinkDAGBuilderConfigurer(this); - dagBuilderConfigurer.doSchemaAlignment(readerBuilders, writerBuilders); - dagBuilderConfigurer.configureDAGBuilders(readerBuilders, writerBuilders); + FlinkDAGBuilderInterceptor interceptor = new FlinkDAGBuilderInterceptor(this); + interceptor.intercept(readerBuilders, null, writerBuilders); /* get parallelism advice for each dag builder */ parallelismAdvisor = new FlinkParallelismAdvisor(commonConfiguration, readerConfigurations, writerConfigurations); diff --git a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/execution/configurer/FlinkDAGBuilderConfigurer.java b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/execution/configurer/FlinkDAGBuilderConfigurer.java deleted file mode 100644 index 7f8fff11a..000000000 --- a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/execution/configurer/FlinkDAGBuilderConfigurer.java +++ /dev/null @@ -1,109 +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 com.bytedance.bitsail.flink.core.execution.configurer; - -import com.bytedance.bitsail.base.connector.reader.DataReaderDAGBuilder; -import com.bytedance.bitsail.base.connector.writer.DataWriterDAGBuilder; -import com.bytedance.bitsail.base.extension.SchemaAlignmentable; -import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.DdlSyncManager; -import com.bytedance.bitsail.common.ddl.sink.SinkEngineConnector; -import com.bytedance.bitsail.common.ddl.source.SourceEngineConnector; -import com.bytedance.bitsail.common.util.Preconditions; -import com.bytedance.bitsail.flink.core.execution.FlinkExecutionEnviron; - -import lombok.AllArgsConstructor; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.List; -import java.util.Objects; - -@AllArgsConstructor -public class FlinkDAGBuilderConfigurer { - private static final Logger LOG = LoggerFactory.getLogger(FlinkDAGBuilderConfigurer.class); - - private final FlinkExecutionEnviron executionEnviron; - - /** - * Try schema alignment if there is only one reader and one writer. - * reader and writer should implement interface {@link SchemaAlignmentable} - * - * @param readerBuilders - * @param writerBuilders - */ - public void doSchemaAlignment(List readerBuilders, - List writerBuilders) throws Exception { - if (readerBuilders.size() != 1 || writerBuilders.size() != 1) { - LOG.warn("Schema alignment is not supported for multi source or sink."); - return; - } - if (!(readerBuilders.get(0) instanceof SchemaAlignmentable)) { - LOG.warn("reader {} does not support schema alignment.", readerBuilders.get(0).getReaderName()); - return; - } - if (!(writerBuilders.get(0) instanceof SchemaAlignmentable)) { - LOG.warn("writer {} does not support schema alignment.", writerBuilders.get(0).getWriterName()); - return; - } - - SchemaAlignmentable source = (SchemaAlignmentable) readerBuilders.get(0); - SchemaAlignmentable sink = (SchemaAlignmentable) writerBuilders.get(0); - - BitSailConfiguration readerConfiguration = executionEnviron.getReaderConfigurations().get(0); - BitSailConfiguration writerConfiguration = executionEnviron.getWriterConfigurations().get(0); - - SourceEngineConnector sourceEngineConnector = (SourceEngineConnector) source.createExternalEngineConnector(executionEnviron, readerConfiguration); - SinkEngineConnector sinkEngineConnector = (SinkEngineConnector) sink.createExternalEngineConnector(executionEnviron, writerConfiguration); - - if (Objects.isNull(sourceEngineConnector) || Objects.isNull(sinkEngineConnector)) { - LOG.warn("Skip schema alignment, source engine connector or sink engine connector not supported."); - return; - } - - DdlSyncManager aligner = new DdlSyncManager( - sourceEngineConnector, - sinkEngineConnector, - executionEnviron.getCommonConfiguration(), - executionEnviron.getGlobalConfiguration(), - executionEnviron.getGlobalConfiguration()); - aligner.doColumnAlignment(source.isSchemaComparable()); - - executionEnviron.refreshConfiguration(); - } - - /** - * configure each of reader/writer DAG builders - * - * @param readerBuilders - * @param writerBuilders - */ - public void configureDAGBuilders(List readerBuilders, - List writerBuilders) throws Exception { - List readerConfigurations = executionEnviron.getReaderConfigurations(); - List writerConfigurations = executionEnviron.getWriterConfigurations(); - Preconditions.checkState(readerBuilders.size() == readerConfigurations.size()); - Preconditions.checkState(writerBuilders.size() == writerConfigurations.size()); - for (int i = 0; i < readerBuilders.size(); ++i) { - readerBuilders.get(i).configure(executionEnviron, readerConfigurations.get(i)); - } - for (int i = 0; i < writerBuilders.size(); ++i) { - writerBuilders.get(i).configure(executionEnviron, writerConfigurations.get(i)); - } - } -} diff --git a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/execution/configurer/FlinkDAGBuilderInterceptor.java b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/execution/configurer/FlinkDAGBuilderInterceptor.java new file mode 100644 index 000000000..3abfd5a31 --- /dev/null +++ b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/execution/configurer/FlinkDAGBuilderInterceptor.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.bytedance.bitsail.flink.core.execution.configurer; + +import com.bytedance.bitsail.base.catalog.TableCatalogFactory; +import com.bytedance.bitsail.base.catalog.TableCatalogFactoryHelper; +import com.bytedance.bitsail.base.connector.BuilderGroup; +import com.bytedance.bitsail.base.connector.reader.DataReaderDAGBuilder; +import com.bytedance.bitsail.base.connector.transformer.DataTransformDAGBuilder; +import com.bytedance.bitsail.base.connector.writer.DataWriterDAGBuilder; +import com.bytedance.bitsail.base.extension.TypeInfoConverterFactory; +import com.bytedance.bitsail.common.catalog.TableCatalogManager; +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.common.type.TypeInfoConverter; +import com.bytedance.bitsail.common.util.Preconditions; +import com.bytedance.bitsail.flink.core.execution.FlinkExecutionEnviron; + +import lombok.AllArgsConstructor; +import org.apache.commons.collections.CollectionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Objects; + +@AllArgsConstructor +public class FlinkDAGBuilderInterceptor { + private static final Logger LOG = LoggerFactory.getLogger(FlinkDAGBuilderInterceptor.class); + + private static final int READER_AND_WRITER_NUMBER = 1; + + private final FlinkExecutionEnviron executionEnviron; + + public void intercept(List readerBuilders, + List transformDAGBuilders, + List writerBuilders) throws Exception { + alignTableCatalog( + readerBuilders, + writerBuilders + ); + + executionEnviron.refreshConfiguration(); + + configureDAGBuilders(readerBuilders, writerBuilders); + } + + private void alignTableCatalog(List readerBuilders, + List writerBuilders) throws Exception { + //1. only support 1 source & 1 sink for align schema catalog. + if (CollectionUtils.size(readerBuilders) != READER_AND_WRITER_NUMBER || + CollectionUtils.size(writerBuilders) != READER_AND_WRITER_NUMBER) { + LOG.warn("Ignore align engine catalog, only support 1 reader and writer now."); + return; + } + DataReaderDAGBuilder dataReaderDAGBuilder = readerBuilders.get(0); + DataWriterDAGBuilder dataWriterDAGBuilder = writerBuilders.get(0); + + TableCatalogFactory readerCatalogFactory = TableCatalogFactoryHelper + .getTableCatalogFactory(dataReaderDAGBuilder.getReaderName()); + + TableCatalogFactory writerCatalogFactory = TableCatalogFactoryHelper + .getTableCatalogFactory(dataWriterDAGBuilder.getWriterName()); + + if (Objects.isNull(readerCatalogFactory) || Objects.isNull(writerCatalogFactory)) { + LOG.warn("Ignore align engine catalog, reader or writer not support table catalog factory."); + return; + } + + if (!(dataReaderDAGBuilder instanceof TypeInfoConverterFactory) || + !(dataWriterDAGBuilder instanceof TypeInfoConverterFactory)) { + LOG.warn("Ignore align engine catalog, reader or writer not support type info converter."); + return; + } + + TypeInfoConverter readerTypeInfoConverter = ((TypeInfoConverterFactory) dataReaderDAGBuilder) + .createTypeInfoConverter(); + TypeInfoConverter writerTypeInfoConverter = ((TypeInfoConverterFactory) dataWriterDAGBuilder) + .createTypeInfoConverter(); + + if (Objects.isNull(readerTypeInfoConverter) || Objects.isNull(writerTypeInfoConverter)) { + LOG.warn("Ignore align engine catalog, reader or writer type info converter is null."); + return; + } + + BitSailConfiguration readerConfiguration = executionEnviron.getReaderConfigurations().get(0); + BitSailConfiguration writerConfiguration = executionEnviron.getWriterConfigurations().get(0); + + TableCatalogManager catalogManager = TableCatalogManager.builder() + .readerTableCatalog(readerCatalogFactory.createTableCatalog(BuilderGroup.READER, executionEnviron, readerConfiguration)) + .writerTableCatalog(writerCatalogFactory.createTableCatalog(BuilderGroup.WRITER, executionEnviron, writerConfiguration)) + .readerTypeInfoConverter(readerTypeInfoConverter) + .writerTypeInfoConverter(writerTypeInfoConverter) + .commonConfiguration(executionEnviron.getCommonConfiguration()) + .readerConfiguration(readerConfiguration) + .writerConfiguration(writerConfiguration) + .build(); + + catalogManager.alignmentCatalogTable(); + } + + /** + * configure each of reader/writer DAG builders + */ + private void configureDAGBuilders(List readerBuilders, + List writerBuilders) throws Exception { + List readerConfigurations = executionEnviron.getReaderConfigurations(); + List writerConfigurations = executionEnviron.getWriterConfigurations(); + Preconditions.checkState(readerBuilders.size() == readerConfigurations.size()); + Preconditions.checkState(writerBuilders.size() == writerConfigurations.size()); + for (int i = 0; i < readerBuilders.size(); ++i) { + readerBuilders.get(i).configure(executionEnviron, readerConfigurations.get(i)); + } + for (int i = 0; i < writerBuilders.size(); ++i) { + writerBuilders.get(i).configure(executionEnviron, writerConfigurations.get(i)); + } + } +} diff --git a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/legacy/connector/InputFormatPlugin.java b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/legacy/connector/InputFormatPlugin.java index 075918123..2fa8f9fbc 100644 --- a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/legacy/connector/InputFormatPlugin.java +++ b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/legacy/connector/InputFormatPlugin.java @@ -20,6 +20,7 @@ import com.bytedance.bitsail.base.dirty.AbstractDirtyCollector; import com.bytedance.bitsail.base.dirty.DirtyCollectorFactory; import com.bytedance.bitsail.base.execution.ProcessResult; +import com.bytedance.bitsail.base.extension.TypeInfoConverterFactory; import com.bytedance.bitsail.base.messenger.BaseStatisticsMessenger; import com.bytedance.bitsail.base.messenger.Messenger; import com.bytedance.bitsail.base.messenger.MessengerFactory; @@ -35,7 +36,6 @@ import com.bytedance.bitsail.common.BitSailException; import com.bytedance.bitsail.common.column.ColumnCast; import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.source.SourceEngineConnector; import com.bytedance.bitsail.common.exception.CommonErrorCode; import com.bytedance.bitsail.common.option.CommonOptions; import com.bytedance.bitsail.common.option.ReaderOptions; @@ -78,7 +78,7 @@ * @desc: */ public abstract class InputFormatPlugin extends RichInputFormat implements - Pluggable { + Pluggable, TypeInfoConverterFactory { private static final Logger LOG = LoggerFactory.getLogger(InputFormatPlugin.class); @@ -378,14 +378,6 @@ public void incCompletedSplits(long count) { public abstract T[] createSplits(int minNumSplits) throws IOException; - public SourceEngineConnector initSourceSchemaManager(BitSailConfiguration commonConf, BitSailConfiguration readerConf) throws Exception { - return null; - } - - public boolean supportSchemaCheck() { - return false; - } - @VisibleForTesting public void setEmptyMessenger() { this.messenger = new BaseStatisticsMessenger(messengerContext); diff --git a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/legacy/connector/OutputFormatPlugin.java b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/legacy/connector/OutputFormatPlugin.java index 1f8fdb9ec..572f4ed07 100644 --- a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/legacy/connector/OutputFormatPlugin.java +++ b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/legacy/connector/OutputFormatPlugin.java @@ -20,6 +20,7 @@ import com.bytedance.bitsail.base.dirty.AbstractDirtyCollector; import com.bytedance.bitsail.base.dirty.DirtyCollectorFactory; import com.bytedance.bitsail.base.execution.ProcessResult; +import com.bytedance.bitsail.base.extension.TypeInfoConverterFactory; import com.bytedance.bitsail.base.messenger.BaseStatisticsMessenger; import com.bytedance.bitsail.base.messenger.Messenger; import com.bytedance.bitsail.base.messenger.MessengerFactory; @@ -34,7 +35,6 @@ import com.bytedance.bitsail.common.BitSailException; import com.bytedance.bitsail.common.column.ColumnCast; import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.sink.SinkEngineConnector; import com.bytedance.bitsail.common.exception.CommonErrorCode; import com.bytedance.bitsail.common.option.CommonOptions; import com.bytedance.bitsail.common.util.Pair; @@ -67,7 +67,7 @@ * @desc: */ public abstract class OutputFormatPlugin extends RichOutputFormat implements - InitializeOnMaster, CleanupWhenUnsuccessful, Pluggable, FinalizeOnMaster { + InitializeOnMaster, CleanupWhenUnsuccessful, Pluggable, FinalizeOnMaster, TypeInfoConverterFactory { private static final Logger LOG = LoggerFactory.getLogger(OutputFormatPlugin.class); @@ -239,10 +239,6 @@ public boolean uniformedParallelism() { return false; } - public SinkEngineConnector initSinkSchemaManager(BitSailConfiguration commonConf, BitSailConfiguration writerConf) throws Exception { - return null; - } - @VisibleForTesting public void setEmptyMessenger() { this.messenger = new BaseStatisticsMessenger(messengerContext); diff --git a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/parallelism/UnionParallelismStrategy.java b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/parallelism/UnionParallelismStrategy.java index 9cc962b21..c446897b6 100644 --- a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/parallelism/UnionParallelismStrategy.java +++ b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/parallelism/UnionParallelismStrategy.java @@ -31,7 +31,7 @@ public enum UnionParallelismStrategy { public int computeUnionParallelism(List readerParallelismAdvices) { List parallelismList = readerParallelismAdvices.stream().map( - advice -> advice.getAdviceParallelism()).collect(Collectors.toList()); + ParallelismAdvice::getAdviceParallelism).collect(Collectors.toList()); switch (this) { case MAX: diff --git a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/reader/FlinkSourceDAGBuilder.java b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/reader/FlinkSourceDAGBuilder.java index d59aee087..f9bf64905 100644 --- a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/reader/FlinkSourceDAGBuilder.java +++ b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/reader/FlinkSourceDAGBuilder.java @@ -28,6 +28,7 @@ import com.bytedance.bitsail.base.execution.ProcessResult; import com.bytedance.bitsail.base.extension.GlobalCommittable; import com.bytedance.bitsail.base.extension.ParallelismComputable; +import com.bytedance.bitsail.base.extension.TypeInfoConverterFactory; import com.bytedance.bitsail.base.messenger.Messenger; import com.bytedance.bitsail.base.messenger.MessengerFactory; import com.bytedance.bitsail.base.messenger.checker.DirtyRecordChecker; @@ -39,6 +40,7 @@ import com.bytedance.bitsail.common.configuration.BitSailConfiguration; import com.bytedance.bitsail.common.exception.CommonErrorCode; import com.bytedance.bitsail.common.option.CommonOptions; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.flink.core.delagate.reader.source.DelegateFlinkSource; import com.bytedance.bitsail.flink.core.delagate.reader.source.operator.DelegateSourceOperatorFactory; import com.bytedance.bitsail.flink.core.execution.FlinkExecutionEnviron; @@ -66,7 +68,7 @@ import java.lang.reflect.InvocationTargetException; public class FlinkSourceDAGBuilder - implements DataReaderDAGBuilder, ParallelismComputable, GlobalCommittable { + implements DataReaderDAGBuilder, ParallelismComputable, GlobalCommittable, TypeInfoConverterFactory { private static final Logger LOG = LoggerFactory.getLogger(FlinkSourceDAGBuilder.class); private final Source source; @@ -183,4 +185,9 @@ public void commit(ProcessResult processResult) throws Exception { public void abort() throws Exception { } + + @Override + public TypeInfoConverter createTypeInfoConverter() { + return source.createTypeInfoConverter(); + } } diff --git a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/reader/PluginableInputFormatDAGBuilder.java b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/reader/PluginableInputFormatDAGBuilder.java index 6fb84082b..ac9ff2c81 100644 --- a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/reader/PluginableInputFormatDAGBuilder.java +++ b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/reader/PluginableInputFormatDAGBuilder.java @@ -21,13 +21,13 @@ import com.bytedance.bitsail.base.execution.ProcessResult; import com.bytedance.bitsail.base.extension.GlobalCommittable; import com.bytedance.bitsail.base.extension.ParallelismComputable; -import com.bytedance.bitsail.base.extension.SchemaAlignmentable; +import com.bytedance.bitsail.base.extension.TypeInfoConverterFactory; import com.bytedance.bitsail.base.parallelism.ParallelismAdvice; import com.bytedance.bitsail.common.BitSailException; import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.ExternalEngineConnector; import com.bytedance.bitsail.common.exception.CommonErrorCode; import com.bytedance.bitsail.common.option.ReaderOptions; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.flink.core.constants.TypeSystem; import com.bytedance.bitsail.flink.core.execution.FlinkExecutionEnviron; import com.bytedance.bitsail.flink.core.legacy.connector.InputFormatPlugin; @@ -58,7 +58,7 @@ * Created 2022/4/21 */ public class PluginableInputFormatDAGBuilder extends FlinkDataReaderDAGBuilder - implements GlobalCommittable, SchemaAlignmentable { + implements GlobalCommittable, TypeInfoConverterFactory { private static final Logger LOG = LoggerFactory.getLogger(PluginableInputFormatDAGBuilder.class); @Getter @@ -134,24 +134,6 @@ public void abort() throws Exception { inputFormatPlugin.onFailureComplete(); } - @Override - public ExternalEngineConnector createExternalEngineConnector(ExecutionEnviron executionEnviron, - BitSailConfiguration readerConfiguration) { - BitSailConfiguration commonConfiguration = executionEnviron.getCommonConfiguration(); - ExternalEngineConnector sourceEngineConnector = null; - try { - sourceEngineConnector = inputFormatPlugin.initSourceSchemaManager(commonConfiguration, readerConfiguration); - } catch (Exception e) { - LOG.error("failed to init source engine connector for {}", this.getReaderName()); - } - return sourceEngineConnector; - } - - @Override - public boolean isSchemaComparable() { - return inputFormatPlugin.supportSchemaCheck(); - } - @Override public ParallelismAdvice getParallelismAdvice(BitSailConfiguration commonConf, BitSailConfiguration readerConf, @@ -190,6 +172,11 @@ public void onDestroy() throws Exception { } } + @Override + public TypeInfoConverter createTypeInfoConverter() { + return inputFormatPlugin.createTypeInfoConverter(); + } + @VisibleForTesting public void setInputFormatPlugin(InputFormatPlugin inputFormatPlugin) { this.inputFormatPlugin = inputFormatPlugin; diff --git a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/writer/FlinkWriterBuilder.java b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/writer/FlinkWriterBuilder.java index 138f2813a..c95541b19 100644 --- a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/writer/FlinkWriterBuilder.java +++ b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/writer/FlinkWriterBuilder.java @@ -26,6 +26,7 @@ import com.bytedance.bitsail.base.execution.Mode; import com.bytedance.bitsail.base.execution.ProcessResult; import com.bytedance.bitsail.base.extension.GlobalCommittable; +import com.bytedance.bitsail.base.extension.TypeInfoConverterFactory; import com.bytedance.bitsail.base.messenger.Messenger; import com.bytedance.bitsail.base.messenger.checker.DirtyRecordChecker; import com.bytedance.bitsail.base.messenger.common.MessengerGroup; @@ -34,6 +35,7 @@ import com.bytedance.bitsail.base.ratelimit.Channel; import com.bytedance.bitsail.common.configuration.BitSailConfiguration; import com.bytedance.bitsail.common.option.CommonOptions; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.flink.core.delagate.writer.DelegateFlinkCommitter; import com.bytedance.bitsail.flink.core.delagate.writer.DelegateFlinkWriter; import com.bytedance.bitsail.flink.core.execution.FlinkExecutionEnviron; @@ -54,7 +56,7 @@ * Created 2022/6/10 */ public class FlinkWriterBuilder - extends FlinkDataWriterDAGBuilder implements GlobalCommittable { + extends FlinkDataWriterDAGBuilder implements GlobalCommittable, TypeInfoConverterFactory { private static final Logger LOG = LoggerFactory.getLogger(FlinkWriterBuilder.class); @@ -173,4 +175,9 @@ public void abort() { public void onDestroy() { } + + @Override + public TypeInfoConverter createTypeInfoConverter() { + return sink.createTypeInfoConverter(); + } } diff --git a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/writer/PluginableOutputFormatDAGBuilder.java b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/writer/PluginableOutputFormatDAGBuilder.java index 20051954a..fd23ffdd5 100644 --- a/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/writer/PluginableOutputFormatDAGBuilder.java +++ b/bitsail-cores/bitsail-core-flink/src/main/java/com/bytedance/bitsail/flink/core/writer/PluginableOutputFormatDAGBuilder.java @@ -20,11 +20,11 @@ import com.bytedance.bitsail.base.execution.ExecutionEnviron; import com.bytedance.bitsail.base.execution.ProcessResult; import com.bytedance.bitsail.base.extension.GlobalCommittable; -import com.bytedance.bitsail.base.extension.SchemaAlignmentable; +import com.bytedance.bitsail.base.extension.TypeInfoConverterFactory; import com.bytedance.bitsail.base.parallelism.ParallelismAdvice; import com.bytedance.bitsail.common.configuration.BitSailConfiguration; -import com.bytedance.bitsail.common.ddl.ExternalEngineConnector; import com.bytedance.bitsail.common.option.WriterOptions; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import com.bytedance.bitsail.flink.core.constants.TypeSystem; import com.bytedance.bitsail.flink.core.legacy.connector.OutputFormatPlugin; import com.bytedance.bitsail.flink.core.option.FlinkCommonOptions; @@ -45,7 +45,8 @@ /** * Created 2022/4/21 */ -public class PluginableOutputFormatDAGBuilder extends FlinkDataWriterDAGBuilder implements GlobalCommittable, SchemaAlignmentable { +public class PluginableOutputFormatDAGBuilder extends FlinkDataWriterDAGBuilder implements GlobalCommittable, + TypeInfoConverterFactory { private static final Logger LOG = LoggerFactory.getLogger(PluginableOutputFormatDAGBuilder.class); @Getter @@ -105,19 +106,6 @@ public void abort() throws Exception { outputFormatPlugin.onFailureComplete(); } - @Override - public ExternalEngineConnector createExternalEngineConnector(ExecutionEnviron executionEnviron, - BitSailConfiguration writerConfiguration) { - BitSailConfiguration commonConfiguration = executionEnviron.getCommonConfiguration(); - ExternalEngineConnector sinkEngineConnector = null; - try { - sinkEngineConnector = outputFormatPlugin.initSinkSchemaManager(commonConfiguration, writerConfiguration); - } catch (Exception e) { - LOG.error("failed to init sink engine connector for {}", this.getWriterName()); - } - return sinkEngineConnector; - } - @Override public ParallelismAdvice getParallelismAdvice(BitSailConfiguration commonConf, BitSailConfiguration writerConf, @@ -156,4 +144,9 @@ public void onDestroy() throws Exception { public void setOutputFormatPlugin(OutputFormatPlugin outputFormatPlugin) { this.outputFormatPlugin = outputFormatPlugin; } + + @Override + public TypeInfoConverter createTypeInfoConverter() { + return outputFormatPlugin.createTypeInfoConverter(); + } } diff --git a/bitsail-cores/bitsail-core-flink/src/test/java/com/bytedance/bitsail/flink/core/legacy/connector/InputFormatPluginTest.java b/bitsail-cores/bitsail-core-flink/src/test/java/com/bytedance/bitsail/flink/core/legacy/connector/InputFormatPluginTest.java index 94d1e08a5..ca2d93bc9 100644 --- a/bitsail-cores/bitsail-core-flink/src/test/java/com/bytedance/bitsail/flink/core/legacy/connector/InputFormatPluginTest.java +++ b/bitsail-cores/bitsail-core-flink/src/test/java/com/bytedance/bitsail/flink/core/legacy/connector/InputFormatPluginTest.java @@ -18,6 +18,8 @@ package com.bytedance.bitsail.flink.core.legacy.connector; import com.bytedance.bitsail.common.BitSailException; +import com.bytedance.bitsail.common.type.BitSailTypeInfoConverter; +import com.bytedance.bitsail.common.type.TypeInfoConverter; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.core.io.GenericInputSplit; @@ -39,6 +41,11 @@ public void createInputSplits() throws IOException { AtomicInteger splitsCreateCalledTime = new AtomicInteger(0); final InputFormatPlugin input = new InputFormatPlugin() { + @Override + public TypeInfoConverter createTypeInfoConverter() { + return new BitSailTypeInfoConverter(); + } + @Override public BaseStatistics getStatistics(BaseStatistics cachedStatistics) { return null; diff --git a/bitsail-test/bitsail-connector-test/src/main/java/com/bytedance/bitsail/test/connector/test/EmbeddedFlinkCluster.java b/bitsail-test/bitsail-connector-test/src/main/java/com/bytedance/bitsail/test/connector/test/EmbeddedFlinkCluster.java index 77ad54719..8dd013e84 100644 --- a/bitsail-test/bitsail-connector-test/src/main/java/com/bytedance/bitsail/test/connector/test/EmbeddedFlinkCluster.java +++ b/bitsail-test/bitsail-connector-test/src/main/java/com/bytedance/bitsail/test/connector/test/EmbeddedFlinkCluster.java @@ -17,6 +17,7 @@ package com.bytedance.bitsail.test.connector.test; +import com.bytedance.bitsail.common.catalog.TableCatalogOptions; import com.bytedance.bitsail.common.configuration.BitSailConfiguration; import com.bytedance.bitsail.common.option.CommonOptions; import com.bytedance.bitsail.core.command.CoreCommandArgs; @@ -50,7 +51,7 @@ public static void submitJob(BitSailConfiguration globalConfiguration) throw private static void overwriteConfiguration(BitSailConfiguration globalConfiguration) { globalConfiguration.set(CommonOptions.JOB_ID, DEFAULT_JOB_ID) - .set(CommonOptions.SYNC_DDL, false) + .set(TableCatalogOptions.SYNC_DDL, false) .set(CommonOptions.DRY_RUN, true) .set(CommonOptions.ENABLE_DYNAMIC_LOADER, false); }