diff --git a/.gitignore b/.gitignore index ce03d7c78..ccb9405fc 100644 --- a/.gitignore +++ b/.gitignore @@ -16,4 +16,6 @@ bin/nohup.out .DS_Store bin/sideSql.txt *.keytab -krb5.conf \ No newline at end of file +krb5.conf +.gradle +gradle \ No newline at end of file diff --git a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java index c82f6f625..da34ba7fc 100644 --- a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java @@ -18,7 +18,18 @@ package com.dtstack.flink.sql.side.cassandra; -import com.datastax.driver.core.*; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ConsistencyLevel; +import com.datastax.driver.core.HostDistance; +import com.datastax.driver.core.PoolingOptions; +import com.datastax.driver.core.QueryOptions; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.SocketOptions; import com.datastax.driver.core.policies.DowngradingConsistencyRetryPolicy; import com.datastax.driver.core.policies.RetryPolicy; import com.dtstack.flink.sql.side.AbstractSideTableInfo; diff --git a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java index c017f1a36..2ee2e442c 100644 --- a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java +++ b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java @@ -19,11 +19,22 @@ package com.dtstack.flink.sql.side.cassandra; -import com.datastax.driver.core.*; +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ConsistencyLevel; +import com.datastax.driver.core.HostDistance; +import com.datastax.driver.core.PoolingOptions; +import com.datastax.driver.core.QueryOptions; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.SocketOptions; import com.datastax.driver.core.policies.DowngradingConsistencyRetryPolicy; import com.datastax.driver.core.policies.RetryPolicy; import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.BaseAsyncReqRow; +import com.dtstack.flink.sql.side.CacheMissVal; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo; import com.google.common.base.Function; diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraOutputFormat.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraOutputFormat.java index 4428fbc76..e5ea13ccc 100644 --- a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraOutputFormat.java +++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraOutputFormat.java @@ -38,6 +38,7 @@ package com.dtstack.flink.sql.sink.cassandra; +import com.dtstack.flink.sql.util.SampleUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; @@ -192,7 +193,7 @@ public void open(int taskNumber, int numTasks) { * @see PreparedStatement */ @Override - public void writeRecord(Tuple2 tuple2) throws IOException { + public void writeRecord(Tuple2 tuple2) { Tuple2 tupleTrans = tuple2; Boolean retract = tupleTrans.getField(0); Row row = tupleTrans.getField(1); @@ -209,10 +210,7 @@ public void writeRecord(Tuple2 tuple2) throws IOException { private void insertWrite(Row row) { try { - - if(outRecords.getCount() % ROW_PRINT_FREQUENCY == 0){ - LOG.info("Receive data : {}", row); - } + SampleUtils.samplingSinkPrint(samplingIntervalCount, LOG, outRecords.getCount(), row.toString()); String cql = buildSql(row); if (cql != null) { diff --git a/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java b/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java index c9a0c447b..d770356a8 100644 --- a/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java +++ b/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java @@ -18,11 +18,11 @@ package com.dtstack.flink.sql.side.clickhouse; +import com.dtstack.flink.sql.classloader.ClassLoaderManager; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.all.AbstractRdbAllReqRow; -import com.dtstack.flink.sql.util.JDBCUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,7 +45,7 @@ public ClickhouseAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "ru.yandex.clickhouse.ClickHouseDriver"); AbstractTableInfo clickhouseTableInfo = super.getTableInfo(tableName, fieldsInfo, props); clickhouseTableInfo.setType(CURR_TYPE); return clickhouseTableInfo; diff --git a/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/table/ClickhouseSinkParser.java b/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/table/ClickhouseSinkParser.java index 5b0f2598f..75478061e 100644 --- a/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/table/ClickhouseSinkParser.java +++ b/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/table/ClickhouseSinkParser.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.sink.clickhouse.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; import com.dtstack.flink.sql.table.AbstractTableInfo; import ru.yandex.clickhouse.domain.ClickHouseDataType; @@ -31,6 +32,7 @@ public class ClickhouseSinkParser extends RdbSinkParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "ru.yandex.clickhouse.ClickHouseDriver"); AbstractTableInfo clickhouseTableInfo = super.getTableInfo(tableName, fieldsInfo, props); clickhouseTableInfo.setType(CURR_TYPE); return clickhouseTableInfo; diff --git a/core/pom.xml b/core/pom.xml index ce7799f5a..117f0d031 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -17,7 +17,6 @@ UTF-8 core - 1.16.0 2.7.9 19.0 1.0.0-SNAPSHOT @@ -33,37 +32,37 @@ org.apache.flink - flink-table-api-java-bridge_2.11 + flink-table-api-java-bridge_${scala.binary.version} ${flink.version} org.apache.flink - flink-table-planner-blink_2.11 + flink-table-planner-blink_${scala.binary.version} ${flink.version} org.apache.flink - flink-table-runtime-blink_2.11 + flink-table-runtime-blink_${scala.binary.version} ${flink.version} org.apache.flink - flink-streaming-java_2.11 + flink-streaming-java_${scala.binary.version} ${flink.version} org.apache.flink - flink-streaming-scala_2.11 + flink-streaming-scala_${scala.binary.version} ${flink.version} org.apache.flink - flink-cep-scala_2.11 + flink-cep-scala_${scala.binary.version} ${flink.version} @@ -75,24 +74,10 @@ org.apache.flink - flink-statebackend-rocksdb_2.11 + flink-statebackend-rocksdb_${scala.binary.version} ${flink.version} - - - org.apache.calcite - calcite-server - - ${calcite.server.version} - - - jackson-databind - com.fasterxml.jackson.core - - - - com.fasterxml.jackson.core jackson-databind @@ -123,6 +108,12 @@ joda-time 2.5 + + + log4j + log4j + 1.2.17 + diff --git a/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java b/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java index 94b8709af..082d490b3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java +++ b/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java @@ -28,6 +28,7 @@ import java.lang.reflect.Method; import java.net.URL; import java.net.URLClassLoader; +import java.sql.DriverManager; import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; @@ -45,11 +46,32 @@ public class ClassLoaderManager { private static final Logger LOG = LoggerFactory.getLogger(ClassLoaderManager.class); + private static final Map pluginClassLoader = new ConcurrentHashMap<>(); + private static final Object LOCK = new Object(); - private static Map pluginClassLoader = new ConcurrentHashMap<>(); // 维表插件的classloader,该集合存放的是AppClassloader private static Map pluginAppClassLoader = new ConcurrentHashMap<>(); + public static void forName(String clazz, ClassLoader classLoader) { + synchronized (LOCK) { + try { + Class.forName(clazz, true, classLoader); + DriverManager.setLoginTimeout(10); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + public synchronized static void forName(String clazz) { + try { + Class driverClass = Class.forName(clazz); + driverClass.newInstance(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + public static R newInstance(String pluginJarPath, ClassLoaderSupplier supplier) throws Exception { ClassLoader classLoader = retrieveClassLoad(pluginJarPath); return ClassLoaderSupplierCallBack.callbackAndReset(supplier, classLoader); @@ -57,8 +79,9 @@ public static R newInstance(String pluginJarPath, ClassLoaderSupplier sup /** * 使用AppClassloader实例化维表插件 + * * @param pluginJarPath 插件路径 - * @param supplier 类加载器 + * @param supplier 类加载器 * @param * @return * @throws Exception @@ -90,6 +113,7 @@ private static DtClassLoader retrieveClassLoad(String pluginJarPath) { /** * 使用AppClassloader加载维表插件 + * * @param pluginJarPath 插件路径 * @return */ @@ -133,7 +157,7 @@ public static List getClassPath() { } Set appClassPaths = new HashSet(); - for(Map.Entry entry : pluginAppClassLoader.entrySet()){ + for (Map.Entry entry : pluginAppClassLoader.entrySet()) { appClassPaths.addAll(Arrays.asList(entry.getValue().getURLs())); } classPaths.addAll(appClassPaths); @@ -141,11 +165,10 @@ public static List getClassPath() { } - public static URLClassLoader loadExtraJar(List jarUrlList, URLClassLoader classLoader) - throws IllegalAccessException, InvocationTargetException { - for(URL url : jarUrlList){ - if(url.toString().endsWith(".jar")){ + throws IllegalAccessException, InvocationTargetException { + for (URL url : jarUrlList) { + if (url.toString().endsWith(".jar")) { urlClassLoaderAddUrl(classLoader, url); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/constrant/ConfigConstrant.java b/core/src/main/java/com/dtstack/flink/sql/constrant/ConfigConstrant.java index 69a4bdcf8..76f01aea6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/constrant/ConfigConstrant.java +++ b/core/src/main/java/com/dtstack/flink/sql/constrant/ConfigConstrant.java @@ -67,7 +67,7 @@ public class ConfigConstrant { public static final String STATE_BACKEND_KEY = "state.backend"; public static final String CHECKPOINTS_DIRECTORY_KEY = "state.checkpoints.dir"; public static final String STATE_BACKEND_INCREMENTAL_KEY = "state.backend.incremental"; - + public static final String SAMPLE_INTERVAL_COUNT = "sample.interval.count"; public static final String RESTOREENABLE = "restore.enable"; public static final String LOG_LEVEL_KEY = "logLevel"; diff --git a/core/src/main/java/com/dtstack/flink/sql/dirtyManager/consumer/AbstractDirtyDataConsumer.java b/core/src/main/java/com/dtstack/flink/sql/dirtyManager/consumer/AbstractDirtyDataConsumer.java new file mode 100644 index 000000000..5d6759187 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/dirtyManager/consumer/AbstractDirtyDataConsumer.java @@ -0,0 +1,101 @@ +/* + * 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.dtstack.flink.sql.dirtyManager.consumer; + +import com.dtstack.flink.sql.dirtyManager.entity.DirtyDataEntity; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.util.Properties; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; + +/** + * @author tiezhu + * Company dtstack + * Date 2020/8/27 星期四 + */ +public abstract class AbstractDirtyDataConsumer implements Runnable, Serializable { + protected static final long serialVersionUID = -6058598201315176687L; + + protected static final Logger LOG = LoggerFactory.getLogger(AbstractDirtyDataConsumer.class); + + protected Long errorLimit = 1000L; + protected AtomicLong errorCount = new AtomicLong(0L); + + protected AtomicLong count = new AtomicLong(0L); + + public AtomicBoolean isRunning = new AtomicBoolean(true); + + protected LinkedBlockingQueue queue; + + /** + * 消费队列数据 + * + * @throws Exception throw exception + */ + public abstract void consume() throws Exception; + + /** + * 关闭消费者,需要释放资源 + */ + public abstract void close(); + + /** + * 初始化消费者,初始化定时任务 + * + * @param properties 任务参数 + * @throws Exception throw exception + */ + public abstract void init(Properties properties) throws Exception; + + /** + * 检验consumer是否正在执行 + */ + public boolean isRunning() { + return isRunning.get(); + } + + @Override + public void run() { + try { + while (isRunning.get()) { + consume(); + } + } catch (Exception e) { + LOG.error("consume dirtyData error", e); + errorCount.incrementAndGet(); + if (errorCount.get() > errorLimit) { + throw new RuntimeException("The task failed due to the number of dirty data consume failed reached the limit " + errorLimit); + } + } + } + + public AbstractDirtyDataConsumer setQueue(LinkedBlockingQueue queue) { + this.queue = queue; + return this; + } + + public void collectDirtyData(DirtyDataEntity dataEntity, long blockingInterval) throws InterruptedException { + queue.offer(dataEntity, blockingInterval, TimeUnit.MILLISECONDS); + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/dirtyManager/consumer/DirtyConsumerFactory.java b/core/src/main/java/com/dtstack/flink/sql/dirtyManager/consumer/DirtyConsumerFactory.java new file mode 100644 index 000000000..fea07357d --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/dirtyManager/consumer/DirtyConsumerFactory.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flink.sql.dirtyManager.consumer; + +import com.dtstack.flink.sql.classloader.ClassLoaderManager; +import com.dtstack.flink.sql.util.PluginUtil; + +import java.io.File; +import java.lang.reflect.Constructor; +import java.util.Objects; + +import static com.dtstack.flink.sql.util.PluginUtil.upperCaseFirstChar; + +/** + * @author tiezhu + * Company dtstack + * Date 2020/12/21 星期一 + */ +public class DirtyConsumerFactory { + private static final String DEFAULT_DIRTY_TYPE = "console"; + private static final String DIRTY_CONSUMER_PATH = "dirtyData"; + private static final String CLASS_PRE_STR = "com.dtstack.flink.sql.dirty"; + private static final String CLASS_POST_STR = "DirtyDataConsumer"; + + /** + * 通过动态方式去加载脏数据插件 + * + * @param dirtyType 脏数据插件类型 + * @param pluginPath 脏数据插件直地址 + * @param pluginLoadMode 插件加载方式 + * @return 脏数据消费者 + * @throws Exception exception + */ + public static AbstractDirtyDataConsumer getDirtyConsumer( + String dirtyType + , String pluginPath + , String pluginLoadMode) throws Exception { + if (Objects.isNull(dirtyType)) { + dirtyType = DEFAULT_DIRTY_TYPE; + } + String consumerType = DIRTY_CONSUMER_PATH + File.separator + dirtyType; + String consumerJar = PluginUtil.getJarFileDirPath(consumerType, pluginPath, pluginLoadMode); + String className = CLASS_PRE_STR + "." + dirtyType.toLowerCase() + "." + upperCaseFirstChar(dirtyType + CLASS_POST_STR); + return ClassLoaderManager.newInstance(consumerJar, cl -> { + Class clazz = cl.loadClass(className); + Constructor constructor = clazz.getConstructor(); + return (AbstractDirtyDataConsumer) constructor.newInstance(); + }); + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/dirtyManager/entity/DirtyDataEntity.java b/core/src/main/java/com/dtstack/flink/sql/dirtyManager/entity/DirtyDataEntity.java new file mode 100644 index 000000000..42466970d --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/dirtyManager/entity/DirtyDataEntity.java @@ -0,0 +1,112 @@ +/* + * 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.dtstack.flink.sql.dirtyManager.entity; + +import java.sql.Date; +import java.text.SimpleDateFormat; + +/** + * @author tiezhu + * Company dtstack + * Date 2020/8/27 星期四 + */ +public class DirtyDataEntity { + private final SimpleDateFormat timeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + /** + * 脏数据信息内容 + */ + private String dirtyData; + + /** + * 脏数据处理时间 + */ + private String processDate; + + /** + * 产生脏数据的原因 + */ + private String cause; + + /** + * 产生异常的字段 + */ + private String field; + + public String getDirtyData() { + return dirtyData; + } + + public void setDirtyData(String dirtyData) { + this.dirtyData = dirtyData; + } + + public String getProcessDate() { + return processDate; + } + + public void setProcessDate(String processDate) { + this.processDate = processDate; + } + + public String getCause() { + return cause; + } + + public void setCause(String cause) { + this.cause = cause; + } + + public String getField() { + return field; + } + + public void setField(String field) { + this.field = field; + } + + public DirtyDataEntity(String dirtyData, Long processDate, String cause, String field) { + this.dirtyData = dirtyData; + this.processDate = timeFormat.format(processDate); + this.cause = cause; + this.field = field; + } + + public DirtyDataEntity(String dirtyData, Long processDate, String cause) { + this.dirtyData = dirtyData; + this.processDate = timeFormat.format(processDate); + this.cause = cause; + } + + @Override + public String toString() { + return "DirtyDataEntity{" + + "dirtyData='" + dirtyData + '\'' + + ", processDate=" + processDate + + ", cause='" + cause + '\'' + + '}'; + } + + /** + * 获取脏数据信息,返回字符数组 + * @return 脏数据信息字符数组 + */ + public String[] get() { + return new String[] {dirtyData, String.valueOf(processDate), cause}; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/dirtyManager/manager/DirtyDataManager.java b/core/src/main/java/com/dtstack/flink/sql/dirtyManager/manager/DirtyDataManager.java new file mode 100644 index 000000000..c13ea55fb --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/dirtyManager/manager/DirtyDataManager.java @@ -0,0 +1,151 @@ +/* + * 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.dtstack.flink.sql.dirtyManager.manager; + +import com.alibaba.fastjson.JSONObject; +import com.dtstack.flink.sql.dirtyManager.consumer.AbstractDirtyDataConsumer; +import com.dtstack.flink.sql.dirtyManager.consumer.DirtyConsumerFactory; +import com.dtstack.flink.sql.dirtyManager.entity.DirtyDataEntity; +import com.dtstack.flink.sql.factory.DTThreadFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.util.Properties; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import static com.dtstack.flink.sql.dirtyManager.manager.DirtyKeys.DEFAULT_BLOCKING_INTERVAL; +import static com.dtstack.flink.sql.dirtyManager.manager.DirtyKeys.DEFAULT_ERROR_LIMIT_RATE; +import static com.dtstack.flink.sql.dirtyManager.manager.DirtyKeys.DEFAULT_PRINT_LIMIT; +import static com.dtstack.flink.sql.dirtyManager.manager.DirtyKeys.DEFAULT_TYPE; +import static com.dtstack.flink.sql.dirtyManager.manager.DirtyKeys.DIRTY_BLOCK_STR; +import static com.dtstack.flink.sql.dirtyManager.manager.DirtyKeys.DIRTY_LIMIT_RATE_STR; +import static com.dtstack.flink.sql.dirtyManager.manager.DirtyKeys.PLUGIN_LOAD_MODE_STR; +import static com.dtstack.flink.sql.dirtyManager.manager.DirtyKeys.PLUGIN_PATH_STR; +import static com.dtstack.flink.sql.dirtyManager.manager.DirtyKeys.PLUGIN_TYPE_STR; +import static com.dtstack.flink.sql.dirtyManager.manager.DirtyKeys.PRINT_LIMIT_STR; + +/** + * @author tiezhu + * Company dtstack + * Date 2020/8/27 星期四 + */ +public class DirtyDataManager implements Serializable { + + public final static int MAX_POOL_SIZE_LIMIT = 5; + private static final long serialVersionUID = 7190970299538893497L; + private static final Logger LOG = LoggerFactory.getLogger(DirtyDataManager.class); + private final static int MAX_TASK_QUEUE_SIZE = 100; + public static AbstractDirtyDataConsumer consumer; + + private static ThreadPoolExecutor dirtyDataConsumer; + /** + * 统计manager收集到的脏数据条数 + */ + private final AtomicLong count = new AtomicLong(0); + /** + * 脏数据写入队列失败条数 + */ + private final AtomicLong errorCount = new AtomicLong(0); + /** + * 写入队列阻塞时间 + */ + private long blockingInterval; + /** + * 任务失败的脏数据比例 + */ + private double errorLimitRate; + + /** + * 通过参数生成manager实例,并同时将consumer实例化 + */ + public static DirtyDataManager newInstance(Properties properties) { + try { + DirtyDataManager manager = new DirtyDataManager(); + manager.blockingInterval = Long.parseLong(String.valueOf(properties.getOrDefault(DIRTY_BLOCK_STR, DEFAULT_BLOCKING_INTERVAL))); + manager.errorLimitRate = Double.parseDouble(String.valueOf(properties.getOrDefault(DIRTY_LIMIT_RATE_STR, DEFAULT_ERROR_LIMIT_RATE))); + consumer = DirtyConsumerFactory.getDirtyConsumer( + properties.getProperty(PLUGIN_TYPE_STR, DEFAULT_TYPE) + , properties.getProperty(PLUGIN_PATH_STR) + , properties.getProperty(PLUGIN_LOAD_MODE_STR) + ); + consumer.init(properties); + consumer.setQueue(new LinkedBlockingQueue<>()); + dirtyDataConsumer = new ThreadPoolExecutor(MAX_POOL_SIZE_LIMIT, MAX_POOL_SIZE_LIMIT, 0, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(MAX_TASK_QUEUE_SIZE), new DTThreadFactory("dirtyDataConsumer", true), new ThreadPoolExecutor.CallerRunsPolicy()); + dirtyDataConsumer.execute(consumer); + return manager; + } catch (Exception e) { + throw new RuntimeException("create dirtyManager error!", e); + } + } + + /** + * 设置脏数据插件默认配置 + * + * @return console的默认配置 + */ + public static String buildDefaultDirty() { + JSONObject jsonObject = new JSONObject(); + jsonObject.put(PLUGIN_TYPE_STR, DEFAULT_TYPE); + jsonObject.put(PRINT_LIMIT_STR, DEFAULT_PRINT_LIMIT); + return jsonObject.toJSONString(); + } + + /** + * 脏数据收集任务停止,任务停止之前,需要将队列中所有的数据清空 + * TODO consumer 关闭时仍有数据没有消费到,假如有500条数据,在结束时实际消费数量可能只有493 + */ + public void close() { + if (checkConsumer()) { + LOG.info("dirty consumer is closing ..."); + consumer.close(); + dirtyDataConsumer.shutdownNow(); + } + } + + /** + * 收集脏数据放入队列缓存中,记录放入失败的数目和存入队列中的总数目,如果放入失败的数目超过一定比例,那么manager任务失败 + */ + public void collectDirtyData(String dataInfo, String cause) { + DirtyDataEntity dirtyDataEntity = new DirtyDataEntity(dataInfo, System.currentTimeMillis(), cause); + try { + consumer.collectDirtyData(dirtyDataEntity, blockingInterval); + count.incrementAndGet(); + } catch (Exception ignored) { + LOG.warn("dirty Data insert error ... Failed number: " + errorCount.incrementAndGet()); + LOG.warn("error dirty data:" + dirtyDataEntity.toString()); + if (errorCount.get() > Math.ceil(count.longValue() * errorLimitRate)) { + // close consumer and manager + close(); + throw new RuntimeException(String.format("The number of failed number 【%s】 reaches the limit, manager fails", errorCount.get())); + } + } + } + + /** + * 查看consumer当前状态 + */ + public boolean checkConsumer() { + return consumer.isRunning(); + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/dirtyManager/manager/DirtyKeys.java b/core/src/main/java/com/dtstack/flink/sql/dirtyManager/manager/DirtyKeys.java new file mode 100644 index 000000000..9fc134690 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/dirtyManager/manager/DirtyKeys.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flink.sql.dirtyManager.manager; + +/** + * Date: 2021/1/6 + * + * @author tiezhu + * Company dtstack + */ +public class DirtyKeys { + public final static String DEFAULT_TYPE = "console"; + public final static String DEFAULT_BLOCKING_INTERVAL = "60"; + public final static String DEFAULT_ERROR_LIMIT_RATE = "0.8"; + public final static String DEFAULT_PRINT_LIMIT = "1000"; + + public final static String DIRTY_BLOCK_STR = "blockingInterval"; + public final static String DIRTY_LIMIT_RATE_STR = "errorLimitRate"; + public final static String PLUGIN_TYPE_STR = "type"; + public final static String PLUGIN_PATH_STR = "pluginPath"; + public final static String PLUGIN_LOAD_MODE_STR = "pluginLoadMode"; + + public final static String PRINT_LIMIT_STR = "printLimit"; +} diff --git a/core/src/main/java/com/dtstack/flink/sql/environment/StreamEnvConfigManager.java b/core/src/main/java/com/dtstack/flink/sql/environment/StreamEnvConfigManager.java index c8ae49606..1c043d6ee 100644 --- a/core/src/main/java/com/dtstack/flink/sql/environment/StreamEnvConfigManager.java +++ b/core/src/main/java/com/dtstack/flink/sql/environment/StreamEnvConfigManager.java @@ -85,10 +85,12 @@ public static void streamExecutionEnvironmentConfig(StreamExecutionEnvironment s ExecutionConfig exeConfig = streamEnv.getConfig(); if (exeConfig.getGlobalJobParameters() == null) { exeConfig.setGlobalJobParameters(globalJobParameters); - } else if (exeConfig.getGlobalJobParameters() instanceof ExecutionConfig.GlobalJobParameters) { + } else if (exeConfig.getGlobalJobParameters() != null) { exeConfig.setGlobalJobParameters(globalJobParameters); } + disableChainOperator(streamEnv, globalJobParameters); + getEnvParallelism(confProperties).ifPresent(streamEnv::setParallelism); getMaxEnvParallelism(confProperties).ifPresent(streamEnv::setMaxParallelism); getBufferTimeoutMillis(confProperties).ifPresent(streamEnv::setBufferTimeout); @@ -364,4 +366,10 @@ private static Long getTtlTime(Integer timeNumber, String timeUnit) { throw new RuntimeException("not support " + timeNumber + timeUnit); } } + + private static void disableChainOperator(StreamExecutionEnvironment env, Configuration configuration) { + if(configuration.getBoolean("disableChainOperator", false)) { + env.disableOperatorChaining(); + } + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/exec/ExecuteProcessHelper.java b/core/src/main/java/com/dtstack/flink/sql/exec/ExecuteProcessHelper.java index 01f22782d..767dda481 100644 --- a/core/src/main/java/com/dtstack/flink/sql/exec/ExecuteProcessHelper.java +++ b/core/src/main/java/com/dtstack/flink/sql/exec/ExecuteProcessHelper.java @@ -20,6 +20,7 @@ import com.dtstack.flink.sql.classloader.ClassLoaderManager; import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.dirtyManager.manager.DirtyDataManager; import com.dtstack.flink.sql.enums.ClusterMode; import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.enums.EPluginLoadMode; @@ -35,6 +36,7 @@ import com.dtstack.flink.sql.parser.InsertSqlParser; import com.dtstack.flink.sql.parser.SqlParser; import com.dtstack.flink.sql.parser.SqlTree; +import com.dtstack.flink.sql.resource.ResourceCheck; import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.SideSqlExec; import com.dtstack.flink.sql.side.table.LookupTableSourceFactory; @@ -46,6 +48,7 @@ import com.dtstack.flink.sql.util.DataTypeUtils; import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; +import com.dtstack.flink.sql.util.SampleUtils; import com.dtstack.flink.sql.util.SqlFormatterUtil; import com.dtstack.flink.sql.watermarker.WaterMarkerAssigner; import com.fasterxml.jackson.databind.ObjectMapper; @@ -85,10 +88,13 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Properties; import java.util.Set; import java.util.TimeZone; +import static com.dtstack.flink.sql.constrant.ConfigConstrant.SAMPLE_INTERVAL_COUNT; + /** * 任务执行时的流程方法 * Date: 2020/2/17 @@ -103,6 +109,8 @@ public class ExecuteProcessHelper { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final String TIME_ZONE = "timezone"; + private static final String PLUGIN_PATH_STR = "pluginPath"; + private static final String PLUGIN_LOAD_STR = "pluginLoadMode"; public static FlinkPlanner flinkPlanner = new FlinkPlanner(); @@ -121,11 +129,23 @@ public static ParamsInfo parseParams(String[] args) throws Exception { String pluginLoadMode = options.getPluginLoadMode(); String deployMode = options.getMode(); String planner = options.getPlanner(); + String dirtyStr = options.getDirtyProperties(); Preconditions.checkArgument(checkRemoteSqlPluginPath(remoteSqlPluginPath, deployMode, pluginLoadMode), "Non-local mode or shipfile deployment mode, remoteSqlPluginPath is required"); String confProp = URLDecoder.decode(options.getConfProp(), Charsets.UTF_8.toString()); Properties confProperties = PluginUtil.jsonStrToObject(confProp, Properties.class); + Properties dirtyProperties = PluginUtil.jsonStrToObject(Objects.isNull(dirtyStr) ? + DirtyDataManager.buildDefaultDirty() : dirtyStr, Properties.class); + + if (Objects.isNull(dirtyProperties.getProperty(PLUGIN_LOAD_STR))) { + dirtyProperties.put(PLUGIN_LOAD_STR, pluginLoadMode); + } + + if (!pluginLoadMode.equalsIgnoreCase(EPluginLoadMode.LOCALTEST.name()) && Objects.isNull(dirtyProperties.getProperty(PLUGIN_PATH_STR))) { + dirtyProperties.setProperty(PLUGIN_PATH_STR, + Objects.isNull(remoteSqlPluginPath) ? localSqlPluginPath : remoteSqlPluginPath); + } List jarUrlList = getExternalJarUrls(options.getAddjar()); @@ -139,6 +159,7 @@ public static ParamsInfo parseParams(String[] args) throws Exception { .setConfProp(confProperties) .setPlanner(planner) .setJarUrlList(jarUrlList) + .setDirtyProperties(dirtyProperties) .build(); } @@ -159,7 +180,6 @@ public static boolean checkRemoteSqlPluginPath(String remoteSqlPluginPath, Strin return true; } - public static StreamTableEnvironment getStreamExecution(ParamsInfo paramsInfo) throws Exception { ClassLoader envClassLoader = Thread.currentThread().getContextClassLoader(); @@ -169,6 +189,8 @@ public static StreamTableEnvironment getStreamExecution(ParamsInfo paramsInfo) t StreamExecutionEnvironment env = ExecuteProcessHelper.getStreamExeEnv(paramsInfo.getConfProp(), paramsInfo.getDeployMode()); StreamTableEnvironment tableEnv = getStreamTableEnv(env, paramsInfo.getConfProp()); + setSamplingIntervalCount(paramsInfo); + ResourceCheck.NEED_CHECK = Boolean.parseBoolean(paramsInfo.getConfProp().getProperty(ResourceCheck.CHECK_STR, "true")); String planner = paramsInfo.getPlanner(); SqlParser.setLocalSqlPluginRoot(paramsInfo.getLocalSqlPluginPath()); @@ -180,19 +202,22 @@ public static StreamTableEnvironment getStreamExecution(ParamsInfo paramsInfo) t //register udf ExecuteProcessHelper.registerUserDefinedFunction(sqlTree, paramsInfo.getJarUrlList(), tableEnv, paramsInfo.isGetPlan()); //register table schema - Set classPathSets = ExecuteProcessHelper.registerTable(sqlTree + Set classPathSets = ExecuteProcessHelper.registerTable( + sqlTree , env , tableEnv , paramsInfo.getLocalSqlPluginPath() , paramsInfo.getRemoteSqlPluginPath() , paramsInfo.getPluginLoadMode() + , paramsInfo.getDirtyProperties() , sideTableMap , registerTableCache , planner); // cache classPathSets ExecuteProcessHelper.registerPluginUrlToCachedFile(env, classPathSets); - ExecuteProcessHelper.sqlTranslation(paramsInfo.getLocalSqlPluginPath() + ExecuteProcessHelper.sqlTranslation( + paramsInfo.getLocalSqlPluginPath() , paramsInfo.getPluginLoadMode() , tableEnv , sqlTree @@ -206,6 +231,13 @@ public static StreamTableEnvironment getStreamExecution(ParamsInfo paramsInfo) t return tableEnv; } + private static void setSamplingIntervalCount(ParamsInfo paramsInfo) { + SampleUtils.setSamplingIntervalCount( + Integer.parseInt( + paramsInfo.getConfProp().getProperty(SAMPLE_INTERVAL_COUNT, "0") + ) + ); + } public static List getExternalJarUrls(String addJarListStr) throws java.io.IOException { List jarUrlList = Lists.newArrayList(); @@ -285,7 +317,6 @@ private static void sqlTranslation(String localSqlPluginPath, } } } - scope++; } } @@ -327,19 +358,25 @@ public static void registerUserDefinedFunction(SqlTree sqlTree, List jarUrl * @return * @throws Exception */ - public static Set registerTable(SqlTree sqlTree + public static Set registerTable( + SqlTree sqlTree , StreamExecutionEnvironment env , StreamTableEnvironment tableEnv , String localSqlPluginPath , String remoteSqlPluginPath , String pluginLoadMode + , Properties dirtyProperties , Map sideTableMap , Map registerTableCache - , String planner) throws Exception { + , String planner + ) throws Exception { Set pluginClassPathSets = Sets.newHashSet(); WaterMarkerAssigner waterMarkerAssigner = new WaterMarkerAssigner(); for (AbstractTableInfo tableInfo : sqlTree.getTableInfoMap().values()) { + // 配置dirty manager + tableInfo.setDirtyProperties(dirtyProperties); + if (tableInfo instanceof AbstractSourceTableInfo) { AbstractSourceTableInfo sourceTableInfo = (AbstractSourceTableInfo) tableInfo; @@ -372,15 +409,29 @@ public static Set registerTable(SqlTree sqlTree } registerTableCache.put(tableInfo.getName(), regTable); - URL sourceTablePathUrl = PluginUtil.buildSourceAndSinkPathByLoadMode(tableInfo.getType(), AbstractSourceTableInfo.SOURCE_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode); + URL sourceTablePathUrl = PluginUtil.buildSourceAndSinkPathByLoadMode( + tableInfo.getType() + , AbstractSourceTableInfo.SOURCE_SUFFIX + , localSqlPluginPath + , remoteSqlPluginPath + , pluginLoadMode); pluginClassPathSets.add(sourceTablePathUrl); } else if (tableInfo instanceof AbstractTargetTableInfo) { - TableSink tableSink = StreamSinkFactory.getTableSink((AbstractTargetTableInfo) tableInfo, localSqlPluginPath, pluginLoadMode); - TypeInformation[] flinkTypes = DataTypeUtils.transformTypes(tableInfo.getFieldClasses()); - tableEnv.registerTableSink(tableInfo.getName(), tableInfo.getFields(), flinkTypes, tableSink); + // TODO Kafka Sink直接注册,其他的Sink要修复才可以。 + if (tableInfo.getType().startsWith("kafka")) { + tableEnv.registerTableSink(tableInfo.getName(), tableSink); + } else { + TypeInformation[] flinkTypes = DataTypeUtils.transformTypes(tableInfo.getFieldClasses()); + tableEnv.registerTableSink(tableInfo.getName(), tableInfo.getFields(), flinkTypes, tableSink); + } - URL sinkTablePathUrl = PluginUtil.buildSourceAndSinkPathByLoadMode(tableInfo.getType(), AbstractTargetTableInfo.TARGET_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode); + URL sinkTablePathUrl = PluginUtil.buildSourceAndSinkPathByLoadMode( + tableInfo.getType() + , AbstractTargetTableInfo.TARGET_SUFFIX + , localSqlPluginPath + , remoteSqlPluginPath + , pluginLoadMode); pluginClassPathSets.add(sinkTablePathUrl); } else if (tableInfo instanceof AbstractSideTableInfo) { String sideOperator = ECacheType.ALL.name().equalsIgnoreCase(((AbstractSideTableInfo) tableInfo).getCacheType()) ? "all" : "async"; @@ -391,7 +442,13 @@ public static Set registerTable(SqlTree sqlTree tableEnv.registerTableSource(tableInfo.getName(), tableSource); } - URL sideTablePathUrl = PluginUtil.buildSidePathByLoadMode(tableInfo.getType(), sideOperator, AbstractSideTableInfo.TARGET_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode); + URL sideTablePathUrl = PluginUtil.buildSidePathByLoadMode( + tableInfo.getType() + , sideOperator + , AbstractSideTableInfo.TARGET_SUFFIX + , localSqlPluginPath + , remoteSqlPluginPath + , pluginLoadMode); pluginClassPathSets.add(sideTablePathUrl); } else { throw new RuntimeException("not support table type:" + tableInfo.getType()); diff --git a/core/src/main/java/com/dtstack/flink/sql/exec/ParamsInfo.java b/core/src/main/java/com/dtstack/flink/sql/exec/ParamsInfo.java index 6d07704db..ff5bebc6e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/exec/ParamsInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/exec/ParamsInfo.java @@ -27,6 +27,7 @@ * 解析传递的参数信息 * Date: 2020/2/24 * Company: www.dtstack.com + * * @author maqi */ public class ParamsInfo { @@ -41,9 +42,19 @@ public class ParamsInfo { private Properties confProp; private String planner; private boolean getPlan = false; - - public ParamsInfo(String sql, String name, List jarUrlList, String localSqlPluginPath, - String remoteSqlPluginPath, String pluginLoadMode, String deployMode, Properties confProp, String planner) { + private Properties dirtyProperties; + + public ParamsInfo( + String sql + , String name + , List jarUrlList + , String localSqlPluginPath + , String remoteSqlPluginPath + , String pluginLoadMode + , String deployMode + , Properties confProp + , Properties dirtyProperties + , String planner) { this.sql = sql; this.name = name; this.jarUrlList = jarUrlList; @@ -53,6 +64,11 @@ public ParamsInfo(String sql, String name, List jarUrlList, String localSql this.deployMode = deployMode; this.confProp = confProp; this.planner = planner; + this.dirtyProperties = dirtyProperties; + } + + public static ParamsInfo.Builder builder() { + return new ParamsInfo.Builder(); } public boolean isGetPlan() { @@ -95,6 +111,10 @@ public Properties getConfProp() { return confProp; } + public Properties getDirtyProperties() { + return dirtyProperties; + } + public String getPlanner() { return planner; } @@ -104,12 +124,14 @@ public String toString() { return "ParamsInfo{" + "sql='" + sql + '\'' + ", name='" + name + '\'' + - ", jarUrlList=" + convertJarUrlListToString(jarUrlList) + + ", jarUrlList=" + jarUrlList + ", localSqlPluginPath='" + localSqlPluginPath + '\'' + ", remoteSqlPluginPath='" + remoteSqlPluginPath + '\'' + ", pluginLoadMode='" + pluginLoadMode + '\'' + ", deployMode='" + deployMode + '\'' + ", confProp=" + confProp + + ", getPlan=" + getPlan + + ", dirtyProperties='" + dirtyProperties + '\'' + ", planner=" + planner + '}'; } @@ -118,9 +140,6 @@ public String convertJarUrlListToString(List jarUrlList) { return jarUrlList.stream().map(URL::toString).reduce((pre, last) -> pre + last).orElse(""); } - public static ParamsInfo.Builder builder() { - return new ParamsInfo.Builder(); - } public static class Builder { private String sql; @@ -132,6 +151,7 @@ public static class Builder { private String deployMode; private Properties confProp; private String planner; + private Properties dirtyProperties; public ParamsInfo.Builder setSql(String sql) { this.sql = sql; @@ -174,14 +194,29 @@ public ParamsInfo.Builder setConfProp(Properties confProp) { return this; } + public ParamsInfo.Builder setDirtyProperties(Properties dirtyProperties) { + this.dirtyProperties = dirtyProperties; + return this; + } + public ParamsInfo.Builder setPlanner(String planner) { this.planner = planner; return this; } public ParamsInfo build() { - return new ParamsInfo(sql, name, jarUrlList, localSqlPluginPath, - remoteSqlPluginPath, pluginLoadMode, deployMode, confProp, planner); + return new ParamsInfo( + sql + , name + , jarUrlList + , localSqlPluginPath + , remoteSqlPluginPath + , pluginLoadMode + , deployMode + , confProp + , dirtyProperties + , planner + ); } } } diff --git a/core/src/main/java/com/dtstack/flink/sql/factory/DTThreadFactory.java b/core/src/main/java/com/dtstack/flink/sql/factory/DTThreadFactory.java index e9f9dfa64..ec4065f40 100644 --- a/core/src/main/java/com/dtstack/flink/sql/factory/DTThreadFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/factory/DTThreadFactory.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package com.dtstack.flink.sql.factory; @@ -33,6 +32,7 @@ public class DTThreadFactory implements ThreadFactory { private final static AtomicInteger THREAD_NUMBER = new AtomicInteger(1); private final ThreadGroup group; private final String namePrefix; + private Boolean isDaemon = false; public DTThreadFactory(String factoryName) { SecurityManager s = System.getSecurityManager(); @@ -43,14 +43,29 @@ public DTThreadFactory(String factoryName) { "-thread-"; } + public DTThreadFactory(String factoryName, Boolean isDaemon) { + SecurityManager s = System.getSecurityManager(); + group = (s != null) ? s.getThreadGroup() : + Thread.currentThread().getThreadGroup(); + namePrefix = factoryName + "-pool-" + + POOL_NUMBER.getAndIncrement() + + "-thread-"; + this.isDaemon = isDaemon; + } + @Override public Thread newThread(Runnable r) { Thread t = new Thread(group, r, namePrefix + THREAD_NUMBER.getAndIncrement(), 0); - if (t.isDaemon()) { - t.setDaemon(false); + if (this.isDaemon) { + t.setDaemon(true); + } else { + if (t.isDaemon()) { + t.setDaemon(false); + } } + if (t.getPriority() != Thread.NORM_PRIORITY) { t.setPriority(Thread.NORM_PRIORITY); } diff --git a/core/src/main/java/com/dtstack/flink/sql/format/DeserializationMetricWrapper.java b/core/src/main/java/com/dtstack/flink/sql/format/DeserializationMetricWrapper.java index 7c8424371..323defe6e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/format/DeserializationMetricWrapper.java +++ b/core/src/main/java/com/dtstack/flink/sql/format/DeserializationMetricWrapper.java @@ -18,7 +18,9 @@ package com.dtstack.flink.sql.format; +import com.dtstack.flink.sql.dirtyManager.manager.DirtyDataManager; import com.dtstack.flink.sql.metric.MetricConstant; +import com.dtstack.flink.sql.util.SampleUtils; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; import org.apache.flink.api.common.serialization.DeserializationSchema; @@ -31,6 +33,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.nio.charset.StandardCharsets; /** * add metric for source @@ -43,7 +46,7 @@ public class DeserializationMetricWrapper extends AbstractDeserializationSchema< private static final Logger LOG = LoggerFactory.getLogger(DeserializationMetricWrapper.class); - private static int dataPrintFrequency = 1000; + private int samplingIntervalCount = SampleUtils.getSamplingIntervalCount(); private DeserializationSchema deserializationSchema; @@ -69,9 +72,15 @@ public class DeserializationMetricWrapper extends AbstractDeserializationSchema< protected transient Meter numInBytesRate; - public DeserializationMetricWrapper(TypeInformation typeInfo, DeserializationSchema deserializationSchema) { + protected DirtyDataManager dirtyDataManager; + + public DeserializationMetricWrapper( + TypeInformation typeInfo + , DeserializationSchema deserializationSchema + , DirtyDataManager dirtyDataManager) { super(typeInfo); this.deserializationSchema = deserializationSchema; + this.dirtyDataManager = dirtyDataManager; } public void initMetric() { @@ -88,11 +97,14 @@ public void initMetric() { } @Override - public Row deserialize(byte[] message) throws IOException { + public Row deserialize(byte[] message) { try { - if (numInRecord.getCount() % dataPrintFrequency == 0) { - LOG.info("receive source data:" + new String(message, "UTF-8")); - } + SampleUtils.samplingSourcePrint( + samplingIntervalCount, + LOG, + numInRecord.getCount(), + new String(message, StandardCharsets.UTF_8) + ); numInRecord.inc(); numInBytes.inc(message.length); beforeDeserialize(); @@ -102,10 +114,7 @@ public Row deserialize(byte[] message) throws IOException { return row; } catch (Exception e) { //add metric of dirty data - if (dirtyDataCounter.getCount() % dataPrintFrequency == 0) { - LOG.info("dirtyData: " + new String(message)); - LOG.error("data parse error", e); - } + dirtyDataManager.collectDirtyData(new String(message), e.getMessage()); dirtyDataCounter.inc(); return null; } diff --git a/core/src/main/java/com/dtstack/flink/sql/function/FunctionManager.java b/core/src/main/java/com/dtstack/flink/sql/function/FunctionManager.java index 2d8580884..324e16fd9 100644 --- a/core/src/main/java/com/dtstack/flink/sql/function/FunctionManager.java +++ b/core/src/main/java/com/dtstack/flink/sql/function/FunctionManager.java @@ -20,7 +20,6 @@ package com.dtstack.flink.sql.function; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.functions.ScalarFunction; diff --git a/core/src/main/java/com/dtstack/flink/sql/option/Options.java b/core/src/main/java/com/dtstack/flink/sql/option/Options.java index 484471021..69c49056a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/option/Options.java +++ b/core/src/main/java/com/dtstack/flink/sql/option/Options.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -6,9 +6,9 @@ * 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. @@ -25,38 +25,39 @@ /** * This class define commandline options for the Launcher program - * + *

* Company: www.dtstack.com + * * @author huyifan.zju@163.com */ public class Options { @OptionRequired(description = "Running mode") - private String mode = ClusterMode.local.name(); + private String mode = ClusterMode.local.name(); - @OptionRequired(required = true,description = "Job name") - private String name; + @OptionRequired(required = true, description = "Job name") + private String name; - @OptionRequired(required = true,description = "Job sql file") - private String sql; + @OptionRequired(required = true, description = "Job sql file") + private String sql; @OptionRequired(description = "Flink configuration directory") - private String flinkconf; + private String flinkconf; @OptionRequired(description = "Yarn and Hadoop configuration directory") - private String yarnconf; + private String yarnconf; @OptionRequired(description = "Sql local plugin root") - private String localSqlPluginPath; + private String localSqlPluginPath; @OptionRequired(description = "Sql remote plugin root") - private String remoteSqlPluginPath ; + private String remoteSqlPluginPath; @OptionRequired(description = "sql ext jar,eg udf jar") - private String addjar; + private String addjar; @OptionRequired(description = "sql ref prop,eg specify event time") - private String confProp = "{}"; + private String confProp = "{}"; @OptionRequired(description = "flink jar path for submit of perjob mode") private String flinkJarPath; @@ -69,12 +70,15 @@ public class Options { @OptionRequired(description = "plugin load mode, by classpath or shipfile") private String pluginLoadMode = EPluginLoadMode.CLASSPATH.name(); - + @OptionRequired(description = "file add to ship file") - private String addShipfile; + private String addShipfile; + + @OptionRequired(description = "dirty plugin properties") + private String dirtyProperties; @OptionRequired(description = "sql planner") - private String planner = PlannerType.FLINK.name(); + private String planner = PlannerType.FLINK.name(); public String getMode() { @@ -189,12 +193,19 @@ public void setAddShipfile(String addShipfile) { this.addShipfile = addShipfile; } + public String getPlanner() { + return planner; + } + public void setPlanner(String planner) { this.planner = planner; } - public String getPlanner() { - return planner; + public String getDirtyProperties() { + return dirtyProperties; } -} + public void setDirtyProperties(String dirtyProperties) { + this.dirtyProperties = dirtyProperties; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/outputformat/AbstractDtRichOutputFormat.java b/core/src/main/java/com/dtstack/flink/sql/outputformat/AbstractDtRichOutputFormat.java index fbcc86bbd..91b6df46c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/outputformat/AbstractDtRichOutputFormat.java +++ b/core/src/main/java/com/dtstack/flink/sql/outputformat/AbstractDtRichOutputFormat.java @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.outputformat; import com.dtstack.flink.sql.metric.MetricConstant; +import com.dtstack.flink.sql.util.SampleUtils; import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Meter; @@ -32,8 +33,8 @@ public abstract class AbstractDtRichOutputFormat extends RichOutputFormat{ public transient Counter outRecords; public transient Counter outDirtyRecords; public transient Meter outRecordsRate; + protected int samplingIntervalCount = SampleUtils.getSamplingIntervalCount(); - protected static int ROW_PRINT_FREQUENCY = 1000; protected static int DIRTY_PRINT_FREQUENCY = 1000; public void initMetric() { diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java index 1e379f90f..132218406 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java @@ -21,7 +21,6 @@ import com.dtstack.flink.sql.util.DtStringUtil; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; -import org.apache.commons.lang3.StringUtils; import java.util.List; import java.util.Map; diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index 1c84672df..1c13946af 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -17,19 +17,19 @@ */ - package com.dtstack.flink.sql.parser; import com.dtstack.flink.sql.enums.PlannerType; import com.dtstack.flink.sql.util.DtStringUtil; +import com.google.common.collect.Lists; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlJoin; import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlMatchRecognize; import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlSnapshot; import org.apache.calcite.sql.SqlSelect; -import org.apache.calcite.sql.SqlMatchRecognize; -import com.google.common.collect.Lists; +import org.apache.calcite.sql.SqlSnapshot; + import java.util.List; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -40,6 +40,7 @@ * parser create tmp table sql * Date: 2018/6/26 * Company: www.dtstack.com + * * @author yanxi */ public class CreateTmpTableParser implements IParser { @@ -53,15 +54,79 @@ public class CreateTmpTableParser implements IParser { private static final Pattern EMPTYVIEW = Pattern.compile(EMPTY_STR); - private FlinkPlanner flinkPlanner = new FlinkPlanner(); + private final FlinkPlanner flinkPlanner = new FlinkPlanner(); - public static CreateTmpTableParser newInstance(){ + public static CreateTmpTableParser newInstance() { return new CreateTmpTableParser(); } + private static void parseNode(SqlNode sqlNode, CreateTmpTableParser.SqlParserResult sqlParseResult) { + SqlKind sqlKind = sqlNode.getKind(); + switch (sqlKind) { + case SELECT: + SqlNode sqlFrom = ((SqlSelect) sqlNode).getFrom(); + if (sqlFrom.getKind() == IDENTIFIER) { + sqlParseResult.addSourceTable(sqlFrom.toString()); + } else { + parseNode(sqlFrom, sqlParseResult); + } + break; + case JOIN: + SqlNode leftNode = ((SqlJoin) sqlNode).getLeft(); + SqlNode rightNode = ((SqlJoin) sqlNode).getRight(); + + if (leftNode.getKind() == IDENTIFIER) { + sqlParseResult.addSourceTable(leftNode.toString()); + } else { + parseNode(leftNode, sqlParseResult); + } + + if (rightNode.getKind() == IDENTIFIER) { + sqlParseResult.addSourceTable(rightNode.toString()); + } else { + parseNode(rightNode, sqlParseResult); + } + break; + case AS: + //不解析column,所以 as 相关的都是表 + SqlNode identifierNode = ((SqlBasicCall) sqlNode).getOperands()[0]; + if (identifierNode.getKind() != IDENTIFIER) { + parseNode(identifierNode, sqlParseResult); + } else { + sqlParseResult.addSourceTable(identifierNode.toString()); + } + break; + case UNION: + SqlNode unionLeft = ((SqlBasicCall) sqlNode).getOperands()[0]; + SqlNode unionRight = ((SqlBasicCall) sqlNode).getOperands()[1]; + if (unionLeft.getKind() == IDENTIFIER) { + sqlParseResult.addSourceTable(unionLeft.toString()); + } else { + parseNode(unionLeft, sqlParseResult); + } + if (unionRight.getKind() == IDENTIFIER) { + sqlParseResult.addSourceTable(unionRight.toString()); + } else { + parseNode(unionRight, sqlParseResult); + } + break; + case MATCH_RECOGNIZE: + SqlMatchRecognize node = (SqlMatchRecognize) sqlNode; + sqlParseResult.addSourceTable(node.getTableRef().toString()); + break; + case SNAPSHOT: + SqlSnapshot sqlSnapshot = (SqlSnapshot) sqlNode; + sqlParseResult.addSourceTable(sqlSnapshot.getTableRef().toString()); + break; + default: + //do nothing + break; + } + } + @Override public boolean verify(String sql) { - if (Pattern.compile(EMPTY_STR).matcher(sql).find()){ + if (Pattern.compile(EMPTY_STR).matcher(sql).find()) { return true; } return TEMPORARYVIEW.matcher(sql).find(); @@ -69,11 +134,11 @@ public boolean verify(String sql) { @Override public void parseSql(String sql, SqlTree sqlTree, String planner) { - if (TEMPORARYVIEW.matcher(sql).find()){ + if (TEMPORARYVIEW.matcher(sql).find()) { Matcher matcher = TEMPORARYVIEW.matcher(sql); String tableName = null; String selectSql = null; - if(matcher.find()) { + if (matcher.find()) { tableName = matcher.group(3); selectSql = "select " + matcher.group(4); } @@ -101,7 +166,7 @@ public void parseSql(String sql, SqlTree sqlTree, String planner) { Matcher matcher = EMPTYVIEW.matcher(sql); String tableName = null; String fieldsInfoStr = null; - if (matcher.find()){ + if (matcher.find()) { tableName = matcher.group(1); fieldsInfoStr = matcher.group(2); } @@ -112,71 +177,6 @@ public void parseSql(String sql, SqlTree sqlTree, String planner) { } } - - } - - private static void parseNode(SqlNode sqlNode, CreateTmpTableParser.SqlParserResult sqlParseResult){ - SqlKind sqlKind = sqlNode.getKind(); - switch (sqlKind){ - case SELECT: - SqlNode sqlFrom = ((SqlSelect)sqlNode).getFrom(); - if(sqlFrom.getKind() == IDENTIFIER){ - sqlParseResult.addSourceTable(sqlFrom.toString()); - }else{ - parseNode(sqlFrom, sqlParseResult); - } - break; - case JOIN: - SqlNode leftNode = ((SqlJoin)sqlNode).getLeft(); - SqlNode rightNode = ((SqlJoin)sqlNode).getRight(); - - if(leftNode.getKind() == IDENTIFIER){ - sqlParseResult.addSourceTable(leftNode.toString()); - }else{ - parseNode(leftNode, sqlParseResult); - } - - if(rightNode.getKind() == IDENTIFIER){ - sqlParseResult.addSourceTable(rightNode.toString()); - }else{ - parseNode(rightNode, sqlParseResult); - } - break; - case AS: - //不解析column,所以 as 相关的都是表 - SqlNode identifierNode = ((SqlBasicCall)sqlNode).getOperands()[0]; - if(identifierNode.getKind() != IDENTIFIER){ - parseNode(identifierNode, sqlParseResult); - }else { - sqlParseResult.addSourceTable(identifierNode.toString()); - } - break; - case UNION: - SqlNode unionLeft = ((SqlBasicCall)sqlNode).getOperands()[0]; - SqlNode unionRight = ((SqlBasicCall)sqlNode).getOperands()[1]; - if(unionLeft.getKind() == IDENTIFIER){ - sqlParseResult.addSourceTable(unionLeft.toString()); - }else{ - parseNode(unionLeft, sqlParseResult); - } - if(unionRight.getKind() == IDENTIFIER){ - sqlParseResult.addSourceTable(unionRight.toString()); - }else{ - parseNode(unionRight, sqlParseResult); - } - break; - case MATCH_RECOGNIZE: - SqlMatchRecognize node = (SqlMatchRecognize) sqlNode; - sqlParseResult.addSourceTable(node.getTableRef().toString()); - break; - case SNAPSHOT: - SqlSnapshot sqlSnapshot = (SqlSnapshot) sqlNode; - sqlParseResult.addSourceTable(sqlSnapshot.getTableRef().toString()); - break; - default: - //do nothing - break; - } } public static class SqlParserResult { @@ -212,13 +212,12 @@ public void setFieldsInfoStr(String fieldsInfoStr) { this.fieldsInfoStr = fieldsInfoStr; } - public void addSourceTable(String sourceTable){ + public void addSourceTable(String sourceTable) { sourceTableList.add(sourceTable); } public List getSourceTableList() { return sourceTableList; } - } } \ No newline at end of file diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/FlinkPlanner.java b/core/src/main/java/com/dtstack/flink/sql/parser/FlinkPlanner.java index f6182e69c..01419d1b5 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/FlinkPlanner.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/FlinkPlanner.java @@ -41,6 +41,7 @@ * * @author maqi */ +@Deprecated public class FlinkPlanner { private final TableConfig tableConfig = new TableConfig(); @@ -60,7 +61,8 @@ public class FlinkPlanner { catalogManager, moduleManager); private final PlannerContext plannerContext = - new PlannerContext(tableConfig, + new PlannerContext( + tableConfig, functionCatalog, catalogManager, asRootSchema(new CatalogManagerCalciteSchema(catalogManager, false)), diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java index 32d671b05..0784fca3c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package com.dtstack.flink.sql.parser; @@ -24,9 +23,9 @@ import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.table.AbstractTableInfoParser; import com.dtstack.flink.sql.util.DtStringUtil; -import org.apache.commons.lang3.StringUtils; -import com.google.common.collect.Lists; import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,6 +38,7 @@ * Reason: * Date: 2018/6/22 * Company: www.dtstack.com + * * @author xuchao */ @@ -46,29 +46,27 @@ public class SqlParser { private static final Logger LOG = LoggerFactory.getLogger(SqlParser.class); private static final char SQL_DELIMITER = ';'; - + private static final Pattern ADD_FILE_AND_JAR_PATTERN = Pattern.compile("(?i).*add\\s+file\\s+.+|(?i).*add\\s+jar\\s+.+"); private static String LOCAL_SQL_PLUGIN_ROOT; - private static List sqlParserList = Lists.newArrayList(CreateFuncParser.newInstance(), CreateTableParser.newInstance(), InsertSqlParser.newInstance(), CreateTmpTableParser.newInstance()); - public static void setLocalSqlPluginRoot(String localSqlPluginRoot){ + public static void setLocalSqlPluginRoot(String localSqlPluginRoot) { LOCAL_SQL_PLUGIN_ROOT = localSqlPluginRoot; } - private static final Pattern ADD_FILE_AND_JAR_PATTERN = Pattern.compile("(?i).*add\\s+file\\s+.+|(?i).*add\\s+jar\\s+.+"); - /** * flink support sql syntax * CREATE TABLE sls_stream() with (); * CREATE (TABLE|SCALA) FUNCTION fcnName WITH com.dtstack.com; * insert into tb1 select * from tb2; + * * @param sql */ public static SqlTree parseSql(String sql, String pluginLoadMode, String planner) throws Exception { - if(StringUtils.isBlank(sql)){ - throw new RuntimeException("sql is not null"); + if (StringUtils.isBlank(sql)) { + throw new IllegalArgumentException("SQL must be not empty!"); } sql = DtStringUtil.dealSqlComment(sql) @@ -80,12 +78,12 @@ public static SqlTree parseSql(String sql, String pluginLoadMode, String planner sqlArr = removeAddFileAndJarStmt(sqlArr); SqlTree sqlTree = new SqlTree(); AbstractTableInfoParser tableInfoParser = new AbstractTableInfoParser(); - for(String childSql : sqlArr){ - if(Strings.isNullOrEmpty(childSql)){ + for (String childSql : sqlArr) { + if (Strings.isNullOrEmpty(childSql)) { continue; } boolean result = false; - for(IParser sqlParser : sqlParserList){ + for (IParser sqlParser : sqlParserList) { try { if (!sqlParser.verify(childSql)) { continue; @@ -100,25 +98,25 @@ public static SqlTree parseSql(String sql, String pluginLoadMode, String planner } } - if(!result){ - throw new RuntimeException(String.format("%s:Syntax does not support,the format of SQL like insert into tb1 select * from tb2.", childSql)); + if (!result) { + throw new RuntimeException(String.format("%s \nSyntax like this does not support, the format of SQL like 'insert into tb1 select field from tb2'.", childSql)); } } //解析exec-sql - if(sqlTree.getExecSqlList().size() == 0){ + if (sqlTree.getExecSqlList().size() == 0) { throw new RuntimeException("sql no executable statement"); } - for(InsertSqlParser.SqlParseResult result : sqlTree.getExecSqlList()){ + for (InsertSqlParser.SqlParseResult result : sqlTree.getExecSqlList()) { List sourceTableList = result.getSourceTableList(); List targetTableList = result.getTargetTableList(); Set tmpTableList = sqlTree.getTmpTableMap().keySet(); - for(String tableName : sourceTableList){ - if (!tmpTableList.contains(tableName)){ + for (String tableName : sourceTableList) { + if (!tmpTableList.contains(tableName)) { CreateTableParser.SqlParserResult createTableResult = sqlTree.getPreDealTableMap().get(tableName); - if(createTableResult == null){ + if (createTableResult == null) { throw new RuntimeException("can't find table " + tableName); } @@ -128,10 +126,10 @@ public static SqlTree parseSql(String sql, String pluginLoadMode, String planner } } - for(String tableName : targetTableList){ - if (!tmpTableList.contains(tableName)){ + for (String tableName : targetTableList) { + if (!tmpTableList.contains(tableName)) { CreateTableParser.SqlParserResult createTableResult = sqlTree.getPreDealTableMap().get(tableName); - if(createTableResult == null){ + if (createTableResult == null) { throw new RuntimeException("can't find table " + tableName); } @@ -142,14 +140,14 @@ public static SqlTree parseSql(String sql, String pluginLoadMode, String planner } } - for (CreateTmpTableParser.SqlParserResult result : sqlTree.getTmpSqlList()){ + for (CreateTmpTableParser.SqlParserResult result : sqlTree.getTmpSqlList()) { List sourceTableList = result.getSourceTableList(); - for(String tableName : sourceTableList){ - if (!sqlTree.getTableInfoMap().keySet().contains(tableName)){ + for (String tableName : sourceTableList) { + if (!sqlTree.getTableInfoMap().containsKey(tableName)) { CreateTableParser.SqlParserResult createTableResult = sqlTree.getPreDealTableMap().get(tableName); - if(createTableResult == null){ + if (createTableResult == null) { CreateTmpTableParser.SqlParserResult tmpTableResult = sqlTree.getTmpTableMap().get(tableName); - if (tmpTableResult == null){ + if (tmpTableResult == null) { throw new RuntimeException("can't find table " + tableName); } } else { @@ -160,7 +158,6 @@ public static SqlTree parseSql(String sql, String pluginLoadMode, String planner } } } - return sqlTree; } @@ -171,7 +168,7 @@ private static List removeAddFileAndJarStmt(List stmts) { List cleanedStmts = Lists.newArrayList(); for (String stmt : stmts) { Matcher matcher = ADD_FILE_AND_JAR_PATTERN.matcher(stmt); - if(!matcher.matches()) { + if (!matcher.matches()) { cleanedStmts.add(stmt); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/resource/ResourceCheck.java b/core/src/main/java/com/dtstack/flink/sql/resource/ResourceCheck.java new file mode 100644 index 000000000..41436c0b3 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/resource/ResourceCheck.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.dtstack.flink.sql.resource; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +/** + * @author: chuixue + * @create: 2020-12-08 17:21 + * @description:资源检测 + **/ +public abstract class ResourceCheck { + public static Boolean NEED_CHECK = true; + public static String CHECK_STR = "checkResource"; + protected static Logger LOG = LoggerFactory.getLogger(ResourceCheck.class); + public String TABLE_TYPE_KEY = "tableType"; + public String SINK_STR = "sink"; + public String SIDE_STR = "side"; + + /** + * 资源可用性检测 + * + * @param checkProperties 校验资源可用性的参数配置 + */ + public abstract void checkResourceStatus(Map checkProperties); +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java index 366f0b1bc..ed0624db4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java @@ -45,6 +45,8 @@ public abstract class AbstractSideTableInfo extends AbstractTableInfo implements Serializable { + public static final String FAST_CHECK = "fastCheck"; + public static final String TARGET_SUFFIX = "Side"; public static final String CACHE_KEY = "cache"; @@ -92,6 +94,8 @@ public abstract class AbstractSideTableInfo extends AbstractTableInfo implements private List predicateInfoes = Lists.newArrayList(); + private boolean fastCheck; + public RowTypeInfo getRowTypeInfo(){ Class[] fieldClass = getFieldClasses(); TypeInformation[] types = new TypeInformation[fieldClass.length]; @@ -213,6 +217,15 @@ public Integer getConnectRetryMaxNum(Integer defaultValue) { public void setConnectRetryMaxNum(Integer connectRetryMaxNum) { this.connectRetryMaxNum = connectRetryMaxNum; } + + public boolean getFastCheck() { + return fastCheck; + } + + public void setFastCheck(boolean fastCheck) { + this.fastCheck = fastCheck; + } + @Override public String toString() { return "Cache Info{" + @@ -224,6 +237,7 @@ public String toString() { ", asyncPoolSize=" + asyncPoolSize + ", asyncFailMaxNum=" + asyncFailMaxNum + ", partitionedJoin=" + partitionedJoin + + ", fastCheck='" + fastCheck + ", cacheMode='" + cacheMode + '\'' + '}'; } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/BaseSideInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/BaseSideInfo.java index 8dbdaebc4..fcd8c032c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/BaseSideInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/BaseSideInfo.java @@ -135,7 +135,7 @@ public void parseSelectFields(JoinInfo joinInfo){ public String getTargetFieldType(String fieldName){ int fieldIndex = sideTableInfo.getFieldList().indexOf(fieldName); if(fieldIndex == -1){ - throw new RuntimeException(sideTableInfo.getName() + "can't find field: " + fieldName); + throw new RuntimeException(sideTableInfo.getName() + " can't find field: " + fieldName); } return sideTableInfo.getFieldTypes()[fieldIndex]; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/JoinNodeDealer.java b/core/src/main/java/com/dtstack/flink/sql/side/JoinNodeDealer.java index 99dcee0df..7fd24cabd 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/JoinNodeDealer.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/JoinNodeDealer.java @@ -551,16 +551,13 @@ private void extractJoinField(SqlNode condition, Set> joi if ( AGGREGATE.contains(condition.getKind()) || AVG_AGG_FUNCTIONS.contains(joinKind) || COMPARISON.contains(joinKind) - || joinKind == OTHER_FUNCTION - || joinKind == DIVIDE - || joinKind == CAST - || joinKind == TRIM - || joinKind == TIMES - || joinKind == PLUS + || FUNCTION.contains(joinKind) + || COVAR_AVG_AGG_FUNCTIONS.contains(joinKind) + || BINARY_ARITHMETIC.contains(joinKind) + || BINARY_COMPARISON.contains(joinKind) || joinKind == NOT_IN || joinKind == OR || joinKind == AND - || joinKind == MINUS || joinKind == TUMBLE || joinKind == TUMBLE_START || joinKind == TUMBLE_END @@ -578,7 +575,7 @@ private void extractJoinField(SqlNode condition, Set> joi || joinKind == TIMESTAMP_DIFF || joinKind == LIKE || joinKind == COALESCE - || joinKind == EQUALS ){ + ){ SqlBasicCall sqlBasicCall = (SqlBasicCall) condition; for(int i=0; i tableMap){ SqlKind joinKind = node.getKind(); - if( AGGREGATE.contains(joinKind) + if(AGGREGATE.contains(joinKind) || AVG_AGG_FUNCTIONS.contains(joinKind) || COMPARISON.contains(joinKind) - || joinKind == OTHER_FUNCTION - || joinKind == DIVIDE - || joinKind == CAST - || joinKind == TRIM - || joinKind == TIMES - || joinKind == PLUS + || FUNCTION.contains(joinKind) + || COVAR_AVG_AGG_FUNCTIONS.contains(joinKind) + || BINARY_ARITHMETIC.contains(joinKind) + || BINARY_COMPARISON.contains(joinKind) || joinKind == NOT_IN || joinKind == OR || joinKind == AND - || joinKind == MINUS || joinKind == TUMBLE || joinKind == TUMBLE_START || joinKind == TUMBLE_END @@ -935,8 +926,7 @@ private SqlIdentifier checkAndReplaceJoinCondition(SqlNode node, Map getProducedType() { return returnType; } -} +} \ No newline at end of file diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java index 5d4c2125d..a80226b86 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java @@ -20,10 +20,10 @@ package com.dtstack.flink.sql.side.operator; import com.dtstack.flink.sql.classloader.ClassLoaderManager; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.BaseAsyncReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.AsyncDataStream; @@ -49,8 +49,13 @@ public class SideAsyncOperator { private static final String ORDERED = "ordered"; - private static BaseAsyncReqRow loadAsyncReq(String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, - List outFieldInfoList, AbstractSideTableInfo sideTableInfo, String pluginLoadMode) throws Exception { + private static BaseAsyncReqRow loadAsyncReq(String sideType + , String sqlRootDir + , RowTypeInfo rowTypeInfo + , JoinInfo joinInfo + , List outFieldInfoList + , AbstractSideTableInfo sideTableInfo + , String pluginLoadMode) throws Exception { String pathOfType = String.format(PATH_FORMAT, sideType); String pluginJarPath = PluginUtil.getJarFileDirPath(pathOfType, sqlRootDir, pluginLoadMode); String className = PluginUtil.getSqlSideClassName(sideType, "side", OPERATOR_TYPE); @@ -60,8 +65,14 @@ private static BaseAsyncReqRow loadAsyncReq(String sideType, String sqlRootDir, .newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } - public static DataStream getSideJoinDataStream(DataStream inputStream, String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, - List outFieldInfoList, AbstractSideTableInfo sideTableInfo, String pluginLoadMode) throws Exception { + public static DataStream getSideJoinDataStream(DataStream inputStream + , String sideType + , String sqlRootDir + , RowTypeInfo rowTypeInfo + , JoinInfo joinInfo + , List outFieldInfoList + , AbstractSideTableInfo sideTableInfo + , String pluginLoadMode) throws Exception { BaseAsyncReqRow asyncDbReq = loadAsyncReq(sideType, sqlRootDir, rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo, pluginLoadMode); //TODO How much should be set for the degree of parallelism? Timeout? capacity settings? diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java index 18db27297..94901d8f9 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java @@ -60,6 +60,6 @@ private static BaseAllReqRow loadFlatMap(String sideType, String sqlRootDir, Row public static DataStream getSideJoinDataStream(DataStream inputStream, String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo, String pluginLoadMode) throws Exception { BaseAllReqRow allReqRow = loadFlatMap(sideType, sqlRootDir, rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo, pluginLoadMode); - return inputStream.flatMap(allReqRow); + return inputStream.flatMap(allReqRow).setParallelism(sideTableInfo.getParallelism()); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableInfo.java index f8fa9c3ef..d81c60f88 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableInfo.java @@ -24,47 +24,50 @@ import java.io.Serializable; import java.util.List; import java.util.Map; +import java.util.Properties; /** * Reason: * Date: 2018/6/22 * Company: www.dtstack.com + * * @author xuchao */ public abstract class AbstractTableInfo implements Serializable { public static final String PARALLELISM_KEY = "parallelism"; - + private final List fieldList = Lists.newArrayList(); + private final List fieldTypeList = Lists.newArrayList(); + private final List fieldClassList = Lists.newArrayList(); + private final List fieldExtraInfoList = Lists.newArrayList(); private String name; - private String type; - private String[] fields; - private String[] fieldTypes; - private Class[] fieldClasses; - - private final List fieldList = Lists.newArrayList(); - - /**key:别名, value: realField */ + /** + * key:别名, value: realField + */ private Map physicalFields = Maps.newLinkedHashMap(); - - private final List fieldTypeList = Lists.newArrayList(); - - private final List fieldClassList = Lists.newArrayList(); - - private final List fieldExtraInfoList = Lists.newArrayList(); - private List primaryKeys; private Integer parallelism = -1; + private Map checkProperties; + + /** + * 构建脏数据插件的相关信息 + */ + private Properties dirtyProperties; public String[] getFieldTypes() { return fieldTypes; } + public void setFieldTypes(String[] fieldTypes) { + this.fieldTypes = fieldTypes; + } + public abstract boolean check(); public String getType() { @@ -79,10 +82,18 @@ public String[] getFields() { return fields; } + public void setFields(String[] fields) { + this.fields = fields; + } + public Class[] getFieldClasses() { return fieldClasses; } + public void setFieldClasses(Class[] fieldClasses) { + this.fieldClasses = fieldClasses; + } + public List getPrimaryKeys() { return primaryKeys; } @@ -104,18 +115,18 @@ public Integer getParallelism() { } public void setParallelism(Integer parallelism) { - if(parallelism == null){ + if (parallelism == null) { return; } - if(parallelism <= 0){ + if (parallelism <= 0) { throw new RuntimeException("Abnormal parameter settings: parallelism > 0"); } this.parallelism = parallelism; } - public void addField(String fieldName){ + public void addField(String fieldName) { if (fieldList.contains(fieldName)) { throw new RuntimeException("redundancy field name " + fieldName + " in table " + getName()); } @@ -123,30 +134,18 @@ public void addField(String fieldName){ fieldList.add(fieldName); } - public void addPhysicalMappings(String aliasName, String physicalFieldName){ + public void addPhysicalMappings(String aliasName, String physicalFieldName) { physicalFields.put(aliasName, physicalFieldName); } - public void addFieldClass(Class fieldClass){ + public void addFieldClass(Class fieldClass) { fieldClassList.add(fieldClass); } - public void addFieldType(String fieldType){ + public void addFieldType(String fieldType) { fieldTypeList.add(fieldType); } - public void setFields(String[] fields) { - this.fields = fields; - } - - public void setFieldTypes(String[] fieldTypes) { - this.fieldTypes = fieldTypes; - } - - public void setFieldClasses(Class[] fieldClasses) { - this.fieldClasses = fieldClasses; - } - public void addFieldExtraInfo(FieldExtraInfo extraInfo) { fieldExtraInfoList.add(extraInfo); } @@ -167,15 +166,36 @@ public Map getPhysicalFields() { return physicalFields; } + public void setPhysicalFields(Map physicalFields) { + this.physicalFields = physicalFields; + } + public List getFieldExtraInfoList() { return fieldExtraInfoList; } - public void setPhysicalFields(Map physicalFields) { - this.physicalFields = physicalFields; + public Map getCheckProperties() { + return checkProperties; + } + + public void setCheckProperties() { + this.checkProperties = buildCheckProperties(); + } + + public Map buildCheckProperties() { + return Maps.newHashMap(); + } + + public Properties getDirtyProperties() { + dirtyProperties.setProperty("tableName", this.name); + return dirtyProperties; + } + + public void setDirtyProperties(Properties dirtyProperties) { + this.dirtyProperties = dirtyProperties; } - public void finish(){ + public void finish() { this.fields = fieldList.toArray(new String[0]); this.fieldClasses = fieldClassList.toArray(new Class[0]); this.fieldTypes = fieldTypeList.toArray(new String[0]); @@ -183,7 +203,7 @@ public void finish(){ /** * field extra info,used to store `not null` `default 0`..., - * + *

* now, only support not null */ public static class FieldExtraInfo implements Serializable { @@ -193,7 +213,7 @@ public static class FieldExtraInfo implements Serializable { */ boolean notNull = false; /** - * field length,eg.char(4) + * field length,eg.char(4) */ int length; diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableInfoParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableInfoParser.java index e146d5b85..887bb8c64 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableInfoParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableInfoParser.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package com.dtstack.flink.sql.table; @@ -39,6 +38,7 @@ * Create table statement parsing table structure to obtain specific information * Date: 2018/6/25 * Company: www.dtstack.com + * * @author xuchao */ @@ -50,85 +50,80 @@ public class AbstractTableInfoParser { private final static Pattern SIDE_PATTERN = Pattern.compile(SIDE_TABLE_SIGN); - private Map sourceTableInfoMap = Maps.newConcurrentMap(); + private Map sourceTableInfoMap = Maps.newConcurrentMap(); - private Map targetTableInfoMap = Maps.newConcurrentMap(); + private Map targetTableInfoMap = Maps.newConcurrentMap(); - private Map sideTableInfoMap = Maps.newConcurrentMap(); + private Map sideTableInfoMap = Maps.newConcurrentMap(); /** - * 解析sql生产对应的实体对象 + * judge dim table of PERIOD FOR SYSTEM_TIME * - * @param tableType 插件类型 - * @param parserResult 表信息 - * @param localPluginRoot 插件路径 - * @param pluginLoadMode 插件模式 - * @param planner dtstack使用DtClassloader,flink使用AppClassLoader + * @param tableField * @return - * @throws Exception */ - public AbstractTableInfo parseWithTableType(int tableType, CreateTableParser.SqlParserResult parserResult, - String localPluginRoot, String pluginLoadMode, String planner) throws Exception { + private static boolean checkIsSideTable(String tableField) { + String[] fieldInfos = StringUtils.split(tableField, ","); + for (String field : fieldInfos) { + Matcher matcher = SIDE_PATTERN.matcher(field.trim()); + if (matcher.find()) { + return true; + } + } + + return false; + } + + //Parsing loaded plugin + public AbstractTableInfo parseWithTableType( + int tableType + , CreateTableParser.SqlParserResult parserResult + , String localPluginRoot + , String pluginLoadMode + , String planner + ) throws Exception { AbstractTableParser absTableParser = null; Map props = parserResult.getPropMap(); String type = MathUtil.getString(props.get(TYPE_KEY)); - if(Strings.isNullOrEmpty(type)){ + if (Strings.isNullOrEmpty(type)) { throw new RuntimeException("create table statement requires property of type"); } - if(tableType == ETableType.SOURCE.getType()){ + if (tableType == ETableType.SOURCE.getType()) { boolean isSideTable = checkIsSideTable(parserResult.getFieldsInfoStr()); - if(!isSideTable){ + if (!isSideTable) { absTableParser = sourceTableInfoMap.get(type); - if(absTableParser == null){ + if (absTableParser == null) { absTableParser = StreamSourceFactory.getSqlParser(type, localPluginRoot, pluginLoadMode); sourceTableInfoMap.put(type, absTableParser); } - }else{ + } else { absTableParser = sideTableInfoMap.get(type); - if(absTableParser == null){ + if (absTableParser == null) { String cacheType = MathUtil.getString(props.get(AbstractSideTableInfo.CACHE_KEY)); absTableParser = StreamSideFactory.getSqlParser(type, localPluginRoot, cacheType, pluginLoadMode, planner); sideTableInfoMap.put(type + cacheType, absTableParser); } } - }else if(tableType == ETableType.SINK.getType()){ + } else if (tableType == ETableType.SINK.getType()) { absTableParser = targetTableInfoMap.get(type); - if(absTableParser == null){ + if (absTableParser == null) { absTableParser = StreamSinkFactory.getSqlParser(type, localPluginRoot, pluginLoadMode); targetTableInfoMap.put(type, absTableParser); } } - if(absTableParser == null){ + if (absTableParser == null) { throw new RuntimeException(String.format("not support %s type of table", type)); } Map prop = Maps.newHashMap(); //Shield case - parserResult.getPropMap().forEach((key,val) -> prop.put(key.toLowerCase(), val)); - + parserResult.getPropMap().forEach((key, val) -> prop.put(key.toLowerCase(), val)); return absTableParser.getTableInfo(parserResult.getTableName(), parserResult.getFieldsInfoStr(), prop); } - - /** - * judge dim table of PERIOD FOR SYSTEM_TIME - * @param tableField - * @return - */ - private static boolean checkIsSideTable(String tableField){ - String[] fieldInfos = StringUtils.split(tableField, ","); - for(String field : fieldInfos){ - Matcher matcher = SIDE_PATTERN.matcher(field.trim()); - if(matcher.find()){ - return true; - } - } - - return false; - } } diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableParser.java index 1e7f72de6..1e30095b0 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableParser.java @@ -17,9 +17,9 @@ */ - package com.dtstack.flink.sql.table; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.util.ClassUtil; import com.dtstack.flink.sql.util.DtStringUtil; import com.google.common.base.Preconditions; @@ -38,6 +38,7 @@ * Reason: * Date: 2018/7/4 * Company: www.dtstack.com + * * @author xuchao */ @@ -47,11 +48,11 @@ public abstract class AbstractTableParser { private static final String NEST_JSON_FIELD_KEY = "nestFieldKey"; private static final String CHAR_TYPE_NO_LENGTH = "CHAR"; - private static Pattern primaryKeyPattern = Pattern.compile("(?i)(^\\s*)PRIMARY\\s+KEY\\s*\\((.*)\\)"); - private static Pattern nestJsonFieldKeyPattern = Pattern.compile("(?i)((@*\\S+\\.)*\\S+)\\s+(.+?)\\s+AS\\s+(\\w+)(\\s+NOT\\s+NULL)?$"); - private static Pattern physicalFieldFunPattern = Pattern.compile("\\w+\\((\\w+)\\)$"); - private static Pattern charTypePattern = Pattern.compile("(?i)CHAR\\((\\d*)\\)$"); - private static Pattern typePattern = Pattern.compile("(\\S+)\\s+(\\w+.*)"); + private static final Pattern primaryKeyPattern = Pattern.compile("(?i)(^\\s*)PRIMARY\\s+KEY\\s*\\((.*)\\)"); + private static final Pattern nestJsonFieldKeyPattern = Pattern.compile("(?i)((@*\\S+\\.)*\\S+)\\s+(.+?)\\s+AS\\s+(\\w+)(\\s+NOT\\s+NULL)?$"); + private static final Pattern physicalFieldFunPattern = Pattern.compile("\\w+\\((\\w+)\\)$"); + private static final Pattern charTypePattern = Pattern.compile("(?i)CHAR\\((\\d*)\\)$"); + private static final Pattern typePattern = Pattern.compile("(\\S+)\\s+(\\w+.*)"); private Map patternMap = Maps.newHashMap(); @@ -69,14 +70,14 @@ protected boolean fieldNameNeedsUpperCase() { public abstract AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) throws Exception; - public boolean dealKeyPattern(String fieldRow, AbstractTableInfo tableInfo){ - for(Map.Entry keyPattern : patternMap.entrySet()){ + public boolean dealKeyPattern(String fieldRow, AbstractTableInfo tableInfo) { + for (Map.Entry keyPattern : patternMap.entrySet()) { Pattern pattern = keyPattern.getValue(); String key = keyPattern.getKey(); Matcher matcher = pattern.matcher(fieldRow); - if(matcher.find()){ + if (matcher.find()) { ITableFieldDealHandler handler = handlerMap.get(key); - if(handler == null){ + if (handler == null) { throw new RuntimeException("parse field [" + fieldRow + "] error."); } @@ -92,11 +93,12 @@ public void parseFieldsInfo(String fieldsInfo, AbstractTableInfo tableInfo) { List fieldRows = DtStringUtil.splitField(fieldsInfo); - for (String fieldRow : fieldRows) { - fieldRow = fieldRow.trim(); + for (int i = 0; i < fieldRows.size(); i++) { + String fieldRow = fieldRows.get(i).trim(); if (StringUtils.isBlank(fieldRow)) { - throw new RuntimeException(String.format("table [%s],exists field empty.", tableInfo.getName())); + throw new RuntimeException(String.format("Empty field appears in position [%s] in table [%s]", + i + 1, tableInfo.getName())); } boolean isMatcherKey = dealKeyPattern(fieldRow, tableInfo); @@ -104,30 +106,44 @@ public void parseFieldsInfo(String fieldsInfo, AbstractTableInfo tableInfo) { continue; } - Tuple2 t = extractType(fieldRow, tableInfo.getName()); - String fieldName = t.f0; - String fieldType = t.f1; + handleKeyNotHaveAlias(fieldRow, tableInfo); + } - Class fieldClass; - AbstractTableInfo.FieldExtraInfo fieldExtraInfo = null; + /* + * check whether filed list contains pks and then add pks into field list. + * because some no-sql database is not primary key. eg :redis、hbase etc... + */ + if (tableInfo instanceof AbstractSideTableInfo) { + tableInfo.getPrimaryKeys().stream() + .filter(pk -> (!tableInfo.getFieldList().contains(pk) && pk.equals("rowkey"))) + .forEach(pk -> handleKeyNotHaveAlias(String.format("%s varchar", pk), tableInfo)); + } - Matcher matcher = charTypePattern.matcher(fieldType); - if (matcher.find()) { - fieldClass = dbTypeConvertToJavaType(CHAR_TYPE_NO_LENGTH); - fieldExtraInfo = new AbstractTableInfo.FieldExtraInfo(); - fieldExtraInfo.setLength(Integer.valueOf(matcher.group(1))); - } else { - fieldClass = dbTypeConvertToJavaType(fieldType); - } + tableInfo.finish(); + } + + private void handleKeyNotHaveAlias(String fieldRow, AbstractTableInfo tableInfo) { + Tuple2 t = extractType(fieldRow, tableInfo.getName()); + String fieldName = t.f0; + String fieldType = t.f1; - tableInfo.addPhysicalMappings(fieldName, fieldName); - tableInfo.addField(fieldName); - tableInfo.addFieldClass(fieldClass); - tableInfo.addFieldType(fieldType); - tableInfo.addFieldExtraInfo(fieldExtraInfo); + Class fieldClass; + AbstractTableInfo.FieldExtraInfo fieldExtraInfo = null; + + Matcher matcher = charTypePattern.matcher(fieldType); + if (matcher.find()) { + fieldClass = dbTypeConvertToJavaType(CHAR_TYPE_NO_LENGTH); + fieldExtraInfo = new AbstractTableInfo.FieldExtraInfo(); + fieldExtraInfo.setLength(Integer.parseInt(matcher.group(1))); + } else { + fieldClass = dbTypeConvertToJavaType(fieldType); } - tableInfo.finish(); + tableInfo.addPhysicalMappings(fieldName, fieldName); + tableInfo.addField(fieldName); + tableInfo.addFieldClass(fieldClass); + tableInfo.addFieldType(fieldType); + tableInfo.addFieldExtraInfo(fieldExtraInfo); } private Tuple2 extractType(String fieldRow, String tableName) { @@ -147,12 +163,13 @@ public void dealPrimaryKey(Matcher matcher, AbstractTableInfo tableInfo) { List primaryKeys = Arrays .stream(primaryFields.split(",")) .map(String::trim) - .collect(Collectors.toList());; + .collect(Collectors.toList()); tableInfo.setPrimaryKeys(primaryKeys); } /** * add parser for alias field + * * @param matcher * @param tableInfo */ diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbstractTargetTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTargetTableInfo.java index dbd99f993..48de59227 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbstractTargetTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTargetTableInfo.java @@ -31,12 +31,16 @@ public abstract class AbstractTargetTableInfo extends AbstractTableInfo { + public static final String FAST_CHECK = "fastCheck"; + public static final String TARGET_SUFFIX = "Sink"; public static final String SINK_DATA_TYPE = "sinkdatatype"; private String sinkDataType = FormatType.JSON.name(); + private boolean fastCheck; + public String getSinkDataType() { return sinkDataType; } @@ -44,4 +48,13 @@ public String getSinkDataType() { public void setSinkDataType(String sinkDataType) { this.sinkDataType = sinkDataType; } + + public boolean getFastCheck() { + return fastCheck; + } + + public void setFastCheck(boolean fastCheck) { + this.fastCheck = fastCheck; + } + } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java index ec88bdbe9..b7d2f12fc 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java @@ -25,6 +25,8 @@ import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; +import java.util.HashMap; +import java.util.Map; /** * Reason: TODO ADD REASON(可选) @@ -41,6 +43,10 @@ public static Class stringConvertClass(String str) { if (lowerStr.startsWith("array")) { return Array.newInstance(Integer.class, 0).getClass(); } + if (lowerStr.startsWith("map")) { + Map m = new HashMap(); + return m.getClass(); + } switch (lowerStr) { case "boolean": diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DataTypeUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/DataTypeUtils.java index ff7512328..0dc3cc23b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DataTypeUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DataTypeUtils.java @@ -21,6 +21,7 @@ import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -46,6 +47,7 @@ import static org.apache.flink.table.api.DataTypes.DECIMAL; import static org.apache.flink.table.api.DataTypes.TIMESTAMP; + /** * @program: flink.sql * @author: wuren @@ -55,6 +57,7 @@ public class DataTypeUtils { private final static Pattern COMPOSITE_TYPE_PATTERN = Pattern.compile("(.+?)<(.+)>"); private final static String ARRAY = "ARRAY"; + private final static String MAP = "MAP"; private final static String ROW = "ROW"; private final static char FIELD_DELIMITER = ','; private final static char TYPE_DELIMITER = ' '; @@ -62,23 +65,6 @@ public class DataTypeUtils { private DataTypeUtils() { } - /** - * 现在只支持ARRAY类型后续可以加入 MAP等类型 - * - * @param compositeTypeString - * @return - */ - public static TypeInformation convertToCompositeType(String compositeTypeString) { - Matcher matcher = matchCompositeType(compositeTypeString); - final String errorMsg = "type " + compositeTypeString + "is not support!"; - Preconditions.checkState(matcher.find(), errorMsg); - - String normalizedType = normalizeType(matcher.group(1)); - Preconditions.checkState(ARRAY.equals(normalizedType), errorMsg); - - return convertToArray(compositeTypeString); - } - /** * 目前ARRAY里只支持ROW和其他基本类型 * @@ -105,6 +91,30 @@ public static TypeInformation convertToArray(String arrayTypeString) { return Types.OBJECT_ARRAY(elementType); } + /** + * 目前Map里只支持基本类型 + * @param mapTypeString + * @return + */ + public static TypeInformation convertToMap(String mapTypeString) { + Matcher matcher = matchCompositeType(mapTypeString); + final String errorMsg = mapTypeString + "convert to map type error!"; + Preconditions.checkState(matcher.find(), errorMsg); + + String normalizedType = normalizeType(matcher.group(1)); + Preconditions.checkState(MAP.equals(normalizedType), errorMsg); + + String kvTypeString = matcher.group(2); + String[] kvTypeStringList = StringUtils.split(kvTypeString, ","); + final String mapTypeErrorMsg = "There can only be key and value two types in map declaration."; + Preconditions.checkState(kvTypeStringList.length == 2, mapTypeErrorMsg); + String keyTypeString = normalizeType(kvTypeStringList[0]); + String valueTypeString = normalizeType(kvTypeStringList[1]); + TypeInformation keyType = convertToAtomicType(keyTypeString); + TypeInformation valueType = convertToAtomicType(valueTypeString); + return Types.MAP(keyType, valueType); + } + /** * 目前ROW里只支持基本类型 * @@ -158,6 +168,7 @@ public static TypeInformation getRowTypeInformation(String[] fieldTypes, Cl return new RowTypeInfo(types, fieldTypes); } + private static Tuple2 genFieldInfo(Iterable fieldInfoStrs) { ArrayList types = Lists.newArrayList(); ArrayList fieldNames = Lists.newArrayList(); diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java index 775e0a985..3856d777e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java @@ -243,19 +243,16 @@ public static SqlNode replaceSelectFieldName(SqlNode selectNode, return createNewIdentify((SqlIdentifier) selectNode, oldTbName, newTbName, mappingField); }else if(selectNode.getKind() == LITERAL || selectNode.getKind() == LITERAL_CHAIN){//字面含义 return selectNode; - }else if( AGGREGATE.contains(selectNode.getKind()) + }else if(AGGREGATE.contains(selectNode.getKind()) || AVG_AGG_FUNCTIONS.contains(selectNode.getKind()) || COMPARISON.contains(selectNode.getKind()) - || selectNode.getKind() == OTHER_FUNCTION - || selectNode.getKind() == DIVIDE - || selectNode.getKind() == CAST - || selectNode.getKind() == TRIM - || selectNode.getKind() == TIMES - || selectNode.getKind() == PLUS + || FUNCTION.contains(selectNode.getKind()) + || COVAR_AVG_AGG_FUNCTIONS.contains(selectNode.getKind()) + || BINARY_ARITHMETIC.contains(selectNode.getKind()) + || BINARY_COMPARISON.contains(selectNode.getKind()) || selectNode.getKind() == NOT_IN || selectNode.getKind() == OR || selectNode.getKind() == AND - || selectNode.getKind() == MINUS || selectNode.getKind() == TUMBLE || selectNode.getKind() == TUMBLE_START || selectNode.getKind() == TUMBLE_END @@ -273,7 +270,6 @@ public static SqlNode replaceSelectFieldName(SqlNode selectNode, || selectNode.getKind() == TIMESTAMP_DIFF || selectNode.getKind() == LIKE || selectNode.getKind() == COALESCE - ){ SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; for(int i=0; i logger.info("sampling source input data: " + message); + samplingPrint(samplingIntervalCount, count, func); + } + + public static void samplingSinkPrint(int samplingIntervalCount, Logger logger, long count, String message) { + Runnable func = () -> logger.info("sampling sink output data: " + message); + samplingPrint(samplingIntervalCount, count, func); + } + + private static void samplingPrint(int samplingIntervalCount, long count, Runnable func) { + if (samplingIntervalCount > 0 && count % samplingIntervalCount == 0) { + func.run(); + } + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ThreadUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/ThreadUtil.java new file mode 100644 index 000000000..d6f57788e --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/util/ThreadUtil.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flink.sql.util; + +import java.util.concurrent.TimeUnit; + +/** + * @author tiezhu + * Date 2020-12-25 + * Company dtstack + */ +public class ThreadUtil { + public static final Long DEFAULT_SLEEP_TIME = 10L; + public static void sleepSeconds(long timeout) { + try { + TimeUnit.SECONDS.sleep(timeout); + } catch (InterruptedException ie) { + throw new RuntimeException(ie); + } + } + + public static void sleepMinutes(long timeout) { + try { + TimeUnit.MINUTES.sleep(timeout); + } catch (InterruptedException ie) { + throw new RuntimeException(ie); + } + } + + public static void sleepMicroseconds(long timeout) { + try { + TimeUnit.MICROSECONDS.sleep(timeout); + } catch (InterruptedException ie) { + throw new RuntimeException(ie); + } + } +} diff --git a/core/src/test/java/com/dtstack/flink/sql/dirtyManager/TestDirtyDataManager.java b/core/src/test/java/com/dtstack/flink/sql/dirtyManager/TestDirtyDataManager.java new file mode 100644 index 000000000..ec08bb9d3 --- /dev/null +++ b/core/src/test/java/com/dtstack/flink/sql/dirtyManager/TestDirtyDataManager.java @@ -0,0 +1,15 @@ +package com.dtstack.flink.sql.dirtyManager; + +import com.dtstack.flink.sql.dirtyManager.manager.DirtyDataManager; + +import java.util.HashMap; +import java.util.Map; + +/** + * @author tiezhu + * Company dtstack + * Date 2020/8/28 星期五 + */ +public class TestDirtyDataManager { + +} diff --git a/db2/db2-side/db2-side-core/src/main/java/com/dtstack/flink/sql/side/db2/table/Db2SideParser.java b/db2/db2-side/db2-side-core/src/main/java/com/dtstack/flink/sql/side/db2/table/Db2SideParser.java index 96be4ec15..bc12e9a38 100644 --- a/db2/db2-side/db2-side-core/src/main/java/com/dtstack/flink/sql/side/db2/table/Db2SideParser.java +++ b/db2/db2-side/db2-side-core/src/main/java/com/dtstack/flink/sql/side/db2/table/Db2SideParser.java @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.side.db2.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -37,6 +38,7 @@ public class Db2SideParser extends RdbSideParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "com.ibm.db2.jcc.DB2Driver"); AbstractTableInfo tableInfo = super.getTableInfo(tableName, fieldsInfo, props); tableInfo.setType(CURR_TYPE); diff --git a/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/table/DbSinkParser.java b/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/table/DbSinkParser.java index 3e823ab20..8a9ad62ab 100644 --- a/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/table/DbSinkParser.java +++ b/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/table/DbSinkParser.java @@ -1,5 +1,6 @@ package com.dtstack.flink.sql.sink.db.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -11,6 +12,7 @@ public class DbSinkParser extends RdbSinkParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "com.ibm.db2.jcc.DB2Driver"); AbstractTableInfo tableInfo = super.getTableInfo(tableName, fieldsInfo, props); tableInfo.setType(CURR_TYPE); return tableInfo; diff --git a/dirtyData/console/pom.xml b/dirtyData/console/pom.xml new file mode 100644 index 000000000..d65d0284c --- /dev/null +++ b/dirtyData/console/pom.xml @@ -0,0 +1,91 @@ + + + + sql.dirtyConsumer + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + dirtyConsumer-console + dirtyConsumer.console + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + false + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + + junit + junit + 4.8.2 + test + + + + log4j + log4j + 1.2.17 + + + + \ No newline at end of file diff --git a/dirtyData/console/src/main/java/com/dtstack/flink/sql/dirty/console/ConsoleDirtyDataConsumer.java b/dirtyData/console/src/main/java/com/dtstack/flink/sql/dirty/console/ConsoleDirtyDataConsumer.java new file mode 100644 index 000000000..3e6e2412e --- /dev/null +++ b/dirtyData/console/src/main/java/com/dtstack/flink/sql/dirty/console/ConsoleDirtyDataConsumer.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flink.sql.dirty.console; + +import com.dtstack.flink.sql.dirtyManager.consumer.AbstractDirtyDataConsumer; +import com.dtstack.flink.sql.dirtyManager.entity.DirtyDataEntity; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; +import java.util.Properties; + +/** + * @author tiezhu + * Company dtstack + * Date 2020/8/27 星期四 + */ +public class ConsoleDirtyDataConsumer extends AbstractDirtyDataConsumer { + private static final long serialVersionUID = 5727194679865135189L; + + private static final Logger LOG = LoggerFactory.getLogger(ConsoleDirtyDataConsumer.class); + + private static final Long DEFAULT_PRINT_LIMIT = 1000L; + + private Long printLimit; + + @Override + public void consume() throws InterruptedException { + DirtyDataEntity dataEntity = queue.take(); + count.incrementAndGet(); + if (count.get() % printLimit == 0) { + LOG.warn("\nget dirtyData: " + dataEntity.getDirtyData() + "\n" + + "cause: " + dataEntity.getCause() + "\n" + + "processTime: " + dataEntity.getProcessDate() + "\n" + + "error field: " + dataEntity.getField()); + } + } + + @Override + public void close() { + isRunning.compareAndSet(true, false); + LOG.info("console dirty consumer close ..."); + } + + @Override + public void init(Properties properties) { + LOG.info("console dirty consumer init ..."); + Object printLimit = properties.get("printLimit"); + this.printLimit = Objects.isNull(printLimit) ? + DEFAULT_PRINT_LIMIT : Long.parseLong(String.valueOf(printLimit)); + } +} diff --git a/dirtyData/console/src/test/java/com/dtstack/flink/sql/dirty/console/TestPrintDirtyDataConsumer.java b/dirtyData/console/src/test/java/com/dtstack/flink/sql/dirty/console/TestPrintDirtyDataConsumer.java new file mode 100644 index 000000000..3d1d64c41 --- /dev/null +++ b/dirtyData/console/src/test/java/com/dtstack/flink/sql/dirty/console/TestPrintDirtyDataConsumer.java @@ -0,0 +1,12 @@ +package com.dtstack.flink.sql.dirty.console; + +/** + * @author tiezhu + * Company dtstack + * Date 2020/8/28 星期五 + */ +public class TestPrintDirtyDataConsumer { + public static void main(String[] args) { + + } +} diff --git a/dirtyData/mysql/pom.xml b/dirtyData/mysql/pom.xml new file mode 100644 index 000000000..07b6b2699 --- /dev/null +++ b/dirtyData/mysql/pom.xml @@ -0,0 +1,89 @@ + + + + sql.dirtyConsumer + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + dirtyConsumer-mysql + + 5.1.46 + + + dirtyConsumer.mysql + + + + mysql + mysql-connector-java + ${mysql.connector.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + false + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/dirtyData/mysql/src/main/java/com/dtstack/flink/sql/dirty/mysql/MysqlDirtyDataConsumer.java b/dirtyData/mysql/src/main/java/com/dtstack/flink/sql/dirty/mysql/MysqlDirtyDataConsumer.java new file mode 100644 index 000000000..8d425e7e1 --- /dev/null +++ b/dirtyData/mysql/src/main/java/com/dtstack/flink/sql/dirty/mysql/MysqlDirtyDataConsumer.java @@ -0,0 +1,174 @@ +/* + * 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.dtstack.flink.sql.dirty.mysql; + +import com.dtstack.flink.sql.dirtyManager.consumer.AbstractDirtyDataConsumer; +import com.dtstack.flink.sql.dirtyManager.entity.DirtyDataEntity; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.sql.Statement; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Properties; +import java.util.stream.Collectors; + +/** + * @author tiezhu + * Company dtstack + * Date 2020/8/27 星期四 + */ +public class MysqlDirtyDataConsumer extends AbstractDirtyDataConsumer { + + private static final long serialVersionUID = -2959753658786001679L; + + private static final String DRIVER_NAME = "com.mysql.jdbc.Driver"; + + private static final int CONN_VALID_TIME = 1000; + + private static final Integer FIELD_NUMBER = 4; + + private final Object LOCK_STR = new Object(); + + private final String[] tableField = {"id", "dirtyData", "processTime", "cause"}; + + private PreparedStatement statement; + + private Connection connection; + + private Long batchSize; + + private void beforeConsume(String url, + String userName, + String password, + String tableName, + boolean isCreatedTable) throws ClassNotFoundException, SQLException { + synchronized (LOCK_STR) { + Class.forName(DRIVER_NAME); + connection = DriverManager.getConnection(url, userName, password); + + // create table for dirty data + if (!isCreatedTable) { + createTable(tableName); + } + + String insertField = Arrays.stream(tableField) + .map(this::quoteIdentifier) + .collect(Collectors.joining(", ")); + String insertSql = "INSERT INTO " + quoteIdentifier(tableName) + + "(" + insertField + ") VALUES (?, ?, ?, ?)"; + statement = connection.prepareStatement(insertSql); + } + } + + private String quoteIdentifier(String tableName) { + return "`" + tableName + "`"; + } + + /** + * 创建存储脏数据的表 + * + * @param tableName 表名 + * @throws SQLException SQL异常 + */ + private void createTable(String tableName) throws SQLException { + Statement statement = null; + try { + String sql = + "CREATE TABLE IF NOT EXISTS \n" + + quoteIdentifier(tableName) + " (\n" + + " `id` bigint not null AUTO_INCREMENT,\n" + + " `dirtyData` text DEFAULT NULL,\n" + + " `processTime` varchar(255) DEFAULT NULL,\n" + + " `cause` text DEFAULT NULL,\n" + + " PRIMARY KEY (id)\n" + + ") DEFAULT CHARSET=utf8;"; + statement = connection.createStatement(); + statement.execute(sql); + } catch (SQLException e) { + throw new RuntimeException("create table error !", e); + } finally { + if (statement != null && !statement.isClosed()) { + statement.close(); + } + } + } + + @Override + public void consume() throws Exception { + DirtyDataEntity entity = queue.take(); + count.incrementAndGet(); + + List data = new ArrayList<>(); + data.add(String.valueOf(count.get())); + Collections.addAll(data, entity.get()); + for (int i = 0; i < FIELD_NUMBER; i++) { + statement.setString(i + 1, Objects.isNull(data.get(i)) ? null : data.get(i)); + } + + statement.addBatch(); + + if (count.get() % batchSize == 0) { + statement.executeBatch(); + } + } + + @Override + public void close() { + isRunning.compareAndSet(true, false); + + try { + if (connection != null && !connection.isValid(CONN_VALID_TIME)) { + connection.close(); + } + + if (statement != null && !statement.isClosed()) { + statement.close(); + } + } catch (SQLException e) { + throw new RuntimeException("close mysql resource error !"); + } + } + + @Override + public void init(Properties properties) throws Exception { + SimpleDateFormat timeFormat = new SimpleDateFormat("yyyy_MM_dd_HH_mm_ss"); + String tableName = (String) properties.getOrDefault("dirtyTableName", + "DirtyData_" + + properties.getProperty("tableName") + "_" + + timeFormat.format(System.currentTimeMillis())); + + String userName = (String) properties.get("userName"); + String password = (String) properties.get("password"); + String url = (String) properties.get("url"); + batchSize = Long.parseLong((String) properties.getOrDefault("batchSize", "10000")); + errorLimit = Long.parseLong((String) properties.getOrDefault("errorLimit", "1000")); + + boolean isCreatedTable = Boolean.parseBoolean( + (String) properties.getOrDefault("isCreatedTable", "false")); + + beforeConsume(url, userName, password, tableName, isCreatedTable); + } +} diff --git a/dirtyData/pom.xml b/dirtyData/pom.xml new file mode 100644 index 000000000..c077855f2 --- /dev/null +++ b/dirtyData/pom.xml @@ -0,0 +1,40 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.dirtyConsumer + pom + + console + mysql + + + + 3.8.1 + 1.0-SNAPSHOT + + + + + junit + junit + ${junit.version} + test + + + + com.dtstack.flink + sql.core + ${sql.core.version} + provided + + + + \ No newline at end of file diff --git a/docs/config.md b/docs/config.md index 465893d16..f618f86c8 100644 --- a/docs/config.md +++ b/docs/config.md @@ -75,11 +75,11 @@ sh submit.sh -key1 val1 -key2 val2 * flinkCheckpointDataURI: 设置checkpoint的外部存储路径,根据实际的需求设定文件路径,hdfs://, file:// * jobmanager.memory.mb: per_job模式下指定jobmanager的内存大小(单位MB, 默认值:768) * taskmanager.memory.mb: per_job模式下指定taskmanager的内存大小(单位MB, 默认值:768) - * taskmanager.num: per_job模式下指定taskmanager的实例数(默认1) - * taskmanager.slots:per_job模式下指定每个taskmanager对应的slot数量(默认1) + * taskmanager.numberOfTaskSlots:per_job模式下指定每个taskmanager对应的slot数量(默认1),通过该参数和sql.env.parallelism可控制tm的个数,即sql.env.parallelism/taskmanager.numberOfTaskSlots 向上取整。 * savePointPath:任务恢复点的路径(默认无) * allowNonRestoredState:指示保存点是否允许非还原状态的标志(默认false) * logLevel: 日志级别动态配置(默认info) + * sample.interval.count:间隔一定数据条数后,将本次进入Flink的数据抽样打印到日志中。默认为0,不进行抽样打印。可以设置一个整数,例如:1000000。 * [prometheus 相关参数](./prometheus.md) per_job可指定metric写入到外部监控组件,以prometheus pushgateway举例 diff --git a/docs/plugin/kafkaSink.md b/docs/plugin/kafkaSink.md index 86b01f3ee..b3a1614bc 100644 --- a/docs/plugin/kafkaSink.md +++ b/docs/plugin/kafkaSink.md @@ -221,3 +221,29 @@ into from MyTable a ``` +## MAP类型示例 +目前Kafka Sink支持Map类型 +```sql +CREATE TABLE ods( + id INT, + name STRING +) WITH ( + ... +); + +CREATE TABLE dwd ( + id INT, + dids MAP> +) WITH ( + type ='kafka', + bootstrapServers ='localhost:9092', + offsetReset ='latest', + groupId='wuren_foo', + topic ='luna_foo', + parallelism ='1' +); + +INSERT INTO dwd + SELECT ods.id, MAP['foo', 1, 'bar', 2] AS dids + FROM ods; +``` diff --git a/docs/plugin/redisSink.md b/docs/plugin/redisSink.md index eb0109f06..8f573cf9a 100644 --- a/docs/plugin/redisSink.md +++ b/docs/plugin/redisSink.md @@ -44,6 +44,7 @@ redis5.0 |minIdle|最小空闲连接数|否||0| |masterName| 哨兵模式下的masterName|否|| |primarykeys|主键字段,多个字段以逗号分割|是|| +|keyExpiredTime|redis sink的key的过期时间。默认是0(永不过期),单位是s。|否|| ## 5.样例: diff --git a/docs/pluginsInfo.md b/docs/pluginsInfo.md index 88fda90c9..e9d23cdbd 100644 --- a/docs/pluginsInfo.md +++ b/docs/pluginsInfo.md @@ -1,9 +1,9 @@ ### 1 插件列表 #### 1.1 源表插件 * [kafka 源表插件](plugin/kafkaSource.md) -* [kafka 结果表插件](plugin/kafkaSink.md) #### 1.2 结果表插件 +* [kafka 结果表插件](plugin/kafkaSink.md) * [elasticsearch 结果表插件](plugin/elasticsearchSink.md) * [hbase 结果表插件](plugin/hbaseSink.md) * [mysql 结果表插件](plugin/mysqlSink.md) diff --git a/elasticsearch5/elasticsearch5-sink/pom.xml b/elasticsearch5/elasticsearch5-sink/pom.xml index 40e198279..7fee4b0ba 100644 --- a/elasticsearch5/elasticsearch5-sink/pom.xml +++ b/elasticsearch5/elasticsearch5-sink/pom.xml @@ -45,13 +45,13 @@ org.apache.flink - flink-connector-elasticsearch5_2.11 + flink-connector-elasticsearch5_${scala.binary.version} ${flink.version} org.apache.flink - flink-connector-elasticsearch-base_2.11 + flink-connector-elasticsearch-base_${scala.binary.version} ${flink.version} diff --git a/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java b/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java index c4c65d920..8b7afac4c 100644 --- a/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java +++ b/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java @@ -19,7 +19,12 @@ package com.dtstack.flink.sql.side.elasticsearch6; import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.BaseAsyncReqRow; +import com.dtstack.flink.sql.side.CacheMissVal; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.PredicateInfo; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.elasticsearch6.table.Elasticsearch6SideTableInfo; import com.dtstack.flink.sql.side.elasticsearch6.util.Es6Util; diff --git a/elasticsearch6/elasticsearch6-sink/pom.xml b/elasticsearch6/elasticsearch6-sink/pom.xml index 59690c251..6219e99dc 100644 --- a/elasticsearch6/elasticsearch6-sink/pom.xml +++ b/elasticsearch6/elasticsearch6-sink/pom.xml @@ -19,7 +19,7 @@ org.apache.flink - flink-streaming-java_2.11 + flink-streaming-java_${scala.binary.version} ${flink.version} provided @@ -50,13 +50,13 @@ org.apache.flink - flink-connector-elasticsearch6_2.11 + flink-connector-elasticsearch6_${scala.binary.version} ${flink.version} org.apache.flink - flink-connector-elasticsearch-base_2.11 + flink-connector-elasticsearch-base_${scala.binary.version} ${flink.version} diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbstractRowKeyModeDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbstractRowKeyModeDealer.java index 18bd1276c..4611a8d8f 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbstractRowKeyModeDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbstractRowKeyModeDealer.java @@ -57,6 +57,8 @@ public abstract class AbstractRowKeyModeDealer { protected List outFieldInfoList; + protected static final String ROWKEY = "rowkey"; + protected BaseSideInfo sideInfo; protected BaseAsyncTableFunction baseAsyncTableFunction; diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java index 086c143f7..f35cc8671 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java @@ -16,6 +16,7 @@ * limitations under the License. */ + package com.dtstack.flink.sql.side.hbase.rowkeydealer; @@ -32,12 +33,22 @@ import org.apache.calcite.sql.JoinType; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.types.Row; +import org.hbase.async.BinaryPrefixComparator; +import org.hbase.async.Bytes; +import org.hbase.async.CompareFilter; +import org.hbase.async.HBaseClient; +import org.hbase.async.KeyValue; +import org.hbase.async.RowFilter; +import org.hbase.async.ScanFilter; import org.hbase.async.Scanner; -import org.hbase.async.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; /** @@ -131,6 +142,12 @@ private String dealOneRow(ArrayList> args, String rowKeyStr, //The order of the fields defined in the data conversion table List sideVal = Lists.newArrayList(); for (String key : colNames) { + + if (ROWKEY.equalsIgnoreCase(key)) { + sideVal.add(rowKeyStr); + continue; + } + Object val = sideMap.get(key); if (val == null) { LOG.error("can't get data with column {}", key); diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java index 41310396b..9b8435857 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java @@ -101,6 +101,12 @@ public void asyncGetData(String tableName, String rowKeyStr, Row input, ResultFu //The order of the fields defined in the data conversion table List sideVal = Lists.newArrayList(); for (String key : colNames) { + + if (ROWKEY.equalsIgnoreCase(key)) { + sideVal.add(rowKeyStr); + continue; + } + Object val = sideMap.get(key); if (val == null) { LOG.error("can't get data with column {}", key); diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java index 7efbb7799..433c90666 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java @@ -16,16 +16,17 @@ * limitations under the License. */ - package com.dtstack.flink.sql.sink.hbase; +import com.dtstack.flink.sql.dirtyManager.manager.DirtyDataManager; +import com.dtstack.flink.sql.factory.DTThreadFactory; import com.dtstack.flink.sql.outputformat.AbstractDtRichOutputFormat; +import com.dtstack.flink.sql.util.SampleUtils; import com.google.common.collect.Maps; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import org.apache.hadoop.hbase.AuthUtil; @@ -44,33 +45,30 @@ import java.io.File; import java.io.IOException; import java.security.PrivilegedAction; -import java.util.LinkedList; import java.util.ArrayList; +import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; /** * @author: jingzhen@dtstack.com * date: 2017-6-29 */ -public class HbaseOutputFormat extends AbstractDtRichOutputFormat { +public class HbaseOutputFormat extends AbstractDtRichOutputFormat> { private static final long serialVersionUID = 3147774650287087471L; private static final Logger LOG = LoggerFactory.getLogger(HbaseOutputFormat.class); - private String host; private String zkParent; private String rowkey; private String tableName; private String[] columnNames; - private String[] columnTypes; private Map columnNameFamily; - private boolean kerberosAuthEnable; private String regionserverKeytabFile; private String regionserverPrincipal; @@ -78,54 +76,52 @@ public class HbaseOutputFormat extends AbstractDtRichOutputFormat { private String zookeeperSaslClient; private String clientPrincipal; private String clientKeytabFile; - private String[] families; private String[] qualifiers; - private transient org.apache.hadoop.conf.Configuration conf; private transient Connection conn; private transient Table table; - private transient ChoreService choreService; - + private transient List records; + private transient volatile boolean closed = false; + /** + * 批量写入的参数 + */ private Integer batchSize; private Long batchWaitInterval; + /** + * 定时任务 + */ + private transient ScheduledExecutorService scheduler; + private transient ScheduledFuture scheduledFuture; + private DirtyDataManager dirtyDataManager; - private transient ScheduledExecutorService executor; - private transient ScheduledFuture scheduledFuture; + private HbaseOutputFormat() { + } - private final List records = new ArrayList<>(); + public static HbaseOutputFormatBuilder buildHbaseOutputFormat() { + return new HbaseOutputFormatBuilder(); + } @Override public void configure(Configuration parameters) { - LOG.warn("---configure---"); - conf = HBaseConfiguration.create(); + // 这里不要做耗时较长的操作,否则会导致AKKA通信超时 + // DO NOTHING } @Override public void open(int taskNumber, int numTasks) throws IOException { LOG.warn("---open---"); + records = new ArrayList<>(); + conf = HBaseConfiguration.create(); openConn(); table = conn.getTable(TableName.valueOf(tableName)); LOG.warn("---open end(get table from hbase) ---"); initMetric(); - // 设置定时任务 - if (batchWaitInterval > 0) { - this.executor = Executors.newScheduledThreadPool( - 1, new ExecutorThreadFactory("hbase-sink-flusher")); - this.scheduledFuture = this.executor.scheduleAtFixedRate(() -> { - if (!records.isEmpty()) { - // 发送数据 - dealBatchOperation(records); - // 清空数据 - records.clear(); - } - }, batchWaitInterval, batchWaitInterval, TimeUnit.MILLISECONDS); - } } - private void openConn(){ - try{ + private void openConn() { + try { if (kerberosAuthEnable) { LOG.info("open kerberos conn"); openKerberosConn(); @@ -135,16 +131,46 @@ private void openConn(){ conf.set(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_ZNODE_QUORUM, zkParent); conn = ConnectionFactory.createConnection(conf); } - }catch (Exception e){ + } catch (Exception e) { throw new RuntimeException(e); } + initScheduledTask(batchWaitInterval); + } + /** + * 初始化定时写入任务 + * + * @param batchWaitInterval 定时任务时间 + */ + private void initScheduledTask(Long batchWaitInterval) { + try { + if (batchWaitInterval > 0) { + this.scheduler = new ScheduledThreadPoolExecutor( + 1, + new DTThreadFactory("hbase-batch-flusher") + ); + + this.scheduledFuture = this.scheduler.scheduleWithFixedDelay( + () -> { + synchronized (HbaseOutputFormat.this) { + if (!records.isEmpty()) { + dealBatchOperation(records); + } + } + }, batchWaitInterval, batchWaitInterval, TimeUnit.MILLISECONDS + ); + } + } catch (Exception e) { + LOG.error("init schedule task failed !"); + throw new RuntimeException(e); + } } + private void openKerberosConn() throws Exception { conf.set(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_QUORUM, host); conf.set(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_ZNODE_QUORUM, zkParent); - LOG.info("kerberos config:{}", this.toString()); + LOG.info("kerberos config:{}", this.conf.toString()); Preconditions.checkArgument(!StringUtils.isEmpty(clientPrincipal), " clientPrincipal not null!"); Preconditions.checkArgument(!StringUtils.isEmpty(clientKeytabFile), " clientKeytabFile not null!"); @@ -174,18 +200,13 @@ private void openKerberosConn() throws Exception { }); } - - @Override - public void writeRecord(Tuple2 tuple2) { - Tuple2 tupleTrans = tuple2; - Boolean retract = tupleTrans.f0; - Row row = tupleTrans.f1; - if (retract) { + public void writeRecord(Tuple2 record) { + if (record.f0) { if (this.batchSize != 0) { - writeBatchRecord(row); + writeBatchRecord(record.f1); } else { - dealInsert(row); + dealInsert(record.f1); } } } @@ -195,35 +216,10 @@ public void writeBatchRecord(Row row) { // 数据累计到batchSize之后开始处理 if (records.size() == this.batchSize) { dealBatchOperation(records); - // 添加完数据之后数据清空records - records.clear(); } } - protected void dealInsert(Row record) { - Put put = getPutByRow(record); - if (put == null || put.isEmpty()) { - outDirtyRecords.inc(); - return; - } - - try { - table.put(put); - } catch (Exception e) { - if (outDirtyRecords.getCount() % DIRTY_PRINT_FREQUENCY == 0 || LOG.isDebugEnabled()) { - LOG.error("record insert failed ..{}", record.toString()); - LOG.error("", e); - } - outDirtyRecords.inc(); - } - - if (outRecords.getCount() % ROW_PRINT_FREQUENCY == 0) { - LOG.info(record.toString()); - } - outRecords.inc(); - } - - protected void dealBatchOperation(List records) { + protected synchronized void dealBatchOperation(List records) { // A null in the result array means that the call for that action failed, even after retries. Object[] results = new Object[records.size()]; try { @@ -246,13 +242,35 @@ protected void dealBatchOperation(List records) { outRecords.inc(); } } - // 打印结果 - if (outRecords.getCount() % ROW_PRINT_FREQUENCY == 0) { - LOG.info(records.toString()); - } + // 只打印最后一条数据 + SampleUtils.samplingSinkPrint(samplingIntervalCount, LOG, outRecords.getCount(), records.get(records.size() - 1).toString()); } catch (IOException | InterruptedException e) { LOG.error("", e); + } finally { + // 添加完数据之后数据清空records + records.clear(); + } + } + + protected void dealInsert(Row record) { + Put put = getPutByRow(record); + if (put == null || put.isEmpty()) { + // 记录脏数据 + outDirtyRecords.inc(); + return; + } + + try { + table.put(put); + } catch (Exception e) { + dirtyDataManager.collectDirtyData( + record.toString() + , e.getMessage()); + outDirtyRecords.inc(); } + + SampleUtils.samplingSinkPrint(samplingIntervalCount, LOG, outRecords.getCount(), record.toString()); + outRecords.inc(); } private Put getPutByRow(Row record) { @@ -292,38 +310,80 @@ private String getRowKeyValues(Row record) { return rowKeyBuilder.getRowKey(row); } - private Map rowConvertMap(Row record){ + private Map rowConvertMap(Row record) { Map rowValue = Maps.newHashMap(); - for(int i = 0; i < columnNames.length; i++){ + for (int i = 0; i < columnNames.length; i++) { rowValue.put(columnNames[i], record.getField(i)); } return rowValue; } @Override - public void close() throws IOException { - if (conn != null) { - conn.close(); - conn = null; + public synchronized void close() throws IOException { + if (closed) { + return; + } + + closed = true; + if (!records.isEmpty()) { + dealBatchOperation(records); } if (scheduledFuture != null) { scheduledFuture.cancel(false); - if (executor != null) { - executor.shutdownNow(); + if (scheduler != null) { + scheduler.shutdownNow(); } } + + if (conn != null) { + conn.close(); + conn = null; + } } - private HbaseOutputFormat() { + + private void fillSyncKerberosConfig(org.apache.hadoop.conf.Configuration config, + String regionserverPrincipal, + String zookeeperSaslClient, + String securityKrb5Conf) { + if (StringUtils.isEmpty(regionserverPrincipal)) { + throw new IllegalArgumentException("Must provide regionserverPrincipal when authentication is Kerberos"); + } + config.set(HbaseConfigUtils.KEY_HBASE_MASTER_KERBEROS_PRINCIPAL, regionserverPrincipal); + config.set(HbaseConfigUtils.KEY_HBASE_REGIONSERVER_KERBEROS_PRINCIPAL, regionserverPrincipal); + config.set(HbaseConfigUtils.KEY_HBASE_SECURITY_AUTHORIZATION, "true"); + config.set(HbaseConfigUtils.KEY_HBASE_SECURITY_AUTHENTICATION, "kerberos"); + + + if (!StringUtils.isEmpty(zookeeperSaslClient)) { + System.setProperty(HbaseConfigUtils.KEY_ZOOKEEPER_SASL_CLIENT, zookeeperSaslClient); + } + + if (!StringUtils.isEmpty(securityKrb5Conf)) { + String krb5ConfPath = System.getProperty("user.dir") + File.separator + securityKrb5Conf; + LOG.info("krb5ConfPath:{}", krb5ConfPath); + System.setProperty(HbaseConfigUtils.KEY_JAVA_SECURITY_KRB5_CONF, krb5ConfPath); + } } - public static HbaseOutputFormatBuilder buildHbaseOutputFormat() { - return new HbaseOutputFormatBuilder(); + @Override + public String toString() { + return "HbaseOutputFormat kerberos{" + + "kerberosAuthEnable=" + kerberosAuthEnable + + ", regionserverKeytabFile='" + regionserverKeytabFile + '\'' + + ", regionserverPrincipal='" + regionserverPrincipal + '\'' + + ", securityKrb5Conf='" + securityKrb5Conf + '\'' + + ", zookeeperSaslClient='" + zookeeperSaslClient + '\'' + + ", clientPrincipal='" + clientPrincipal + '\'' + + ", clientKeytabFile='" + clientKeytabFile + '\'' + + ", batchSize='" + batchSize + '\'' + + ", batchWaitInterval='" + batchWaitInterval + '\'' + + '}'; } public static class HbaseOutputFormatBuilder { - private HbaseOutputFormat format; + private final HbaseOutputFormat format; private HbaseOutputFormatBuilder() { format = new HbaseOutputFormat(); @@ -355,11 +415,6 @@ public HbaseOutputFormatBuilder setColumnNames(String[] columnNames) { return this; } - public HbaseOutputFormatBuilder setColumnTypes(String[] columnTypes) { - format.columnTypes = columnTypes; - return this; - } - public HbaseOutputFormatBuilder setColumnNameFamily(Map columnNameFamily) { format.columnNameFamily = columnNameFamily; return this; @@ -400,6 +455,11 @@ public HbaseOutputFormatBuilder setClientKeytabFile(String clientKeytabFile) { return this; } + public HbaseOutputFormatBuilder setDirtyManager(DirtyDataManager dirtyDataManager) { + format.dirtyDataManager = dirtyDataManager; + return this; + } + public HbaseOutputFormatBuilder setBatchSize(Integer batchSize) { format.batchSize = batchSize; return this; @@ -434,42 +494,5 @@ public HbaseOutputFormat finish() { return format; } - - } - - private void fillSyncKerberosConfig(org.apache.hadoop.conf.Configuration config, String regionserverPrincipal, - String zookeeperSaslClient, String securityKrb5Conf) throws IOException { - if (StringUtils.isEmpty(regionserverPrincipal)) { - throw new IllegalArgumentException("Must provide regionserverPrincipal when authentication is Kerberos"); - } - config.set(HbaseConfigUtils.KEY_HBASE_MASTER_KERBEROS_PRINCIPAL, regionserverPrincipal); - config.set(HbaseConfigUtils.KEY_HBASE_REGIONSERVER_KERBEROS_PRINCIPAL, regionserverPrincipal); - config.set(HbaseConfigUtils.KEY_HBASE_SECURITY_AUTHORIZATION, "true"); - config.set(HbaseConfigUtils.KEY_HBASE_SECURITY_AUTHENTICATION, "kerberos"); - - - if (!StringUtils.isEmpty(zookeeperSaslClient)) { - System.setProperty(HbaseConfigUtils.KEY_ZOOKEEPER_SASL_CLIENT, zookeeperSaslClient); - } - - if (!StringUtils.isEmpty(securityKrb5Conf)) { - String krb5ConfPath = System.getProperty("user.dir") + File.separator + securityKrb5Conf; - LOG.info("krb5ConfPath:{}", krb5ConfPath); - System.setProperty(HbaseConfigUtils.KEY_JAVA_SECURITY_KRB5_CONF, krb5ConfPath); - } } - - @Override - public String toString() { - return "HbaseOutputFormat kerberos{" + - "kerberosAuthEnable=" + kerberosAuthEnable + - ", regionserverKeytabFile='" + regionserverKeytabFile + '\'' + - ", regionserverPrincipal='" + regionserverPrincipal + '\'' + - ", securityKrb5Conf='" + securityKrb5Conf + '\'' + - ", zookeeperSaslClient='" + zookeeperSaslClient + '\'' + - ", clientPrincipal='" + clientPrincipal + '\'' + - ", clientKeytabFile='" + clientKeytabFile + '\'' + - '}'; - } - } diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseSink.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseSink.java index ce39506e1..51d79e6af 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseSink.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseSink.java @@ -17,9 +17,9 @@ */ - package com.dtstack.flink.sql.sink.hbase; +import com.dtstack.flink.sql.dirtyManager.manager.DirtyDataManager; import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.hbase.table.HbaseTableInfo; import com.dtstack.flink.sql.table.AbstractTargetTableInfo; @@ -36,37 +36,38 @@ import org.apache.flink.types.Row; import java.util.Map; +import java.util.Properties; /** * Date: 2018/09/14 * Company: www.dtstack.com + * * @author sishu.yss */ public class HbaseSink implements RetractStreamTableSink, IStreamSinkGener { protected String[] fieldNames; protected Map columnNameFamily; - TypeInformation[] fieldTypes; protected String zookeeperQuorum; protected String port; protected String parent; protected String tableName; protected String rowkey; protected String registerTabName; - protected boolean kerberosAuthEnable; protected String regionserverKeytabFile; protected String regionserverPrincipal; protected String securityKrb5Conf; protected String zookeeperSaslClient; - protected String batchSize; protected String batchWaitInterval; - + TypeInformation[] fieldTypes; private String clientPrincipal; private String clientKeytabFile; private int parallelism = 1; + private Properties dirtyProperties; + public HbaseSink() { // TO DO NOTHING @@ -81,7 +82,7 @@ public HbaseSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { this.tableName = hbaseTableInfo.getTableName(); this.rowkey = hbaseTableInfo.getRowkey(); this.columnNameFamily = hbaseTableInfo.getColumnNameFamily(); - this.registerTabName = hbaseTableInfo.getName(); + this.registerTabName = hbaseTableInfo.getName(); this.kerberosAuthEnable = hbaseTableInfo.isKerberosAuthEnable(); this.regionserverKeytabFile = hbaseTableInfo.getRegionserverKeytabFile(); @@ -92,6 +93,8 @@ public HbaseSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { this.clientKeytabFile = hbaseTableInfo.getClientKeytabFile(); this.clientPrincipal = hbaseTableInfo.getClientPrincipal(); + this.dirtyProperties = hbaseTableInfo.getDirtyProperties(); + this.batchSize = hbaseTableInfo.getBatchSize(); this.batchWaitInterval = hbaseTableInfo.getBatchWaitInterval(); @@ -121,6 +124,7 @@ public DataStreamSink> consumeDataStream(DataStream> consumeDataStream(DataStream 0) { dataStreamSink.setParallelism(parallelism); } - + return dataStreamSink; } diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java index 78f847e06..51cc935b8 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java @@ -74,27 +74,26 @@ public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map fieldValues = new HashMap(); + Map fieldValues; try { ObjectMapper objectMapper = new ObjectMapper(); fieldValues = objectMapper.readValue(partitionfieldValuesStr, Map.class); diff --git a/impala/impala-side/impala-side-core/src/main/java/com/dtstack/flink/sql/side/impala/table/ImpalaSideTableInfo.java b/impala/impala-side/impala-side-core/src/main/java/com/dtstack/flink/sql/side/impala/table/ImpalaSideTableInfo.java index aaf2563ba..504ab520e 100644 --- a/impala/impala-side/impala-side-core/src/main/java/com/dtstack/flink/sql/side/impala/table/ImpalaSideTableInfo.java +++ b/impala/impala-side/impala-side-core/src/main/java/com/dtstack/flink/sql/side/impala/table/ImpalaSideTableInfo.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.side.impala.table; +import com.dtstack.flink.sql.core.rdb.JdbcResourceCheck; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.google.common.base.Preconditions; @@ -194,6 +195,10 @@ public boolean check() { Preconditions.checkNotNull(this.getPartitionfields(), "impala field of partitionfields is required"); } + if (getFastCheck()) { + JdbcResourceCheck.getInstance().checkResourceStatus(this.getCheckProperties()); + } + return true; } } diff --git a/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/ImpalaOutputFormat.java b/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/ImpalaOutputFormat.java index 0e8288eca..70a021e46 100644 --- a/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/ImpalaOutputFormat.java +++ b/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/ImpalaOutputFormat.java @@ -18,12 +18,13 @@ package com.dtstack.flink.sql.sink.impala; +import com.dtstack.flink.sql.classloader.ClassLoaderManager; import com.dtstack.flink.sql.factory.DTThreadFactory; import com.dtstack.flink.sql.outputformat.AbstractDtRichOutputFormat; import com.dtstack.flink.sql.sink.rdb.JDBCTypeConvertUtils; import com.dtstack.flink.sql.table.AbstractTableInfo; -import com.dtstack.flink.sql.util.JDBCUtils; import com.dtstack.flink.sql.util.KrbUtils; +import com.dtstack.flink.sql.util.SampleUtils; import com.google.common.collect.Maps; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.java.tuple.Tuple2; @@ -34,7 +35,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.rmi.RemoteException; import java.security.PrivilegedExceptionAction; import java.sql.Connection; import java.sql.DriverManager; @@ -52,7 +52,6 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -80,7 +79,6 @@ public class ImpalaOutputFormat extends AbstractDtRichOutputFormat staticPartitionFields = new ArrayList<>(); + public List fieldNames; + public List fieldTypes; + public List fieldExtraInfoList; protected transient Connection connection; protected transient Statement statement; protected transient PreparedStatement updateStatement; - - private transient volatile boolean closed = false; - private int batchCount = 0; - - // |------------------------------------------------| - // | partitionCondition |Array of valueCondition| - // |------------------------------------------------| - // | ptOne, ptTwo, ptThree | [(v1, v2, v3, v4, v5)]| DP - // |------------------------------------------------| - // | ptOne = v1, ptTwo = v2 | [(v3, v4, v5)] | SP - // |------------------------------------------------| - // | ptOne, ptTwo = v2 | [(v1, v3, v4, v5)] | DP and SP - // |------------------------------------------------| - // | noPartition | [(v1, v2, v3, v4, v5)]| kudu or disablePartition - // |------------------------------------------------| - private transient Map> rowDataMap; - protected String keytabPath; protected String krb5confPath; protected String principal; @@ -129,13 +114,20 @@ public class ImpalaOutputFormat extends AbstractDtRichOutputFormat fieldNames; - public List fieldTypes; - public List fieldExtraInfoList; - - // partition field of static partition which matched by ${field} - private final List staticPartitionFields = new ArrayList<>(); - + private transient volatile boolean closed = false; + private int batchCount = 0; + // |------------------------------------------------| + // | partitionCondition |Array of valueCondition| + // |------------------------------------------------| + // | ptOne, ptTwo, ptThree | [(v1, v2, v3, v4, v5)]| DP + // |------------------------------------------------| + // | ptOne = v1, ptTwo = v2 | [(v3, v4, v5)] | SP + // |------------------------------------------------| + // | ptOne, ptTwo = v2 | [(v1, v3, v4, v5)] | DP and SP + // |------------------------------------------------| + // | noPartition | [(v1, v2, v3, v4, v5)]| kudu or disablePartition + // |------------------------------------------------| + private transient Map> rowDataMap; // valueFieldsName -> 重组之后的fieldNames,为了重组row data字段值对应 // 需要对partition字段做特殊处理,比如原来的字段顺序为(age, name, id),但是因为partition,写入的SQL为 // INSERT INTO tableName(name, id) PARTITION(age) VALUES(?, ?, ?) @@ -147,6 +139,10 @@ public class ImpalaOutputFormat extends AbstractDtRichOutputFormat scheduledFuture; + public static Builder getImpalaBuilder() { + return new Builder(); + } + @Override public void configure(Configuration parameters) { } @@ -229,7 +225,7 @@ private void openConnect() throws IOException { * get jdbc connection */ private void openJdbc() { - JDBCUtils.forName(DRIVER_NAME, getClass().getClassLoader()); + ClassLoaderManager.forName(DRIVER_NAME, getClass().getClassLoader()); try { connection = DriverManager.getConnection(dbUrl, userName, password); statement = connection.createStatement(); @@ -397,9 +393,7 @@ public void writeRecord(Tuple2 record) throws IOException { return; } - if (outRecords.getCount() % RECEIVE_DATA_PRINT_FREQUENCY == 0 || LOG.isDebugEnabled()) { - LOG.info("Receive data : {}", record); - } + SampleUtils.samplingSinkPrint(samplingIntervalCount, LOG, outRecords.getCount(), record.toString()); if (updateMode.equalsIgnoreCase(UPDATE_MODE)) { rows.add(Row.copy(record.f1)); @@ -627,10 +621,6 @@ private String quoteIdentifier(String identifier) { return "`" + identifier + "`"; } - public static Builder getImpalaBuilder() { - return new Builder(); - } - public static class Builder { private final ImpalaOutputFormat format = new ImpalaOutputFormat(); diff --git a/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/table/ImpalaSinkParser.java b/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/table/ImpalaSinkParser.java index 003b5a6ab..e7e28d1bd 100644 --- a/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/table/ImpalaSinkParser.java +++ b/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/table/ImpalaSinkParser.java @@ -101,6 +101,7 @@ public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map[] fieldClasses = kafka11SinkTableInfo.getFieldClasses(); TypeInformation[] types = IntStream.range(0, fieldClasses.length) - .mapToObj(i -> TypeInformation.of(fieldClasses[i])) + .mapToObj( + i -> { + if (fieldClasses[i].isArray()) { + return DataTypeUtils.convertToArray(fieldTypes[i]); + } + if (fieldClasses[i] == new HashMap().getClass()) { + return DataTypeUtils.convertToMap(fieldTypes[i]); + } + return TypeInformation.of(fieldClasses[i]); + }) .toArray(TypeInformation[]::new); return types; } @@ -101,8 +114,13 @@ protected TableSchema buildTableSchema(String[] fieldNames, TypeInformation[] } protected String[] getPartitionKeys(KafkaSinkTableInfo kafkaSinkTableInfo) { - if (StringUtils.isNotBlank(kafkaSinkTableInfo.getPartitionKeys())) { - return StringUtils.split(kafkaSinkTableInfo.getPartitionKeys(), ','); + String keysStr = kafkaSinkTableInfo.getPartitionKeys(); + if (StringUtils.isNotBlank(keysStr)) { + String[] keys = StringUtils.split(keysStr, ","); + String[] cleanedKeys = Arrays.stream(keys) + .map(x -> x.trim()) + .toArray(String[]::new); + return cleanedKeys; } return null; } diff --git a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/AbstractKafkaConsumerFactory.java b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/AbstractKafkaConsumerFactory.java index 1c2ec567b..481c59270 100644 --- a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/AbstractKafkaConsumerFactory.java +++ b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/AbstractKafkaConsumerFactory.java @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.source.kafka; +import com.dtstack.flink.sql.dirtyManager.manager.DirtyDataManager; import com.dtstack.flink.sql.format.DeserializationMetricWrapper; import com.dtstack.flink.sql.format.FormatType; import com.dtstack.flink.sql.format.dtnest.DtNestRowDeserializationSchema; @@ -50,7 +51,8 @@ protected DeserializationMetricWrapper createDeserializationMetricWrapper(KafkaS Calculate calculate) { return new KafkaDeserializationMetricWrapper(typeInformation, createDeserializationSchema(kafkaSourceTableInfo, typeInformation), - calculate); + calculate, + DirtyDataManager.newInstance(kafkaSourceTableInfo.getDirtyProperties())); } protected DeserializationSchema createDeserializationSchema(KafkaSourceTableInfo kafkaSourceTableInfo, TypeInformation typeInformation) { diff --git a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaDeserializationMetricWrapper.java b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaDeserializationMetricWrapper.java index 24a72736c..fe6555821 100644 --- a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaDeserializationMetricWrapper.java +++ b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaDeserializationMetricWrapper.java @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.source.kafka; +import com.dtstack.flink.sql.dirtyManager.manager.DirtyDataManager; import com.dtstack.flink.sql.format.DeserializationMetricWrapper; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -57,8 +58,12 @@ public class KafkaDeserializationMetricWrapper extends DeserializationMetricWrap private Calculate calculate; - public KafkaDeserializationMetricWrapper(TypeInformation typeInfo, DeserializationSchema deserializationSchema, Calculate calculate) { - super(typeInfo, deserializationSchema); + public KafkaDeserializationMetricWrapper( + TypeInformation typeInfo + , DeserializationSchema deserializationSchema + , Calculate calculate + , DirtyDataManager dirtyDataManager) { + super(typeInfo, deserializationSchema, dirtyDataManager); this.calculate = calculate; } diff --git a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 6cb5643a1..de566fe35 100644 --- a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -41,9 +41,10 @@ public class KafkaSourceParser extends AbstractSourceParser { public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) throws Exception { KafkaSourceTableInfo kafkaSourceTableInfo = new KafkaSourceTableInfo(); - parseFieldsInfo(fieldsInfo, kafkaSourceTableInfo); kafkaSourceTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafkaSourceTableInfo); + kafkaSourceTableInfo.setType(MathUtil.getString(props.get(KafkaSourceTableInfo.TYPE_KEY.toLowerCase()))); kafkaSourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSourceTableInfo.PARALLELISM_KEY.toLowerCase()))); kafkaSourceTableInfo.setBootstrapServers(MathUtil.getString(props.get(KafkaSourceTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); diff --git a/kafka-base/pom.xml b/kafka-base/pom.xml index 9db588576..21933e70a 100644 --- a/kafka-base/pom.xml +++ b/kafka-base/pom.xml @@ -55,7 +55,7 @@ under the License. org.apache.flink - flink-connector-kafka_2.11 + flink-connector-kafka_${scala.binary.version} ${flink.version} provided diff --git a/kafka/kafka-source/pom.xml b/kafka/kafka-source/pom.xml index 0a337cf87..c817f41c6 100644 --- a/kafka/kafka-source/pom.xml +++ b/kafka/kafka-source/pom.xml @@ -20,7 +20,7 @@ sql.source.kafka-base - flink-connector-kafka_2.11 + flink-connector-kafka_${scala.binary.version} org.apache.flink diff --git a/kafka/pom.xml b/kafka/pom.xml index 772671ff6..fe39e9d11 100644 --- a/kafka/pom.xml +++ b/kafka/pom.xml @@ -20,7 +20,7 @@ org.apache.flink - flink-connector-kafka_2.11 + flink-connector-kafka_${scala.binary.version} ${flink.version} diff --git a/kafka10/pom.xml b/kafka10/pom.xml index d8ad7370f..fb980eb2b 100644 --- a/kafka10/pom.xml +++ b/kafka10/pom.xml @@ -20,7 +20,7 @@ org.apache.flink - flink-connector-kafka-0.10_2.11 + flink-connector-kafka-0.10_${scala.binary.version} ${flink.version} diff --git a/kafka11/pom.xml b/kafka11/pom.xml index 90972fa82..39e7172cd 100644 --- a/kafka11/pom.xml +++ b/kafka11/pom.xml @@ -20,7 +20,7 @@ org.apache.flink - flink-connector-kafka-0.11_2.11 + flink-connector-kafka-0.11_${scala.binary.version} ${flink.version} diff --git a/kingbase/kingbase-side/kingbase-side-core/src/main/java/com/dtstack/flink/sql/side/kingbase/table/KingbaseSideParser.java b/kingbase/kingbase-side/kingbase-side-core/src/main/java/com/dtstack/flink/sql/side/kingbase/table/KingbaseSideParser.java index ef8c78305..8153918d7 100644 --- a/kingbase/kingbase-side/kingbase-side-core/src/main/java/com/dtstack/flink/sql/side/kingbase/table/KingbaseSideParser.java +++ b/kingbase/kingbase-side/kingbase-side-core/src/main/java/com/dtstack/flink/sql/side/kingbase/table/KingbaseSideParser.java @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.side.kingbase.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -34,6 +35,7 @@ public class KingbaseSideParser extends RdbSideParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "com.kingbase8.Driver"); AbstractTableInfo kingbaseTableInfo = super.getTableInfo(tableName, fieldsInfo, props); kingbaseTableInfo.setType(CURRENT_TYPE); return kingbaseTableInfo; diff --git a/kingbase/kingbase-sink/src/main/java/com/dtstack/flink/sql/sink/kingbase/table/KingbaseSinkParser.java b/kingbase/kingbase-sink/src/main/java/com/dtstack/flink/sql/sink/kingbase/table/KingbaseSinkParser.java index 1a82d49ac..a22d09004 100644 --- a/kingbase/kingbase-sink/src/main/java/com/dtstack/flink/sql/sink/kingbase/table/KingbaseSinkParser.java +++ b/kingbase/kingbase-sink/src/main/java/com/dtstack/flink/sql/sink/kingbase/table/KingbaseSinkParser.java @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.sink.kingbase.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -34,6 +35,7 @@ public class KingbaseSinkParser extends RdbSinkParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "com.kingbase8.Driver"); AbstractTableInfo kingbaseTableInfo = super.getTableInfo(tableName, fieldsInfo, props); kingbaseTableInfo.setType(CURRENT_TYPE); return kingbaseTableInfo; diff --git a/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllReqRow.java b/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllReqRow.java index c79cd58f2..d65e391e2 100644 --- a/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllReqRow.java +++ b/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllReqRow.java @@ -1,6 +1,10 @@ package com.dtstack.flink.sql.side.kudu; -import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.BaseAllReqRow; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.PredicateInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.kudu.table.KuduSideTableInfo; import com.dtstack.flink.sql.side.kudu.utils.KuduUtil; import com.dtstack.flink.sql.util.KrbUtils; @@ -16,14 +20,25 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.kudu.ColumnSchema; import org.apache.kudu.Schema; -import org.apache.kudu.client.*; +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduException; +import org.apache.kudu.client.KuduPredicate; +import org.apache.kudu.client.KuduScanner; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.PartialRow; +import org.apache.kudu.client.RowResult; +import org.apache.kudu.client.RowResultIterator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.security.PrivilegedAction; import java.sql.SQLException; -import java.util.*; +import java.util.Arrays; +import java.util.Calendar; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicReference; public class KuduAllReqRow extends BaseAllReqRow { diff --git a/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncReqRow.java b/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncReqRow.java index e4c6e4914..7b65c7f4d 100644 --- a/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncReqRow.java +++ b/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncReqRow.java @@ -1,7 +1,12 @@ package com.dtstack.flink.sql.side.kudu; import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.BaseAsyncReqRow; +import com.dtstack.flink.sql.side.CacheMissVal; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.PredicateInfo; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.kudu.table.KuduSideTableInfo; import com.dtstack.flink.sql.side.kudu.utils.KuduUtil; @@ -19,7 +24,13 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.kudu.ColumnSchema; import org.apache.kudu.Schema; -import org.apache.kudu.client.*; +import org.apache.kudu.client.AsyncKuduClient; +import org.apache.kudu.client.AsyncKuduScanner; +import org.apache.kudu.client.KuduException; +import org.apache.kudu.client.KuduPredicate; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.RowResult; +import org.apache.kudu.client.RowResultIterator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduOutputFormat.java b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduOutputFormat.java index 6f2a31caa..ea5d68416 100644 --- a/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduOutputFormat.java +++ b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduOutputFormat.java @@ -20,13 +20,12 @@ import com.dtstack.flink.sql.outputformat.AbstractDtRichOutputFormat; import com.dtstack.flink.sql.util.KrbUtils; +import com.dtstack.flink.sql.util.SampleUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.kudu.client.AsyncKuduClient; -import org.apache.kudu.client.AsyncKuduSession; import org.apache.kudu.client.KuduClient; import org.apache.kudu.client.KuduException; import org.apache.kudu.client.KuduSession; @@ -52,26 +51,12 @@ public class KuduOutputFormat extends AbstractDtRichOutputFormat { private static final long serialVersionUID = 1L; private static final Logger LOG = LoggerFactory.getLogger(KuduOutputFormat.class); - - public enum WriteMode { - // insert - INSERT, - // update - UPDATE, - // update or insert - UPSERT - } - + protected String[] fieldNames; + TypeInformation[] fieldTypes; + boolean enableKrb; private String kuduMasters; - private String tableName; - private WriteMode writeMode; - - protected String[] fieldNames; - - TypeInformation[] fieldTypes; - private KuduClient client; private KuduTable table; @@ -90,11 +75,14 @@ public enum WriteMode { private String principal; private String keytab; private String krb5conf; - boolean enableKrb; private KuduOutputFormat() { } + public static KuduOutputFormatBuilder buildKuduOutputFormat() { + return new KuduOutputFormatBuilder(); + } + @Override public void configure(Configuration parameters) { @@ -112,11 +100,11 @@ private void establishConnection() throws IOException { kuduClientBuilder.workerCount(workerCount); } if (null != defaultSocketReadTimeoutMs) { - kuduClientBuilder.workerCount(defaultSocketReadTimeoutMs); + kuduClientBuilder.defaultSocketReadTimeoutMs(defaultSocketReadTimeoutMs); } if (null != defaultOperationTimeoutMs) { - kuduClientBuilder.workerCount(defaultOperationTimeoutMs); + kuduClientBuilder.defaultOperationTimeoutMs(defaultOperationTimeoutMs); } if (enableKrb) { @@ -130,15 +118,21 @@ private void establishConnection() throws IOException { } else { client = kuduClientBuilder.build(); } - LOG.info("connect kudu is succeed!"); + if (client.tableExists(tableName)) { table = client.openTable(tableName); } + if (Objects.isNull(table)) { + throw new IllegalArgumentException( + String.format("Table [%s] Open Failed , please check table exists", tableName)); + } + LOG.info("connect kudu is succeed!"); + session = client.newSession(); } @Override - public void writeRecord(Tuple2 record) throws IOException { + public void writeRecord(Tuple2 record) { Tuple2 tupleTrans = record; Boolean retract = tupleTrans.getField(0); if (!retract) { @@ -155,9 +149,7 @@ public void writeRecord(Tuple2 record) throws IOException { } try { - if (outRecords.getCount() % ROW_PRINT_FREQUENCY == 0) { - LOG.info("Receive data : {}", row); - } + SampleUtils.samplingSinkPrint(samplingIntervalCount, LOG, outRecords.getCount(), row.toString()); session.apply(toOperation(writeMode, row)); outRecords.inc(); } catch (KuduException e) { @@ -188,8 +180,97 @@ public void close() { } } - public static KuduOutputFormatBuilder buildKuduOutputFormat() { - return new KuduOutputFormatBuilder(); + private Operation toOperation(WriteMode writeMode, Row row) { + Operation operation = toOperation(writeMode); + PartialRow partialRow = operation.getRow(); + + for (int index = 0; index < row.getArity(); index++) { + //解决kudu中全小写字段找不到的bug + String fieldName = fieldNames[index].toLowerCase(); + if (row.getField(index) == null) { + partialRow.setNull(fieldName); + } else { + if (fieldTypes[index].getTypeClass() == String.class) { + partialRow.addString(fieldName, (String) row.getField(index)); + continue; + } + if (fieldTypes[index].getTypeClass() == Float.class) { + partialRow.addFloat(fieldName, (Float) row.getField(index)); + continue; + } + if (fieldTypes[index].getTypeClass() == Byte.class) { + partialRow.addByte(fieldName, (Byte) row.getField(index)); + continue; + } + + if (fieldTypes[index].getTypeClass() == Short.class) { + partialRow.addShort(fieldName, (Short) row.getField(index)); + continue; + } + + if (fieldTypes[index].getTypeClass() == Integer.class) { + partialRow.addInt(fieldName, (Integer) row.getField(index)); + continue; + } + + if (fieldTypes[index].getTypeClass() == Long.class) { + partialRow.addLong(fieldName, (Long) row.getField(index)); + continue; + } + + if (fieldTypes[index].getTypeClass() == Double.class) { + partialRow.addDouble(fieldName, (Double) row.getField(index)); + continue; + } + + if (fieldTypes[index].getTypeClass() == BigDecimal.class) { + partialRow.addDecimal(fieldName, (BigDecimal) row.getField(index)); + continue; + } + if (fieldTypes[index].getTypeClass() == Boolean.class) { + partialRow.addBoolean(fieldName, (Boolean) row.getField(index)); + continue; + } + + if (fieldTypes[index].getTypeClass() == Date.class) { + partialRow.addTimestamp(fieldName, new Timestamp(((Date) row.getField(index)).getTime())); + continue; + } + + if (fieldTypes[index].getTypeClass() == Timestamp.class) { + partialRow.addTimestamp(fieldName, (Timestamp) row.getField(index)); + continue; + } + + if (fieldTypes[index].getTypeClass() == byte[].class) { + partialRow.addBinary(fieldName, (byte[]) row.getField(index)); + continue; + } + throw new IllegalArgumentException("Illegal var type: " + fieldTypes[index]); + } + } + return operation; + + } + + private Operation toOperation(WriteMode writeMode) { + switch (writeMode) { + case INSERT: + return table.newInsert(); + case UPDATE: + return table.newUpdate(); + default: + return table.newUpsert(); + } + } + + public enum WriteMode { + // insert + INSERT, + // update + UPDATE, + // update or insert + UPSERT } public static class KuduOutputFormatBuilder { @@ -276,93 +357,4 @@ public KuduOutputFormat finish() { } } - private Operation toOperation(WriteMode writeMode, Row row) { - if (null == table) { - throw new IllegalArgumentException("Table Open Failed , please check table exists"); - } - Operation operation = toOperation(writeMode); - PartialRow partialRow = operation.getRow(); - - for (int index = 0; index < row.getArity(); index++) { - //解决kudu中全小写字段找不到的bug - String fieldName = fieldNames[index].toLowerCase(); - if (row.getField(index) == null) { - partialRow.setNull(fieldName); - } else { - if (fieldTypes[index].getTypeClass() == String.class) { - partialRow.addString(fieldName, (String) row.getField(index)); - continue; - } - if (fieldTypes[index].getTypeClass() == Float.class) { - partialRow.addFloat(fieldName, (Float) row.getField(index)); - continue; - } - if (fieldTypes[index].getTypeClass() == Byte.class) { - partialRow.addByte(fieldName, (Byte) row.getField(index)); - continue; - } - - if (fieldTypes[index].getTypeClass() == Short.class) { - partialRow.addShort(fieldName, (Short) row.getField(index)); - continue; - } - - if (fieldTypes[index].getTypeClass() == Integer.class) { - partialRow.addInt(fieldName, (Integer) row.getField(index)); - continue; - } - - if (fieldTypes[index].getTypeClass() == Long.class) { - partialRow.addLong(fieldName, (Long) row.getField(index)); - continue; - } - - if (fieldTypes[index].getTypeClass() == Double.class) { - partialRow.addDouble(fieldName, (Double) row.getField(index)); - continue; - } - - if (fieldTypes[index].getTypeClass() == BigDecimal.class) { - partialRow.addDecimal(fieldName, (BigDecimal) row.getField(index)); - continue; - } - if (fieldTypes[index].getTypeClass() == Boolean.class) { - partialRow.addBoolean(fieldName, (Boolean) row.getField(index)); - continue; - } - - if (fieldTypes[index].getTypeClass() == Date.class) { - partialRow.addTimestamp(fieldName, new Timestamp(((Date) row.getField(index)).getTime())); - continue; - } - - if (fieldTypes[index].getTypeClass() == Timestamp.class) { - partialRow.addTimestamp(fieldName, (Timestamp) row.getField(index)); - continue; - } - - if (fieldTypes[index].getTypeClass() == byte[].class) { - partialRow.addBinary(fieldName, (byte[]) row.getField(index)); - continue; - } - throw new IllegalArgumentException("Illegal var type: " + fieldTypes[index]); - } - } - return operation; - - } - - private Operation toOperation(WriteMode writeMode) { - switch (writeMode) { - case INSERT: - return table.newInsert(); - case UPDATE: - return table.newUpdate(); - case UPSERT: - return table.newUpsert(); - default: - return table.newUpsert(); - } - } - } \ No newline at end of file diff --git a/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduSink.java b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduSink.java index 0b15501b5..c7421cf60 100644 --- a/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduSink.java +++ b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduSink.java @@ -21,28 +21,19 @@ public class KuduSink implements RetractStreamTableSink, Serializable, IStreamSinkGener { + protected String[] fieldNames; + TypeInformation[] fieldTypes; + boolean enableKrb; private String kuduMasters; - private String tableName; - private KuduOutputFormat.WriteMode writeMode; - - protected String[] fieldNames; - - TypeInformation[] fieldTypes; - private Integer workerCount; - private Integer defaultOperationTimeoutMs; - private Integer defaultSocketReadTimeoutMs; - private int parallelism = 1; - private String principal; private String keytab; private String krb5conf; - boolean enableKrb; @Override public KuduSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { @@ -66,7 +57,7 @@ public KuduSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { @Override public DataStreamSink> consumeDataStream(DataStream> dataStream) { KuduOutputFormat.KuduOutputFormatBuilder builder = KuduOutputFormat.buildKuduOutputFormat(); - builder.setKuduMasters(this.kuduMasters) + KuduOutputFormat kuduOutputFormat = builder.setKuduMasters(this.kuduMasters) .setTableName(this.tableName) .setWriteMode(writeMode) .setWorkerCount(this.workerCount) @@ -77,8 +68,8 @@ public DataStreamSink> consumeDataStream(DataStream org.apache.flink - flink-yarn_2.11 + flink-yarn_${scala.binary.version} ${flink.version} @@ -40,12 +40,6 @@ slf4j-log4j12 1.6.1 - - - com.alibaba - fastjson - 1.2.70 - diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 1ad7e7d73..1f354fe35 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -19,14 +19,15 @@ package com.dtstack.flink.sql.launcher; +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.TypeReference; +import com.dtstack.flink.sql.Main; +import com.dtstack.flink.sql.dirtyManager.manager.DirtyDataManager; +import com.dtstack.flink.sql.enums.ClusterMode; import com.dtstack.flink.sql.launcher.entity.JobParamsInfo; import com.dtstack.flink.sql.launcher.executor.StandaloneExecutor; import com.dtstack.flink.sql.launcher.executor.YarnJobClusterExecutor; import com.dtstack.flink.sql.launcher.executor.YarnSessionClusterExecutor; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.TypeReference; -import com.dtstack.flink.sql.enums.ClusterMode; -import com.dtstack.flink.sql.Main; import com.dtstack.flink.sql.option.OptionParser; import com.dtstack.flink.sql.option.Options; import com.dtstack.flink.sql.util.PluginUtil; @@ -41,6 +42,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Properties; /** @@ -72,12 +74,15 @@ public static JobParamsInfo parseArgs(String[] args) throws Exception { String queue = launcherOptions.getQueue(); String pluginLoadMode = launcherOptions.getPluginLoadMode(); String addShipfile = launcherOptions.getAddShipfile(); + String dirtyStr = launcherOptions.getDirtyProperties(); String yarnSessionConf = URLDecoder.decode(launcherOptions.getYarnSessionConf(), Charsets.UTF_8.toString()); Properties yarnSessionConfProperties = PluginUtil.jsonStrToObject(yarnSessionConf, Properties.class); String confProp = URLDecoder.decode(launcherOptions.getConfProp(), Charsets.UTF_8.toString()); Properties confProperties = PluginUtil.jsonStrToObject(confProp, Properties.class); + Properties dirtyProperties = PluginUtil.jsonStrToObject(Objects.isNull(dirtyStr) ? + DirtyDataManager.buildDefaultDirty() : dirtyStr, Properties.class); return JobParamsInfo.builder() .setExecArgs(execArgs) @@ -92,10 +97,12 @@ public static JobParamsInfo parseArgs(String[] args) throws Exception { .setFlinkJarPath(flinkJarPath) .setPluginLoadMode(pluginLoadMode) .setQueue(queue) + .setDirtyProperties(dirtyProperties) .setAddShipfile(addShipfile) .build(); } + private static String[] parseJson(String[] args) { BufferedReader reader = null; StringBuilder lastStr = new StringBuilder(); @@ -130,7 +137,6 @@ private static String[] parseJson(String[] args) { return list.toArray(new String[0]); } - public static void main(String[] args) throws Exception { JobParamsInfo jobParamsInfo = parseArgs(args); ClusterMode execMode = ClusterMode.valueOf(jobParamsInfo.getMode()); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/entity/JobParamsInfo.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/entity/JobParamsInfo.java index 9e7529ef0..628b847ff 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/entity/JobParamsInfo.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/entity/JobParamsInfo.java @@ -43,10 +43,23 @@ public class JobParamsInfo { private final Properties confProperties; private final Properties yarnSessionConfProperties; private final String addShipFile; - - private JobParamsInfo(String mode, String name, String queue, String localPluginRoot, String flinkConfDir, String yarnConfDir, - String pluginLoadMode, String[] execArgs, Properties confProperties, Properties yarnSessionConfProperties, - String udfJar, String flinkJarPath, String addShipFile) { + private final Properties dirtyProperties; + + private JobParamsInfo( + String mode + , String name + , String queue + , String localPluginRoot + , String flinkConfDir + , String yarnConfDir + , String pluginLoadMode + , String[] execArgs + , Properties confProperties + , Properties yarnSessionConfProperties + , String udfJar + , String flinkJarPath + , String addShipFile + , Properties dirtyProperties) { this.mode = mode; this.name = name; this.queue = queue; @@ -60,6 +73,11 @@ private JobParamsInfo(String mode, String name, String queue, String localPlugin this.udfJar = udfJar; this.flinkJarPath = flinkJarPath; this.addShipFile = addShipFile; + this.dirtyProperties = dirtyProperties; + } + + public static JobParamsInfo.Builder builder() { + return new JobParamsInfo.Builder(); } public String getMode() { @@ -114,11 +132,10 @@ public String getAddShipFile() { return addShipFile; } - public static JobParamsInfo.Builder builder() { - return new JobParamsInfo.Builder(); + public Properties getDirtyProperties() { + return dirtyProperties; } - public static class Builder { private String mode; private String name; @@ -133,6 +150,7 @@ public static class Builder { private Properties confProperties; private Properties yarnSessionConfProperties; private String addShipfile; + private Properties dirtyProperties; public JobParamsInfo.Builder setMode(String mode) { this.mode = mode; @@ -199,10 +217,27 @@ public JobParamsInfo.Builder setAddShipfile(String addShipfile) { return this; } + public JobParamsInfo.Builder setDirtyProperties(Properties dirtyProperties) { + this.dirtyProperties = dirtyProperties; + return this; + } + public JobParamsInfo build() { - return new JobParamsInfo(mode, name, queue, localPluginRoot, flinkConfDir, - yarnConfDir, pluginLoadMode, execArgs, confProperties, - yarnSessionConfProperties, udfJar, flinkJarPath, addShipfile); + return new JobParamsInfo( + mode + , name + , queue + , localPluginRoot + , flinkConfDir + , yarnConfDir + , pluginLoadMode + , execArgs + , confProperties + , yarnSessionConfProperties + , udfJar + , flinkJarPath + , addShipfile + , dirtyProperties); } } @@ -222,6 +257,7 @@ public String toString() { ", confProperties=" + confProperties + ", yarnSessionConfProperties=" + yarnSessionConfProperties + ", addShipFile='" + addShipFile + '\'' + + ", dirtyProperties=" + dirtyProperties + '}'; } } diff --git a/localTest/pom.xml b/localTest/pom.xml index 32cde2cd9..9e0ac1f56 100644 --- a/localTest/pom.xml +++ b/localTest/pom.xml @@ -69,6 +69,18 @@ + + com.dtstack.flink + dirtyConsumer.mysql + 1.0-SNAPSHOT + + + + com.dtstack.flink + dirtyConsumer.console + 1.0-SNAPSHOT + + com.dtstack.flink sql.source.kafka11 diff --git a/localTest/src/main/java/com/dtstack/flink/sql/localTest/LocalTest.java b/localTest/src/main/java/com/dtstack/flink/sql/localTest/LocalTest.java index 5502c34c5..6a4570165 100644 --- a/localTest/src/main/java/com/dtstack/flink/sql/localTest/LocalTest.java +++ b/localTest/src/main/java/com/dtstack/flink/sql/localTest/LocalTest.java @@ -50,7 +50,7 @@ public static void main(String[] args) throws Exception { setLogLevel("INFO"); List propertiesList = new ArrayList<>(); - String sqlPath = "/Users/chuixue/Desktop/tmp/sqlFile.sql"; + String sqlPath = "/Users/wtz/dtstack/sql/test/JoinDemoThree.sql"; Map conf = new HashMap<>(); JSONObject properties = new JSONObject(); @@ -66,6 +66,7 @@ public static void main(String[] args) throws Exception { conf.put("-confProp", properties.toString()); conf.put("-pluginLoadMode", "LocalTest"); conf.put("-planner", "flink"); + conf.put("-dirtyProperties", buildDirtyStr()); for (Map.Entry keyValue : conf.entrySet()) { propertiesList.add(keyValue.getKey()); @@ -75,6 +76,23 @@ public static void main(String[] args) throws Exception { Main.main(propertiesList.toArray(new String[0])); } + private static String buildDirtyStr() { + JSONObject jsonObject = new JSONObject(); + jsonObject.put("type", "console"); + // 多少条数据打印一次 + jsonObject.put("printLimit", "100"); + jsonObject.put("url", "jdbc:mysql://localhost:3306/tiezhu"); + jsonObject.put("userName", "root"); + jsonObject.put("password", "abc123"); + jsonObject.put("isCreateTable", "false"); + // 多少条数据写入一次 + jsonObject.put("batchSize", "1"); + jsonObject.put("tableName", "dirtyData"); + + return jsonObject.toJSONString(); + + } + private static String readSQL(String sqlPath) { try { byte[] array = Files.readAllBytes(Paths.get(sqlPath)); diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java index df8293522..3f1bdebb4 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.sink.mongo; import com.dtstack.flink.sql.outputformat.AbstractDtRichOutputFormat; +import com.dtstack.flink.sql.util.SampleUtils; import com.mongodb.MongoClient; import com.mongodb.MongoClientURI; import com.mongodb.client.MongoCollection; @@ -70,7 +71,7 @@ public void open(int taskNumber, int numTasks) throws IOException { } @Override - public void writeRecord(Tuple2 tuple2) throws IOException { + public void writeRecord(Tuple2 tuple2) { Tuple2 tupleTrans = tuple2; Boolean retract = tupleTrans.getField(0); @@ -102,9 +103,7 @@ public void writeRecord(Tuple2 tuple2) throws IOException { dbCollection.insertOne(doc); } - if (outRecords.getCount() % ROW_PRINT_FREQUENCY == 0){ - LOG.info(record.toString()); - } + SampleUtils.samplingSinkPrint(samplingIntervalCount, LOG, outRecords.getCount(), record.toString()); outRecords.inc(); } diff --git a/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java b/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java index b0a38e344..4488ca7d7 100644 --- a/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java +++ b/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java @@ -19,7 +19,9 @@ package com.dtstack.flink.sql.side.mysql.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.dtstack.flink.sql.table.AbstractTableInfo; import java.util.Map; @@ -38,7 +40,8 @@ public class MysqlSideParser extends RdbSideParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - AbstractTableInfo mysqlTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + props.put(JdbcCheckKeys.DRIVER_NAME, "com.mysql.jdbc.Driver"); + RdbSideTableInfo mysqlTableInfo = (RdbSideTableInfo) super.getTableInfo(tableName, fieldsInfo, props); mysqlTableInfo.setType(CURR_TYPE); return mysqlTableInfo; } diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java index 0a1749f04..3b1d260d7 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java @@ -58,6 +58,7 @@ public JDBCUpsertOutputFormat getOutputFormat() { .setKeyFields(primaryKeys) .setAllReplace(allReplace) .setUpdateMode(updateMode) + .setName(name) .build(); } } diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java index 49105a7a8..8359fa0ff 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.sink.mysql.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -37,6 +38,7 @@ public class MysqlSinkParser extends RdbSinkParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "com.mysql.jdbc.Driver"); AbstractTableInfo mysqlTableInfo = super.getTableInfo(tableName, fieldsInfo, props); mysqlTableInfo.setType(CURR_TYPE); return mysqlTableInfo; diff --git a/oceanbase/oceanbase-side/oceanbase-side-core/src/main/java/com/dtstack/flink/sql/side/oceanbase/table/OceanbaseSideParser.java b/oceanbase/oceanbase-side/oceanbase-side-core/src/main/java/com/dtstack/flink/sql/side/oceanbase/table/OceanbaseSideParser.java index a5c6324a9..aad37ae2f 100644 --- a/oceanbase/oceanbase-side/oceanbase-side-core/src/main/java/com/dtstack/flink/sql/side/oceanbase/table/OceanbaseSideParser.java +++ b/oceanbase/oceanbase-side/oceanbase-side-core/src/main/java/com/dtstack/flink/sql/side/oceanbase/table/OceanbaseSideParser.java @@ -17,6 +17,7 @@ */ package com.dtstack.flink.sql.side.oceanbase.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -32,6 +33,7 @@ public class OceanbaseSideParser extends RdbSideParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "com.mysql.jdbc.Driver"); AbstractTableInfo oceanbaseTableInfo = super.getTableInfo(tableName, fieldsInfo, props); oceanbaseTableInfo.setType(CURRENT_TYPE); return oceanbaseTableInfo; diff --git a/oceanbase/oceanbase-sink/src/main/java/com/dtstack/flink/sql/sink/oceanbase/table/OceanbaseSinkParser.java b/oceanbase/oceanbase-sink/src/main/java/com/dtstack/flink/sql/sink/oceanbase/table/OceanbaseSinkParser.java index af1a41d6b..96b783a6f 100644 --- a/oceanbase/oceanbase-sink/src/main/java/com/dtstack/flink/sql/sink/oceanbase/table/OceanbaseSinkParser.java +++ b/oceanbase/oceanbase-sink/src/main/java/com/dtstack/flink/sql/sink/oceanbase/table/OceanbaseSinkParser.java @@ -17,6 +17,7 @@ */ package com.dtstack.flink.sql.sink.oceanbase.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -32,6 +33,7 @@ public class OceanbaseSinkParser extends RdbSinkParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "com.mysql.jdbc.Driver"); AbstractTableInfo oceanbaseTableInfo = super.getTableInfo(tableName, fieldsInfo, props); diff --git a/oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.java b/oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.java index 897b77d30..59fdd795d 100644 --- a/oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.java +++ b/oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.java @@ -17,6 +17,7 @@ */ package com.dtstack.flink.sql.side.oracle.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -28,6 +29,7 @@ public class OracleSideParser extends RdbSideParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "oracle.jdbc.driver.OracleDriver"); AbstractTableInfo oracleTableInfo = super.getTableInfo(tableName, fieldsInfo, props); oracleTableInfo.setType(CURR_TYPE); return oracleTableInfo; diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java index 1732c8cc2..2eeee3fb2 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java @@ -17,6 +17,7 @@ */ package com.dtstack.flink.sql.sink.oracle.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -34,6 +35,7 @@ public class OracleSinkParser extends RdbSinkParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "oracle.jdbc.driver.OracleDriver"); AbstractTableInfo oracleTableInfo = super.getTableInfo(tableName, fieldsInfo, props); oracleTableInfo.setType(CURR_TYPE); return oracleTableInfo; diff --git a/polardb/polardb-side/polardb-side-core/src/main/java/com/dtstack/flink/sql/side/polardb/table/PolardbSideParser.java b/polardb/polardb-side/polardb-side-core/src/main/java/com/dtstack/flink/sql/side/polardb/table/PolardbSideParser.java index 71aad7cbb..4e2683b14 100644 --- a/polardb/polardb-side/polardb-side-core/src/main/java/com/dtstack/flink/sql/side/polardb/table/PolardbSideParser.java +++ b/polardb/polardb-side/polardb-side-core/src/main/java/com/dtstack/flink/sql/side/polardb/table/PolardbSideParser.java @@ -18,6 +18,7 @@ */ package com.dtstack.flink.sql.side.polardb.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -26,6 +27,7 @@ /** * Date: 2019/12/20 * Company: www.dtstack.com + * * @author yinxi */ public class PolardbSideParser extends RdbSideParser { @@ -33,6 +35,7 @@ public class PolardbSideParser extends RdbSideParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "com.mysql.cj.jdbc.Driver"); AbstractTableInfo mysqlTableInfo = super.getTableInfo(tableName, fieldsInfo, props); mysqlTableInfo.setType(CURR_TYPE); return mysqlTableInfo; diff --git a/polardb/polardb-sink/src/main/java/com/dtstack/flink/sql/sink/polardb/table/PolardbSinkParser.java b/polardb/polardb-sink/src/main/java/com/dtstack/flink/sql/sink/polardb/table/PolardbSinkParser.java index b4f02665e..80d62f3d0 100644 --- a/polardb/polardb-sink/src/main/java/com/dtstack/flink/sql/sink/polardb/table/PolardbSinkParser.java +++ b/polardb/polardb-sink/src/main/java/com/dtstack/flink/sql/sink/polardb/table/PolardbSinkParser.java @@ -17,6 +17,7 @@ */ package com.dtstack.flink.sql.sink.polardb.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -25,6 +26,7 @@ /** * Date: 2019/12/20 * Company: www.dtstack.com + * * @author yinxi */ public class PolardbSinkParser extends RdbSinkParser { @@ -32,6 +34,7 @@ public class PolardbSinkParser extends RdbSinkParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "com.mysql.cj.jdbc.Driver"); AbstractTableInfo polardbTableInfo = super.getTableInfo(tableName, fieldsInfo, props); polardbTableInfo.setType(CURR_TYPE); return polardbTableInfo; diff --git a/pom.xml b/pom.xml index c2f6e4394..d95474c65 100644 --- a/pom.xml +++ b/pom.xml @@ -36,7 +36,10 @@ elasticsearch6 cassandra kudu - aws + + + + dirtyData @@ -44,7 +47,8 @@ UTF-8 - 1.11.1 + 1.11.3 + 2.12 2.7.3 4.12 2.21.0 @@ -91,6 +95,11 @@ test ${jacoco.version} + + com.alibaba + fastjson + 1.2.70 + @@ -140,6 +149,21 @@ + + org.apache.maven.plugins + maven-clean-plugin + 3.1.0 + + + + ${basedir}/plugins + + + ${basedir}/sqlplugins + + + + diff --git a/postgresql/postgresql-side/postgresql-side-core/src/main/java/com/dtstack/flink/sql/side/postgresql/table/PostgresqlSideParser.java b/postgresql/postgresql-side/postgresql-side-core/src/main/java/com/dtstack/flink/sql/side/postgresql/table/PostgresqlSideParser.java index be0c5ea8b..fe1afb578 100644 --- a/postgresql/postgresql-side/postgresql-side-core/src/main/java/com/dtstack/flink/sql/side/postgresql/table/PostgresqlSideParser.java +++ b/postgresql/postgresql-side/postgresql-side-core/src/main/java/com/dtstack/flink/sql/side/postgresql/table/PostgresqlSideParser.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.side.postgresql.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -38,6 +39,7 @@ public class PostgresqlSideParser extends RdbSideParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "org.postgresql.Driver"); AbstractTableInfo pgTableInfo = super.getTableInfo(tableName, fieldsInfo, props); pgTableInfo.setType(CURR_TYPE); return pgTableInfo; diff --git a/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlSinkParser.java b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlSinkParser.java index e94cfc8a0..9e9db93ee 100644 --- a/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlSinkParser.java +++ b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlSinkParser.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.sink.postgresql.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -36,6 +37,7 @@ public class PostgresqlSinkParser extends RdbSinkParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "org.postgresql.Driver"); AbstractTableInfo pgTableInfo = super.getTableInfo(tableName, fieldsInfo, props); pgTableInfo.setType(CURR_TYPE); return pgTableInfo; diff --git a/rdb/pom.xml b/rdb/pom.xml index 5cd5d4187..3756d91d3 100644 --- a/rdb/pom.xml +++ b/rdb/pom.xml @@ -15,6 +15,7 @@ rdb-side rdb-sink + rdb-core pom diff --git a/rdb/rdb-core/pom.xml b/rdb/rdb-core/pom.xml new file mode 100644 index 000000000..32ad8b6c0 --- /dev/null +++ b/rdb/rdb-core/pom.xml @@ -0,0 +1,20 @@ + + + + sql.rdb + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + rdb-core + sql.core.rdb + + + 8 + 8 + + + \ No newline at end of file diff --git a/rdb/rdb-core/src/main/java/com/dtstack/flink/sql/core/rdb/JdbcCheckKeys.java b/rdb/rdb-core/src/main/java/com/dtstack/flink/sql/core/rdb/JdbcCheckKeys.java new file mode 100644 index 000000000..6eef874fa --- /dev/null +++ b/rdb/rdb-core/src/main/java/com/dtstack/flink/sql/core/rdb/JdbcCheckKeys.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flink.sql.core.rdb; + +/** + * @author tiezhu + * Date 2020-12-25 + * Company dtstack + */ +public class JdbcCheckKeys { + public static final String DRIVER_NAME = "driverName"; + public static final String URL_KEY = "url"; + public static final String USER_NAME_KEY = "userName"; + public static final String PASSWORD_KEY = "password"; + public static final String TABLE_TYPE_KEY = "tableType"; + public static final String NEED_CHECK = "needCheck"; + public static final String SCHEMA_KEY = "schema"; + public static final String TABLE_NAME_KEY = "tableName"; + // create 语句中的name + public static final String OPERATION_NAME_KEY = "operationName"; + // 用来检查update、replace等操作的column + public static final String COLUMN_KEY = "column"; + public static final String TABLE_INFO_KEY = "tableInfo"; +} diff --git a/rdb/rdb-core/src/main/java/com/dtstack/flink/sql/core/rdb/JdbcResourceCheck.java b/rdb/rdb-core/src/main/java/com/dtstack/flink/sql/core/rdb/JdbcResourceCheck.java new file mode 100644 index 000000000..2b28258a7 --- /dev/null +++ b/rdb/rdb-core/src/main/java/com/dtstack/flink/sql/core/rdb/JdbcResourceCheck.java @@ -0,0 +1,139 @@ +/* + * 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.dtstack.flink.sql.core.rdb; + +import com.dtstack.flink.sql.core.rdb.util.JdbcConnectUtil; +import com.dtstack.flink.sql.resource.ResourceCheck; +import org.apache.commons.lang.StringUtils; +import org.apache.flink.runtime.execution.SuppressRestartsException; + +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * @author: chuixue + * @create: 2020-12-08 17:24 + * @description:jdbc资源检测 + **/ +public class JdbcResourceCheck extends ResourceCheck { + private static final String DELETE_STR = "delete"; + private static final String SELECT_STR = "select"; + private static final String INSERT_STR = "insert"; + private static final String UPDATE_STR = "update"; + private static final String REPLACE_STR = "replace"; + + private static final String CHECK_SELECT_SQL = "select 1 from $table where 1=2;"; + private static final String CHECK_DELETE_SQL = "delete from $table where 1=3;"; + private static final String CHECK_INSERT_SQL = "insert into $table (select * from $table where 1=2);"; + + private static final Map PRIVILEGE_SQL_MAP = new HashMap<>(); + private static final JdbcResourceCheck Instance = new JdbcResourceCheck(); + + static { + PRIVILEGE_SQL_MAP.put(SELECT_STR, CHECK_SELECT_SQL); + PRIVILEGE_SQL_MAP.put(DELETE_STR, CHECK_DELETE_SQL); + PRIVILEGE_SQL_MAP.put(INSERT_STR, CHECK_INSERT_SQL); + } + + private JdbcResourceCheck() { + } + + public static JdbcResourceCheck getInstance() { + return Instance; + } + + @Override + public void checkResourceStatus(Map checkProperties) { + if (!NEED_CHECK || !Boolean.parseBoolean(checkProperties.get(JdbcCheckKeys.NEED_CHECK))) { + LOG.warn("Ignore checking [{}] type data source , tableName is [{}]." + , checkProperties.get(JdbcCheckKeys.TABLE_TYPE_KEY) + , checkProperties.get(JdbcCheckKeys.TABLE_NAME_KEY)); + return; + } + + LOG.info("start checking [{}] type data source , tableName is [{}]." + , checkProperties.get(JdbcCheckKeys.TABLE_TYPE_KEY) + , checkProperties.get(JdbcCheckKeys.TABLE_NAME_KEY)); + List privilegeList = new ArrayList<>(); + if (checkProperties.get(TABLE_TYPE_KEY).equalsIgnoreCase(SIDE_STR)) { + privilegeList.add(SELECT_STR); + } + if (checkProperties.get(TABLE_TYPE_KEY).equalsIgnoreCase(SINK_STR)) { + privilegeList.add(INSERT_STR); + // privilegeList.add(DELETE_STR); + } + checkPrivilege( + checkProperties.get(JdbcCheckKeys.DRIVER_NAME) + , checkProperties.get(JdbcCheckKeys.URL_KEY) + , checkProperties.get(JdbcCheckKeys.USER_NAME_KEY) + , checkProperties.get(JdbcCheckKeys.PASSWORD_KEY) + , checkProperties.get(JdbcCheckKeys.TABLE_NAME_KEY) + , checkProperties.get(JdbcCheckKeys.SCHEMA_KEY) + , privilegeList + ); + LOG.info("data source is available and user [{}] has the corresponding permissions {} for [{}] type , tableName is [{}]" + , checkProperties.get(JdbcCheckKeys.USER_NAME_KEY) + , privilegeList.toString() + , checkProperties.get(JdbcCheckKeys.TABLE_TYPE_KEY) + , checkProperties.get(JdbcCheckKeys.TABLE_NAME_KEY)); + } + + public void checkPrivilege( + String driverName + , String url + , String userName + , String password + , String tableName + , String schema + , List privilegeList) { + Connection connection = + JdbcConnectUtil.getConnectWithRetry(driverName, url, userName, password); + Statement statement = null; + String tableInfo = Objects.isNull(schema) ? tableName : schema + "." + tableName; + String privilege = null; + try { + statement = connection.createStatement(); + for (String s : privilegeList) { + privilege = s; + statement.execute( + StringUtils.replace( + PRIVILEGE_SQL_MAP.get(privilege.toLowerCase()), + "$table", + tableName + ) + ); + } + } catch (SQLException sqlException) { + if (sqlException.getMessage().contains("command denied")) { + throw new SuppressRestartsException(new Throwable( + String.format("user [%s] don't have [%s] privilege of table [%s]", userName, privilege, tableInfo))); + } + + throw new SuppressRestartsException(new IllegalArgumentException(sqlException.getMessage())); + } finally { + JdbcConnectUtil.closeConnectionResource(null, statement, connection, false); + } + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/util/JDBCUtils.java b/rdb/rdb-core/src/main/java/com/dtstack/flink/sql/core/rdb/util/JdbcConnectUtil.java similarity index 50% rename from core/src/main/java/com/dtstack/flink/sql/util/JDBCUtils.java rename to rdb/rdb-core/src/main/java/com/dtstack/flink/sql/core/rdb/util/JdbcConnectUtil.java index 1d87a1c75..a5a33d51b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/JDBCUtils.java +++ b/rdb/rdb-core/src/main/java/com/dtstack/flink/sql/core/rdb/util/JdbcConnectUtil.java @@ -16,9 +16,12 @@ * limitations under the License. */ +package com.dtstack.flink.sql.core.rdb.util; -package com.dtstack.flink.sql.util; - +import com.dtstack.flink.sql.classloader.ClassLoaderManager; +import com.dtstack.flink.sql.util.ThreadUtil; +import com.google.common.base.Preconditions; +import org.apache.flink.runtime.execution.SuppressRestartsException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,31 +32,16 @@ import java.sql.Statement; import java.util.Objects; -public class JDBCUtils { - private static final Logger LOG = LoggerFactory.getLogger(JDBCUtils.class); - - private static final Object LOCK = new Object(); - - public static void forName(String clazz, ClassLoader classLoader) { - synchronized (LOCK) { - try { - Class.forName(clazz, true, classLoader); - DriverManager.setLoginTimeout(10); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - } - - - public synchronized static void forName(String clazz) { - try { - Class driverClass = Class.forName(clazz); - driverClass.newInstance(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } +/** + * @author tiezhu + * Date 2020-12-25 + * Company dtstack + */ +public class JdbcConnectUtil { + private static final int DEFAULT_RETRY_NUM = 3; + private static final long DEFAULT_RETRY_TIME_WAIT = 3L; + private static final int DEFAULT_VALID_TIME = 10; + private static final Logger LOG = LoggerFactory.getLogger(JdbcConnectUtil.class); /** * 关闭连接资源 @@ -61,9 +49,13 @@ public synchronized static void forName(String clazz) { * @param rs ResultSet * @param stmt Statement * @param conn Connection - * @param commit + * @param commit 手动提交事务 */ - public static void closeConnectionResource(ResultSet rs, Statement stmt, Connection conn, boolean commit) { + public static void closeConnectionResource( + ResultSet rs + , Statement stmt + , Connection conn + , boolean commit) { if (Objects.nonNull(rs)) { try { rs.close(); @@ -102,7 +94,7 @@ public static void closeConnectionResource(ResultSet rs, Statement stmt, Connect */ public static void commit(Connection conn) { try { - if (!conn.isClosed() && !conn.getAutoCommit()) { + if (!conn.isClosed() && conn.isValid(DEFAULT_VALID_TIME) && !conn.getAutoCommit()) { conn.commit(); } } catch (SQLException e) { @@ -117,7 +109,7 @@ public static void commit(Connection conn) { */ public static void rollBack(Connection conn) { try { - if (!conn.isClosed() && !conn.getAutoCommit()) { + if (!conn.isClosed() && conn.isValid(DEFAULT_VALID_TIME) && !conn.getAutoCommit()) { conn.rollback(); } } catch (SQLException e) { @@ -125,4 +117,44 @@ public static void rollBack(Connection conn) { } } + /** + * get connect from datasource and retry when failed. + * + * @param driverName driver name for rdb datasource + * @param url connect url + * @param userName connect user name + * @param password password for user name + * @return a valid connection + */ + public static Connection getConnectWithRetry( + String driverName + , String url + , String userName + , String password) { + String errorMessage = "\nGet connect failed with properties: \nurl: " + url + + (Objects.isNull(userName) ? "" : "\nuserName: " + userName + + "\nerror message: "); + String errorCause = null; + + ClassLoaderManager.forName(driverName, JdbcConnectUtil.class.getClassLoader()); + Preconditions.checkNotNull(url, "url can't be null!"); + + for (int i = 0; i < DEFAULT_RETRY_NUM; i++) { + try { + return Objects.isNull(userName) ? + DriverManager.getConnection(url) : DriverManager.getConnection(url, userName, password); + } catch (Exception e) { + if (Objects.isNull(e.getCause())) { + errorCause = e.getMessage(); + } else { + errorCause = e.getCause().toString(); + } + + LOG.warn(errorMessage + errorCause); + LOG.warn("Connect will retry after [{}] s. Retry time [{}] ...", DEFAULT_RETRY_TIME_WAIT, i + 1); + ThreadUtil.sleepSeconds(DEFAULT_RETRY_TIME_WAIT); + } + } + throw new SuppressRestartsException(new Throwable(errorMessage + errorCause)); + } } diff --git a/rdb/rdb-side/pom.xml b/rdb/rdb-side/pom.xml index c7350877f..3665aa189 100644 --- a/rdb/rdb-side/pom.xml +++ b/rdb/rdb-side/pom.xml @@ -35,6 +35,12 @@ ${vertx.version} + + com.dtstack.flink + sql.core.rdb + 1.0-SNAPSHOT + + diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/AbstractRdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/AbstractRdbAllReqRow.java index d35034598..9f230cb47 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/AbstractRdbAllReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/AbstractRdbAllReqRow.java @@ -18,11 +18,12 @@ package com.dtstack.flink.sql.side.rdb.all; +import com.dtstack.flink.sql.core.rdb.JdbcResourceCheck; +import com.dtstack.flink.sql.core.rdb.util.JdbcConnectUtil; import com.dtstack.flink.sql.side.BaseAllReqRow; import com.dtstack.flink.sql.side.BaseSideInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.dtstack.flink.sql.side.rdb.util.SwitchUtil; -import com.dtstack.flink.sql.util.JDBCUtils; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.calcite.sql.JoinType; @@ -66,8 +67,8 @@ public abstract class AbstractRdbAllReqRow extends BaseAllReqRow { private static final int CONN_RETRY_NUM = 3; private static final int DEFAULT_FETCH_SIZE = 1000; - - private AtomicReference>>> cacheRef = new AtomicReference<>(); + private static volatile boolean resourceCheck = true; + private final AtomicReference>>> cacheRef = new AtomicReference<>(); public AbstractRdbAllReqRow(BaseSideInfo sideInfo) { super(sideInfo); @@ -75,8 +76,14 @@ public AbstractRdbAllReqRow(BaseSideInfo sideInfo) { @Override public void open(Configuration parameters) throws Exception { - super.open(parameters); RdbSideTableInfo tableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); + synchronized (AbstractRdbAllReqRow.class) { + if (resourceCheck) { + resourceCheck = false; + JdbcResourceCheck.getInstance().checkResourceStatus(tableInfo.getCheckProperties()); + } + } + super.open(parameters); LOG.info("rdb dim table config info: {} ", tableInfo.toString()); } @@ -114,7 +121,7 @@ public void flatMap(Row value, Collector out) throws Exception { } String cacheKey = inputParams.stream() - .map(e -> String.valueOf(e)) + .map(String::valueOf) .collect(Collectors.joining("_")); List> cacheList = cacheRef.get().get(cacheKey); @@ -139,7 +146,7 @@ protected Object dealTimeAttributeType(Class entry, O boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(entry); if (obj instanceof LocalDateTime && isTimeIndicatorTypeInfo) { //去除上一层OutputRowtimeProcessFunction 调用时区导致的影响 - obj = ((Timestamp) obj).getTime() + (long)LOCAL_TZ.getOffset(((Timestamp) obj).getTime()); + obj = ((Timestamp) obj).getTime() + (long) LOCAL_TZ.getOffset(((Timestamp) obj).getTime()); } return obj; } @@ -192,13 +199,13 @@ private void queryAndFillData(Map>> tmpCache, C String cacheKey = sideInfo.getEqualFieldList().stream() .map(oneRow::get) - .map(e -> String.valueOf(e)) + .map(String::valueOf) .collect(Collectors.joining("_")); tmpCache.computeIfAbsent(cacheKey, key -> Lists.newArrayList()) .add(oneRow); } - JDBCUtils.closeConnectionResource(resultSet, statement, connection, false); + JdbcConnectUtil.closeConnectionResource(resultSet, statement, connection, false); } public int getFetchSize() { diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/AbstractRdbTableFunction.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/AbstractRdbTableFunction.java index 36eec90a9..ae8a14612 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/AbstractRdbTableFunction.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/AbstractRdbTableFunction.java @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.side.rdb.all; +import com.dtstack.flink.sql.core.rdb.JdbcResourceCheck; import com.dtstack.flink.sql.side.BaseSideInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.dtstack.flink.sql.side.rdb.util.SwitchUtil; @@ -41,6 +42,7 @@ **/ abstract public class AbstractRdbTableFunction extends BaseTableFunction { private static final Logger LOG = LoggerFactory.getLogger(AbstractRdbTableFunction.class); + private static volatile boolean resourceCheck = true; public AbstractRdbTableFunction(BaseSideInfo sideInfo) { super(sideInfo); @@ -48,8 +50,14 @@ public AbstractRdbTableFunction(BaseSideInfo sideInfo) { @Override public void open(FunctionContext context) throws Exception { - super.open(context); RdbSideTableInfo tableInfo = (RdbSideTableInfo) sideTableInfo; + synchronized (AbstractRdbAllReqRow.class) { + if (resourceCheck) { + resourceCheck = false; + JdbcResourceCheck.getInstance().checkResourceStatus(tableInfo.getCheckProperties()); + } + } + super.open(context); LOG.info("rdb dim table config info: {} ", tableInfo.toString()); } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index fb4b07da8..dad34cd5e 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.side.rdb.async; +import com.dtstack.flink.sql.core.rdb.JdbcResourceCheck; import com.dtstack.flink.sql.enums.ECacheContentType; import com.dtstack.flink.sql.factory.DTThreadFactory; import com.dtstack.flink.sql.side.BaseAsyncReqRow; @@ -28,6 +29,7 @@ import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.dtstack.flink.sql.side.rdb.util.SwitchUtil; import com.dtstack.flink.sql.util.DateUtil; +import com.dtstack.flink.sql.util.ThreadUtil; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import io.vertx.core.json.JsonArray; @@ -62,47 +64,41 @@ public class RdbAsyncReqRow extends BaseAsyncReqRow { - private static final long serialVersionUID = 2098635244857937720L; - - private static final Logger LOG = LoggerFactory.getLogger(RdbAsyncReqRow.class); - public final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 1; - public final static int DEFAULT_VERTX_WORKER_POOL_SIZE = Runtime.getRuntime().availableProcessors() * 2; - public final static int DEFAULT_DB_CONN_POOL_SIZE = DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE + DEFAULT_VERTX_WORKER_POOL_SIZE; - - public final static int MAX_DB_CONN_POOL_SIZE_LIMIT = 20; - + public final static int MAX_DB_CONN_POOL_SIZE_LIMIT = 5; public final static int DEFAULT_IDLE_CONNECTION_TEST_PEROID = 60; - public final static boolean DEFAULT_TEST_CONNECTION_ON_CHECKIN = true; - public final static String DT_PROVIDER_CLASS = "com.dtstack.flink.sql.side.rdb.provider.DTC3P0DataSourceProvider"; - public final static String PREFERRED_TEST_QUERY_SQL = "SELECT 1 FROM DUAL"; - + private static final long serialVersionUID = 2098635244857937720L; + private static final Logger LOG = LoggerFactory.getLogger(RdbAsyncReqRow.class); + private final static int MAX_TASK_QUEUE_SIZE = 100000; + private static volatile boolean resourceCheck = true; + private final AtomicBoolean connectionStatus = new AtomicBoolean(true); private transient SQLClient rdbSqlClient; - - private AtomicBoolean connectionStatus = new AtomicBoolean(true); - private transient ThreadPoolExecutor executor; - private final static int MAX_TASK_QUEUE_SIZE = 100000; - + public RdbAsyncReqRow(BaseSideInfo sideInfo) { + super(sideInfo); + init(sideInfo); + } @Override public void open(Configuration parameters) throws Exception { + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); + synchronized (RdbAsyncReqRow.class) { + if (resourceCheck) { + resourceCheck = false; + JdbcResourceCheck.getInstance().checkResourceStatus(rdbSideTableInfo.getCheckProperties()); + } + } super.open(parameters); executor = new ThreadPoolExecutor(MAX_DB_CONN_POOL_SIZE_LIMIT, MAX_DB_CONN_POOL_SIZE_LIMIT, 0, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>(MAX_TASK_QUEUE_SIZE), new DTThreadFactory("rdbAsyncExec"), new ThreadPoolExecutor.CallerRunsPolicy()); } - public RdbAsyncReqRow(BaseSideInfo sideInfo) { - super(sideInfo); - init(sideInfo); - } - protected void init(BaseSideInfo sideInfo) { RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); int defaultAsyncPoolSize = Math.min(MAX_DB_CONN_POOL_SIZE_LIMIT, DEFAULT_DB_CONN_POOL_SIZE); @@ -111,7 +107,8 @@ protected void init(BaseSideInfo sideInfo) { } @Override - protected void preInvoke(Row input, ResultFuture resultFuture) { } + protected void preInvoke(Row input, ResultFuture resultFuture) { + } @Override public void handleAsyncInvoke(Map inputParams, Row input, ResultFuture resultFuture) throws Exception { @@ -127,7 +124,7 @@ public void handleAsyncInvoke(Map inputParams, Row input, Result executor.execute(() -> connectWithRetry(params, input, resultFuture, rdbSqlClient)); } - protected void asyncQueryData( Map inputParams, + protected void asyncQueryData(Map inputParams, Row input, ResultFuture resultFuture, SQLClient rdbSqlClient, @@ -135,21 +132,22 @@ protected void asyncQueryData( Map inputParams, AtomicBoolean finishFlag, CountDownLatch latch) { doAsyncQueryData(inputParams, - input, resultFuture, - rdbSqlClient, - failCounter, - finishFlag, - latch); + input, + resultFuture, + rdbSqlClient, + failCounter, + finishFlag, + latch); } final protected void doAsyncQueryData( - Map inputParams, - Row input, - ResultFuture resultFuture, - SQLClient rdbSqlClient, - AtomicLong failCounter, - AtomicBoolean finishFlag, - CountDownLatch latch) { + Map inputParams, + Row input, + ResultFuture resultFuture, + SQLClient rdbSqlClient, + AtomicLong failCounter, + AtomicBoolean finishFlag, + CountDownLatch latch) { rdbSqlClient.getConnection(conn -> { try { if (conn.failed()) { @@ -182,12 +180,14 @@ private void connectWithRetry(Map inputParams, Row input, Result while (!finishFlag.get()) { try { CountDownLatch latch = new CountDownLatch(1); - asyncQueryData(inputParams, - input, resultFuture, - rdbSqlClient, - failCounter, - finishFlag, - latch); + asyncQueryData( + inputParams, + input, + resultFuture, + rdbSqlClient, + failCounter, + finishFlag, + latch); try { latch.await(); } catch (InterruptedException e) { @@ -199,11 +199,7 @@ private void connectWithRetry(Map inputParams, Row input, Result connectionStatus.set(false); } if (!finishFlag.get()) { - try { - Thread.sleep(3000); - } catch (Exception e) { - LOG.error("", e); - } + ThreadUtil.sleepSeconds(ThreadUtil.DEFAULT_SLEEP_TIME); } } } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncTableFunction.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncTableFunction.java index a7db919ac..fa2f50765 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncTableFunction.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncTableFunction.java @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.side.rdb.async; +import com.dtstack.flink.sql.core.rdb.JdbcResourceCheck; import com.dtstack.flink.sql.enums.ECacheContentType; import com.dtstack.flink.sql.factory.DTThreadFactory; import com.dtstack.flink.sql.side.BaseSideInfo; @@ -88,8 +89,17 @@ public class RdbAsyncTableFunction extends BaseAsyncTableFunction { private final static int MAX_TASK_QUEUE_SIZE = 100000; + private static volatile boolean resourceCheck = true; + @Override public void open(FunctionContext context) throws Exception { + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); + synchronized (RdbAsyncReqRow.class) { + if (resourceCheck) { + resourceCheck = false; + JdbcResourceCheck.getInstance().checkResourceStatus(rdbSideTableInfo.getCheckProperties()); + } + } super.open(context); executor = new ThreadPoolExecutor(MAX_DB_CONN_POOL_SIZE_LIMIT, MAX_DB_CONN_POOL_SIZE_LIMIT, 0, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>(MAX_TASK_QUEUE_SIZE), new DTThreadFactory("rdbAsyncExec"), new ThreadPoolExecutor.CallerRunsPolicy()); diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java index 79942ae11..e18dba5f9 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java @@ -25,6 +25,8 @@ import java.util.Map; +import static com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo.DRIVER_NAME; + /** * Reason: * Date: 2018/11/26 @@ -48,6 +50,9 @@ public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map * 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. @@ -17,8 +17,14 @@ */ package com.dtstack.flink.sql.side.rdb.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; +import com.dtstack.flink.sql.core.rdb.JdbcResourceCheck; +import com.dtstack.flink.sql.resource.ResourceCheck; import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; + +import java.util.Map; /** * Reason: @@ -29,17 +35,19 @@ */ public class RdbSideTableInfo extends AbstractSideTableInfo { - private static final long serialVersionUID = -1L; - + public static final String DRIVER_NAME = "driverName"; public static final String URL_KEY = "url"; - public static final String TABLE_NAME_KEY = "tableName"; - public static final String USER_NAME_KEY = "userName"; - public static final String PASSWORD_KEY = "password"; - public static final String SCHEMA_KEY = "schema"; + private static final long serialVersionUID = -1L; + private String driverName; + private String url; + private String tableName; + private String userName; + private String password; + private String schema; @Override public boolean check() { @@ -49,19 +57,13 @@ public boolean check() { Preconditions.checkNotNull(password, "rdb of password is required"); Preconditions.checkArgument(getFieldList().size() == getFieldExtraInfoList().size(), "fields and fieldExtraInfoList attributes must be the same length"); + // 是否在client端快速检测表资源是否可用,这样在client能访问资源的情况下快速失败,不用提交到集群检测 + if (getFastCheck()) { + JdbcResourceCheck.getInstance().checkResourceStatus(this.getCheckProperties()); + } return true; } - private String url; - - private String tableName; - - private String userName; - - private String password; - - private String schema; - public String getSchema() { return schema; } @@ -102,6 +104,14 @@ public void setPassword(String password) { this.password = password; } + public String getDriverName() { + return driverName; + } + + public void setDriverName(String driverName) { + this.driverName = driverName; + } + @Override public String toString() { String cacheInfo = super.toString(); @@ -109,8 +119,23 @@ public String toString() { "url='" + url + '\'' + ", tableName='" + tableName + '\'' + ", schema='" + schema + '\'' + + ", driverName='" + driverName + '\'' + '}'; return cacheInfo + " , " + connectionInfo; } + @Override + public Map buildCheckProperties() { + Map properties = Maps.newHashMap(); + properties.put(JdbcCheckKeys.DRIVER_NAME, getDriverName()); + properties.put(JdbcCheckKeys.URL_KEY, getUrl()); + properties.put(JdbcCheckKeys.USER_NAME_KEY, getUserName()); + properties.put(JdbcCheckKeys.PASSWORD_KEY, getPassword()); + properties.put(JdbcCheckKeys.SCHEMA_KEY, getSchema()); + properties.put(JdbcCheckKeys.TABLE_NAME_KEY, getTableName()); + properties.put(JdbcCheckKeys.OPERATION_NAME_KEY, getName()); + properties.put(JdbcCheckKeys.TABLE_TYPE_KEY, "side"); + properties.put(JdbcCheckKeys.NEED_CHECK, String.valueOf(ResourceCheck.NEED_CHECK)); + return properties; + } } diff --git a/rdb/rdb-sink/pom.xml b/rdb/rdb-sink/pom.xml index 176615769..e0a0d1078 100644 --- a/rdb/rdb-sink/pom.xml +++ b/rdb/rdb-sink/pom.xml @@ -15,4 +15,12 @@ rdb-sink jar + + + com.dtstack.flink + sql.core.rdb + 1.0-SNAPSHOT + + + \ No newline at end of file diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/AbstractRdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/AbstractRdbSink.java index be283262a..11978772e 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/AbstractRdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/AbstractRdbSink.java @@ -46,6 +46,8 @@ * @author maqi */ public abstract class AbstractRdbSink implements RetractStreamTableSink, Serializable, IStreamSinkGener { + protected String name; + protected String dbUrl; protected String userName; @@ -93,6 +95,7 @@ public AbstractRdbSink(JDBCDialect jdbcDialect) { @Override public AbstractRdbSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { RdbTableInfo rdbTableInfo = (RdbTableInfo) targetTableInfo; + this.name = rdbTableInfo.getName(); this.batchNum = rdbTableInfo.getBatchSize() == null ? batchNum : rdbTableInfo.getBatchSize(); this.batchWaitInterval = rdbTableInfo.getBatchWaitInterval() == null ? batchWaitInterval : rdbTableInfo.getBatchWaitInterval(); diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/JDBCOptions.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/JDBCOptions.java index f476bfadf..eacdc9b37 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/JDBCOptions.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/JDBCOptions.java @@ -19,8 +19,12 @@ package com.dtstack.flink.sql.sink.rdb; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; +import com.dtstack.flink.sql.resource.ResourceCheck; import com.dtstack.flink.sql.sink.rdb.dialect.JDBCDialect; +import com.google.common.collect.Maps; +import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -76,6 +80,20 @@ public String getSchema() { return schema; } + public Map buildCheckProperties() { + Map properties = Maps.newHashMap(); + properties.put(JdbcCheckKeys.DRIVER_NAME, getDriverName()); + properties.put(JdbcCheckKeys.URL_KEY, getDbUrl()); + properties.put(JdbcCheckKeys.USER_NAME_KEY, getUsername()); + properties.put(JdbcCheckKeys.PASSWORD_KEY, getPassword()); + properties.put(JdbcCheckKeys.SCHEMA_KEY, getSchema()); + properties.put(JdbcCheckKeys.TABLE_NAME_KEY, getTableName()); + properties.put(JdbcCheckKeys.OPERATION_NAME_KEY, "jdbcOutputFormat"); + properties.put(JdbcCheckKeys.TABLE_TYPE_KEY, "sink"); + properties.put(JdbcCheckKeys.NEED_CHECK, String.valueOf(ResourceCheck.NEED_CHECK)); + return properties; + } + public static Builder builder() { return new Builder(); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/AbstractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/AbstractJDBCOutputFormat.java index 2a3ce5e90..f8556c0ab 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/AbstractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/AbstractJDBCOutputFormat.java @@ -18,8 +18,8 @@ package com.dtstack.flink.sql.sink.rdb.format; +import com.dtstack.flink.sql.classloader.ClassLoaderManager; import com.dtstack.flink.sql.outputformat.AbstractDtRichOutputFormat; -import com.dtstack.flink.sql.util.JDBCUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; import org.slf4j.Logger; @@ -47,7 +47,7 @@ public abstract class AbstractJDBCOutputFormat extends AbstractDtRichOutputFo protected final String username; protected final String password; - private final String driverName; + protected final String driverName; protected final String dbURL; protected transient Connection connection; @@ -64,7 +64,7 @@ public void configure(Configuration parameters) { } protected void establishConnection() throws SQLException, ClassNotFoundException, IOException { - JDBCUtils.forName(driverName, getClass().getClassLoader()); + ClassLoaderManager.forName(driverName, getClass().getClassLoader()); if (username == null) { connection = DriverManager.getConnection(dbURL); } else { diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/JDBCUpsertOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/JDBCUpsertOutputFormat.java index 783c59e11..5626def9e 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/JDBCUpsertOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/JDBCUpsertOutputFormat.java @@ -19,24 +19,25 @@ package com.dtstack.flink.sql.sink.rdb.format; +import com.dtstack.flink.sql.core.rdb.JdbcResourceCheck; import com.dtstack.flink.sql.enums.EUpdateMode; import com.dtstack.flink.sql.factory.DTThreadFactory; import com.dtstack.flink.sql.sink.rdb.JDBCOptions; import com.dtstack.flink.sql.sink.rdb.dialect.JDBCDialect; +import com.dtstack.flink.sql.sink.rdb.writer.AbstractUpsertWriter; import com.dtstack.flink.sql.sink.rdb.writer.AppendOnlyWriter; import com.dtstack.flink.sql.sink.rdb.writer.JDBCWriter; -import com.dtstack.flink.sql.sink.rdb.writer.AbstractUpsertWriter; +import com.dtstack.flink.sql.util.SampleUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.types.Row; -import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.security.PrivilegedAction; import java.sql.SQLException; import java.util.List; +import java.util.Map; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; @@ -55,8 +56,7 @@ public class JDBCUpsertOutputFormat extends AbstractJDBCOutputFormat checkProperties; + private final int flushMaxSize; private final long flushIntervalMills; private final boolean allReplace; @@ -73,11 +75,13 @@ public class JDBCUpsertOutputFormat extends AbstractJDBCOutputFormat tuple2) throws IOException { + public synchronized void writeRecord(Tuple2 tuple2) { checkConnectionOpen(); try { - if (outRecords.getCount() % RECEIVEDATA_PRINT_FREQUENTY == 0 || LOG.isDebugEnabled()) { - LOG.info("Receive data : {}", tuple2); - } - // Receive data + SampleUtils.samplingSinkPrint(samplingIntervalCount, LOG, outRecords.getCount(), tuple2.toString()); outRecords.inc(); jdbcWriter.addRecord(tuple2); @@ -223,7 +232,9 @@ public synchronized void close() throws IOException { } try { - jdbcWriter.close(); + if(jdbcWriter != null){ + jdbcWriter.close(); + } } catch (SQLException e) { LOG.warn("Close JDBC writer failed.", e); } @@ -249,6 +260,7 @@ public static class Builder { protected boolean allReplace = DEFAULT_ALLREPLACE_VALUE; protected String updateMode; protected JDBCWriter jdbcWriter; + protected String name; /** * required, jdbc options. @@ -325,6 +337,11 @@ public Builder setJDBCWriter(JDBCWriter jdbcWriter) { return this; } + public Builder setName(String name) { + this.name = name; + return this; + } + /** * Finalizes the configuration and checks validity. * @@ -334,7 +351,7 @@ public JDBCUpsertOutputFormat build() { checkNotNull(options, "No options supplied."); checkNotNull(fieldNames, "No fieldNames supplied."); return new JDBCUpsertOutputFormat( - options, fieldNames, keyFields, partitionFields, fieldTypes, flushMaxSize, flushIntervalMills, allReplace, updateMode, jdbcWriter); + name, options, fieldNames, keyFields, partitionFields, fieldTypes, flushMaxSize, flushIntervalMills, allReplace, updateMode, jdbcWriter); } } } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java index aca7acb6c..ec613bd56 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java @@ -49,6 +49,9 @@ public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map buildCheckProperties() { + Map properties = Maps.newHashMap(); + + properties.put(JdbcCheckKeys.DRIVER_NAME, getDriverName()); + properties.put(JdbcCheckKeys.URL_KEY, getUrl()); + properties.put(JdbcCheckKeys.USER_NAME_KEY, getUserName()); + properties.put(JdbcCheckKeys.PASSWORD_KEY, getPassword()); + properties.put(JdbcCheckKeys.SCHEMA_KEY, getSchema()); + properties.put(JdbcCheckKeys.TABLE_NAME_KEY, getTableName()); + properties.put(JdbcCheckKeys.OPERATION_NAME_KEY, getName()); + properties.put(JdbcCheckKeys.TABLE_TYPE_KEY, "sink"); + properties.put(JdbcCheckKeys.NEED_CHECK, String.valueOf(ResourceCheck.NEED_CHECK)); + + return properties; + } } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/writer/AbstractUpsertWriter.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/writer/AbstractUpsertWriter.java index acb083312..b57b865e6 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/writer/AbstractUpsertWriter.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/writer/AbstractUpsertWriter.java @@ -69,7 +69,7 @@ public static AbstractUpsertWriter create( Arrays.stream(pkFields).map(f -> fieldTypes[f]).toArray(); String deleteSql = dialect.getDeleteStatement(schema, tableName, keyFields); - LOG.info("deleteSQL is :{}", deleteSql); + // LOG.info("deleteSQL is :{}", deleteSql); Optional upsertSql = dialect.getUpsertStatement(schema, tableName, fieldNames, keyFields, allReplace); LOG.info("execute UpsertStatement: {}", upsertSql.orElse("use UsingInsertUpdateStatement")); diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java index d57260e94..688503a92 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java @@ -36,12 +36,22 @@ import org.apache.flink.util.Collector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import redis.clients.jedis.*; +import redis.clients.jedis.HostAndPort; +import redis.clients.jedis.Jedis; +import redis.clients.jedis.JedisCluster; +import redis.clients.jedis.JedisCommands; +import redis.clients.jedis.JedisPool; +import redis.clients.jedis.JedisSentinelPool; import java.io.Closeable; import java.io.IOException; import java.sql.SQLException; -import java.util.*; +import java.util.Calendar; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.atomic.AtomicReference; /** * @author yanxi diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index da9dd85d1..e79144d31 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -19,7 +19,11 @@ package com.dtstack.flink.sql.side.redis; import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.BaseAsyncReqRow; +import com.dtstack.flink.sql.side.CacheMissVal; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.redis.enums.RedisType; import com.dtstack.flink.sql.side.redis.table.RedisSideReqRow; diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java index e57cb5bff..b8aa79771 100644 --- a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java @@ -18,8 +18,8 @@ package com.dtstack.flink.sql.side.redis.table; -import com.dtstack.flink.sql.side.ISideReqRow; import com.dtstack.flink.sql.side.BaseSideInfo; +import com.dtstack.flink.sql.side.ISideReqRow; import com.dtstack.flink.sql.util.TableUtils; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java index ddfb876df..311676929 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -20,6 +20,7 @@ import com.dtstack.flink.sql.outputformat.AbstractDtRichOutputFormat; import com.dtstack.flink.sql.sink.redis.enums.RedisType; +import com.dtstack.flink.sql.util.SampleUtils; import com.google.common.collect.Maps; import org.apache.commons.lang3.StringUtils; import org.apache.commons.pool2.impl.GenericObjectPoolConfig; @@ -37,10 +38,9 @@ import java.io.Closeable; import java.io.IOException; -import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Set; /** @@ -49,9 +49,17 @@ public class RedisOutputFormat extends AbstractDtRichOutputFormat { private static final Logger LOG = LoggerFactory.getLogger(RedisOutputFormat.class); + protected String[] fieldNames; + + protected TypeInformation[] fieldTypes; + + protected List primaryKeys; + + protected int timeout = 10000; + private String url; - private String database; + private String database = "0"; private String tableName; @@ -67,13 +75,7 @@ public class RedisOutputFormat extends AbstractDtRichOutputFormat { private String masterName; - protected String[] fieldNames; - - protected TypeInformation[] fieldTypes; - - protected List primaryKeys; - - protected int timeout; + protected int keyExpiredTime; private JedisPool pool; @@ -83,8 +85,13 @@ public class RedisOutputFormat extends AbstractDtRichOutputFormat { private GenericObjectPoolConfig poolConfig; - private RedisOutputFormat(){ + private RedisOutputFormat() { + } + + public static RedisOutputFormatBuilder buildRedisOutputFormat() { + return new RedisOutputFormatBuilder(); } + @Override public void configure(Configuration parameters) { @@ -96,15 +103,15 @@ public void open(int taskNumber, int numTasks) throws IOException { initMetric(); } - private GenericObjectPoolConfig setPoolConfig(String maxTotal, String maxIdle, String minIdle){ + private GenericObjectPoolConfig setPoolConfig(String maxTotal, String maxIdle, String minIdle) { GenericObjectPoolConfig config = new GenericObjectPoolConfig(); - if (maxTotal != null){ + if (maxTotal != null) { config.setMaxTotal(Integer.parseInt(maxTotal)); } - if (maxIdle != null){ + if (maxIdle != null) { config.setMaxIdle(Integer.parseInt(maxIdle)); } - if (minIdle != null){ + if (minIdle != null) { config.setMinIdle(Integer.parseInt(minIdle)); } return config; @@ -121,36 +128,28 @@ private void establishConnection() { for (String ipPort : nodes) { ipPorts.add(ipPort); String[] ipPortPair = StringUtils.split(ipPort, ":"); - addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.valueOf(ipPortPair[1].trim()))); - } - if (timeout == 0){ - timeout = 10000; - } - if (database == null) - { - database = "0"; + addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.parseInt(ipPortPair[1].trim()))); } - switch (redisType){ - //单机 - case 1: + switch (RedisType.parse(redisType)) { + case STANDALONE: pool = new JedisPool(poolConfig, firstIp, Integer.parseInt(firstPort), timeout, password, Integer.parseInt(database)); jedis = pool.getResource(); break; - //哨兵 - case 2: + case SENTINEL: jedisSentinelPool = new JedisSentinelPool(masterName, ipPorts, poolConfig, timeout, password, Integer.parseInt(database)); jedis = jedisSentinelPool.getResource(); break; - //集群 - case 3: + case CLUSTER: jedis = new JedisCluster(addresses, timeout, timeout, 10, password, poolConfig); + break; default: + throw new RuntimeException("unsupported redis type[ " + redisType + "]"); } } @Override - public void writeRecord(Tuple2 record) throws IOException { + public void writeRecord(Tuple2 record) { Tuple2 tupleTrans = record; Boolean retract = tupleTrans.getField(0); if (!retract) { @@ -160,41 +159,31 @@ public void writeRecord(Tuple2 record) throws IOException { if (row.getArity() != fieldNames.length) { return; } - - HashMap map = new HashMap<>(8); - for (String primaryKey : primaryKeys) { - for (int i = 0; i < fieldNames.length; i++) { - if (fieldNames[i].equals(primaryKey)) { - map.put(primaryKey, i); - } - } - } - - List kvList = new LinkedList<>(); - for (String primaryKey : primaryKeys){ - StringBuilder primaryKv = new StringBuilder(); - int index = map.get(primaryKey).intValue(); - primaryKv.append(primaryKey).append(":").append(row.getField(index)); - kvList.add(primaryKv.toString()); + Map refData = Maps.newHashMap(); + for (int i = 0; i < fieldNames.length; i++) { + refData.put(fieldNames[i], row.getField(i)); } + save(refData); + SampleUtils.samplingSinkPrint(samplingIntervalCount, LOG, outRecords.getCount(), row.toString()); + outRecords.inc(); + } - String perKey = String.join(":", kvList); - for (int i = 0; i < fieldNames.length; i++) { - StringBuilder key = new StringBuilder(); - key.append(tableName).append(":").append(perKey).append(":").append(fieldNames[i]); - String value = "null"; - Object field = row.getField(i); - if (field != null) { - value = field.toString(); + /** + * 1. build key from map. + * 2. save key and value. + * 3. set expired time for key when keyExpiredTime has been set. + * @param refData + */ + private synchronized void save(Map refData) { + String key = buildCacheKey(refData); + try { + refData.forEach((field, value) -> jedis.hset(key, field, String.valueOf(value))); + } finally { + if (keyExpiredTime != 0) { + jedis.expire(key, keyExpiredTime); } - jedis.set(key.toString(), value); } - - if (outRecords.getCount() % ROW_PRINT_FREQUENCY == 0){ - LOG.info(record.toString()); - } - outRecords.inc(); } @Override @@ -205,96 +194,108 @@ public void close() throws IOException { if (pool != null) { pool.close(); } - if (jedis != null){ - if (jedis instanceof Closeable){ + if (jedis != null) { + if (jedis instanceof Closeable) { ((Closeable) jedis).close(); } } } - public static RedisOutputFormatBuilder buildRedisOutputFormat(){ - return new RedisOutputFormatBuilder(); + public String buildCacheKey(Map refData) { + StringBuilder keyBuilder = new StringBuilder(tableName); + for (String primaryKey : primaryKeys) { + if (!refData.containsKey(primaryKey)) { + return null; + } + keyBuilder.append("_").append(refData.get(primaryKey)); + } + return keyBuilder.toString(); } - public static class RedisOutputFormatBuilder{ + public static class RedisOutputFormatBuilder { private final RedisOutputFormat redisOutputFormat; - protected RedisOutputFormatBuilder(){ + protected RedisOutputFormatBuilder() { this.redisOutputFormat = new RedisOutputFormat(); } - public RedisOutputFormatBuilder setUrl(String url){ + public RedisOutputFormatBuilder setUrl(String url) { redisOutputFormat.url = url; return this; } - public RedisOutputFormatBuilder setDatabase(String database){ + public RedisOutputFormatBuilder setDatabase(String database) { redisOutputFormat.database = database; return this; } - public RedisOutputFormatBuilder setTableName(String tableName){ + public RedisOutputFormatBuilder setTableName(String tableName) { redisOutputFormat.tableName = tableName; return this; } - public RedisOutputFormatBuilder setPassword(String password){ + public RedisOutputFormatBuilder setPassword(String password) { redisOutputFormat.password = password; return this; } - public RedisOutputFormatBuilder setFieldNames(String[] fieldNames){ + public RedisOutputFormatBuilder setFieldNames(String[] fieldNames) { redisOutputFormat.fieldNames = fieldNames; return this; } - public RedisOutputFormatBuilder setFieldTypes(TypeInformation[] fieldTypes){ + public RedisOutputFormatBuilder setFieldTypes(TypeInformation[] fieldTypes) { redisOutputFormat.fieldTypes = fieldTypes; return this; } - public RedisOutputFormatBuilder setPrimaryKeys(List primaryKeys){ + public RedisOutputFormatBuilder setPrimaryKeys(List primaryKeys) { redisOutputFormat.primaryKeys = primaryKeys; return this; } - public RedisOutputFormatBuilder setTimeout(int timeout){ + public RedisOutputFormatBuilder setTimeout(int timeout) { redisOutputFormat.timeout = timeout; return this; } - public RedisOutputFormatBuilder setRedisType(int redisType){ + public RedisOutputFormatBuilder setRedisType(int redisType) { redisOutputFormat.redisType = redisType; return this; } - public RedisOutputFormatBuilder setMaxTotal(String maxTotal){ + public RedisOutputFormatBuilder setMaxTotal(String maxTotal) { redisOutputFormat.maxTotal = maxTotal; return this; } - public RedisOutputFormatBuilder setMaxIdle(String maxIdle){ + public RedisOutputFormatBuilder setMaxIdle(String maxIdle) { redisOutputFormat.maxIdle = maxIdle; return this; } - public RedisOutputFormatBuilder setMinIdle(String minIdle){ + public RedisOutputFormatBuilder setMinIdle(String minIdle) { redisOutputFormat.minIdle = minIdle; return this; } - public RedisOutputFormatBuilder setMasterName(String masterName){ + public RedisOutputFormatBuilder setMasterName(String masterName) { redisOutputFormat.masterName = masterName; return this; } - public RedisOutputFormat finish(){ - if (redisOutputFormat.url == null){ + public RedisOutputFormatBuilder setKeyExpiredTime(int keyExpiredTime){ + redisOutputFormat.keyExpiredTime = keyExpiredTime; + return this; + } + + public RedisOutputFormat finish() { + if (redisOutputFormat.url == null) { throw new IllegalArgumentException("No URL supplied."); } - if (redisOutputFormat.tableName == null){ + if (redisOutputFormat.tableName == null) { throw new IllegalArgumentException("No tablename supplied."); } diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java index ec5dcf092..259a7821f 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java @@ -71,7 +71,9 @@ public class RedisSink implements RetractStreamTableSink, IStreamSinkGener< protected String registerTableName; - public RedisSink(){ + protected int keyExpiredTime; + + public RedisSink() { } @@ -92,6 +94,7 @@ public RedisSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { this.parallelism = Objects.isNull(redisTableInfo.getParallelism()) ? parallelism : redisTableInfo.getParallelism(); this.registerTableName = redisTableInfo.getName(); + this.keyExpiredTime = redisTableInfo.getKeyExpiredTime(); return this; } @@ -103,7 +106,7 @@ public TypeInformation getRecordType() { @Override public DataStreamSink> consumeDataStream(DataStream> dataStream) { RedisOutputFormat.RedisOutputFormatBuilder builder = RedisOutputFormat.buildRedisOutputFormat(); - builder.setUrl(this.url) + RedisOutputFormat redisOutputFormat = builder.setUrl(this.url) .setDatabase(this.database) .setTableName(this.tableName) .setPassword(this.password) @@ -115,8 +118,9 @@ public DataStreamSink> consumeDataStream(DataStream primaryKeysList = Lists.newArrayList(); if (!StringUtils.isEmpty(primaryKeysStr)) { diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java index 2425f8c38..938974f04 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java @@ -49,6 +49,8 @@ public class RedisTableInfo extends AbstractTargetTableInfo { public static final String PRIMARY_KEYS_NAME = "primarykeys"; + public static final String KEY_EXPIRED_TIME = "keyExpiredTime"; + public RedisTableInfo(){ setType(CURR_TYPE); } @@ -73,6 +75,8 @@ public RedisTableInfo(){ private String masterName; + private int keyExpiredTime; + public String getUrl() { return url; } @@ -153,6 +157,14 @@ public void setMasterName(String masterName) { this.masterName = masterName; } + public int getKeyExpiredTime() { + return keyExpiredTime; + } + + public void setKeyExpiredTime(int keyExpiredTime) { + this.keyExpiredTime = keyExpiredTime; + } + @Override public boolean check() { Preconditions.checkNotNull(url, "redis field of URL is required"); diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java index 5ea2d31a4..ca2450c40 100644 --- a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java @@ -17,6 +17,7 @@ */ package com.dtstack.flink.sql.source.serversocket; +import com.dtstack.flink.sql.dirtyManager.manager.DirtyDataManager; import com.dtstack.flink.sql.format.DeserializationMetricWrapper; import com.dtstack.flink.sql.format.dtnest.DtNestRowDeserializationSchema; import com.dtstack.flink.sql.source.serversocket.table.ServersocketSourceTableInfo; @@ -74,7 +75,7 @@ public CustomerSocketTextStreamFunction(ServersocketSourceTableInfo tableInfo, T Map rowAndFieldMapping, List fieldExtraInfos) { this.tableInfo = tableInfo; this.deserializationSchema = new DtNestRowDeserializationSchema(typeInfo, rowAndFieldMapping, fieldExtraInfos, CHARSET_NAME); - this.deserializationMetricWrapper = new DeserializationMetricWrapper(typeInfo, deserializationSchema); + this.deserializationMetricWrapper = new DeserializationMetricWrapper(typeInfo, deserializationSchema, DirtyDataManager.newInstance(tableInfo.getDirtyProperties())); } @Override diff --git a/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java b/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java index e06c13898..7b87aa09b 100644 --- a/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java +++ b/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java @@ -17,6 +17,7 @@ */ package com.dtstack.flink.sql.side.sqlserver.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; import com.dtstack.flink.sql.table.AbstractTableInfo; import java.util.Map; @@ -31,6 +32,7 @@ public class SqlserverSideParser extends RdbSideParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "net.sourceforge.jtds.jdbc.Driver"); AbstractTableInfo sqlServerTableInfo = super.getTableInfo(tableName, fieldsInfo, props); sqlServerTableInfo.setType(CURR_TYPE); return sqlServerTableInfo; diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java index 5300884bc..0e6522446 100644 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java @@ -17,6 +17,7 @@ */ package com.dtstack.flink.sql.sink.sqlserver.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -34,6 +35,7 @@ public class SqlserverSinkParser extends RdbSinkParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "net.sourceforge.jtds.jdbc.Driver"); AbstractTableInfo sqlserverTableInfo = super.getTableInfo(tableName, fieldsInfo, props); sqlserverTableInfo.setType(CURR_TYPE); return sqlserverTableInfo; diff --git a/tidb/tidb-side/tidb-side-core/src/main/java/com/dtstack/flink/sql/side/tidb/table/TidbSideParser.java b/tidb/tidb-side/tidb-side-core/src/main/java/com/dtstack/flink/sql/side/tidb/table/TidbSideParser.java index ae8f59686..4d5088738 100644 --- a/tidb/tidb-side/tidb-side-core/src/main/java/com/dtstack/flink/sql/side/tidb/table/TidbSideParser.java +++ b/tidb/tidb-side/tidb-side-core/src/main/java/com/dtstack/flink/sql/side/tidb/table/TidbSideParser.java @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.side.tidb.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -34,6 +35,7 @@ public class TidbSideParser extends RdbSideParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "com.mysql.jdbc.Driver"); AbstractTableInfo tableInfo = super.getTableInfo(tableName, fieldsInfo, props); tableInfo.setType(CURRENT_TYPE); return tableInfo; diff --git a/tidb/tidb-sink/src/main/java/com/dtstack/flink/sql/sink/tidb/table/TidbSinkParser.java b/tidb/tidb-sink/src/main/java/com/dtstack/flink/sql/sink/tidb/table/TidbSinkParser.java index d00206cc2..4f7001e85 100644 --- a/tidb/tidb-sink/src/main/java/com/dtstack/flink/sql/sink/tidb/table/TidbSinkParser.java +++ b/tidb/tidb-sink/src/main/java/com/dtstack/flink/sql/sink/tidb/table/TidbSinkParser.java @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.sink.tidb.table; +import com.dtstack.flink.sql.core.rdb.JdbcCheckKeys; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -34,6 +35,7 @@ public class TidbSinkParser extends RdbSinkParser { @Override public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + props.put(JdbcCheckKeys.DRIVER_NAME, "com.mysql.jdbc.Driver"); AbstractTableInfo tableInfo = super.getTableInfo(tableName, fieldsInfo, props); tableInfo.setType(CURRENT_TYPE); return tableInfo;