diff --git a/.gitignore b/.gitignore index d7842e1ed..c8f97e79e 100644 --- a/.gitignore +++ b/.gitignore @@ -15,3 +15,5 @@ lib/ bin/nohup.out .DS_Store bin/sideSql.txt +krb5.conf +*.keytab \ No newline at end of file diff --git a/SECURITY.md b/SECURITY.md new file mode 100644 index 000000000..034e84803 --- /dev/null +++ b/SECURITY.md @@ -0,0 +1,21 @@ +# Security Policy + +## Supported Versions + +Use this section to tell people about which versions of your project are +currently being supported with security updates. + +| Version | Supported | +| ------- | ------------------ | +| 5.1.x | :white_check_mark: | +| 5.0.x | :x: | +| 4.0.x | :white_check_mark: | +| < 4.0 | :x: | + +## Reporting a Vulnerability + +Use this section to tell people how to report a vulnerability. + +Tell them where to go, how often they can expect to get an update on a +reported vulnerability, what to expect if the vulnerability is accepted or +declined, etc. diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraSink.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraSink.java index 26152a7d3..a78ef97f7 100644 --- a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraSink.java +++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraSink.java @@ -34,6 +34,8 @@ import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; +import java.util.Objects; + /** * Reason: * Date: 2018/11/22 @@ -57,6 +59,8 @@ public class CassandraSink implements RetractStreamTableSink, IStreamSinkGe protected Integer readTimeoutMillis; protected Integer connectTimeoutMillis; protected Integer poolTimeoutMillis; + protected Integer parallelism = 1; + protected String registerTableName; public CassandraSink() { // TO DO NOTHING @@ -77,6 +81,9 @@ public CassandraSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { this.readTimeoutMillis = cassandraTableInfo.getReadTimeoutMillis(); this.connectTimeoutMillis = cassandraTableInfo.getConnectTimeoutMillis(); this.poolTimeoutMillis = cassandraTableInfo.getPoolTimeoutMillis(); + this.parallelism = Objects.isNull(cassandraTableInfo.getParallelism()) ? + parallelism : cassandraTableInfo.getParallelism(); + this.registerTableName = cassandraTableInfo.getTableName(); return this; } @@ -100,7 +107,7 @@ public void emitDataStream(DataStream> dataStream) { CassandraOutputFormat outputFormat = builder.finish(); RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(outputFormat); - dataStream.addSink(richSinkFunction); + dataStream.addSink(richSinkFunction).setParallelism(parallelism).name(registerTableName); } @Override diff --git a/core/src/main/java/com/dtstack/flink/sql/GetPlan.java b/core/src/main/java/com/dtstack/flink/sql/GetPlan.java index ba6518b3d..409224577 100644 --- a/core/src/main/java/com/dtstack/flink/sql/GetPlan.java +++ b/core/src/main/java/com/dtstack/flink/sql/GetPlan.java @@ -24,6 +24,9 @@ import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import java.net.URL; +import java.net.URLClassLoader; + /** * local模式获取sql任务的执行计划 * Date: 2020/2/17 @@ -33,15 +36,22 @@ public class GetPlan { public static String getExecutionPlan(String[] args) { + ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader(); + ClassLoader envClassLoader = StreamExecutionEnvironment.class.getClassLoader(); + ClassLoader plannerClassLoader = new URLClassLoader(new URL[0], envClassLoader); try { long start = System.currentTimeMillis(); ParamsInfo paramsInfo = ExecuteProcessHelper.parseParams(args); + paramsInfo.setGetPlan(true); + Thread.currentThread().setContextClassLoader(plannerClassLoader); StreamExecutionEnvironment env = ExecuteProcessHelper.getStreamExecution(paramsInfo); String executionPlan = env.getExecutionPlan(); long end = System.currentTimeMillis(); return ApiResult.createSuccessResultJsonStr(executionPlan, end - start); } catch (Exception e) { return ApiResult.createErrorResultJsonStr(ExceptionUtils.getFullStackTrace(e)); + } finally { + Thread.currentThread().setContextClassLoader(currentClassLoader); } } } 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 2e62e11ab..bd805007e 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 @@ -20,10 +20,12 @@ import com.dtstack.flink.sql.util.PluginUtil; import com.dtstack.flink.sql.util.ReflectionUtils; +import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.FileInputStream; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.net.URL; @@ -73,7 +75,17 @@ private static DtClassLoader retrieveClassLoad(String pluginJarPath) { private static DtClassLoader retrieveClassLoad(List jarUrls) { jarUrls.sort(Comparator.comparing(URL::toString)); - String jarUrlkey = StringUtils.join(jarUrls, "_"); + + List jarMd5s = new ArrayList<>(jarUrls.size()); + for (URL jarUrl : jarUrls) { + try (FileInputStream inputStream = new FileInputStream(jarUrl.getPath())){ + String jarMd5 = DigestUtils.md5Hex(inputStream); + jarMd5s.add(jarMd5); + } catch (Exception e) { + throw new RuntimeException("Exceptions appears when read file:" + e); + } + } + String jarUrlkey = StringUtils.join(jarMd5s, "_"); return pluginClassLoader.computeIfAbsent(jarUrlkey, k -> { try { URL[] urls = jarUrls.toArray(new URL[jarUrls.size()]); diff --git a/core/src/main/java/com/dtstack/flink/sql/constrant/PluginParamConsts.java b/core/src/main/java/com/dtstack/flink/sql/constrant/PluginParamConsts.java new file mode 100644 index 000000000..baf314c19 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/constrant/PluginParamConsts.java @@ -0,0 +1,30 @@ +/* + * 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.constrant; + +/** + * @program: flinkStreamSQL + * @author: wuren + * @create: 2020/09/15 + **/ +public class PluginParamConsts { + public static final String PRINCIPAL = "principal"; + public static final String KEYTAB = "keytab"; + public static final String KRB5_CONF = "krb5conf"; +} diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/ColumnType.java b/core/src/main/java/com/dtstack/flink/sql/enums/ColumnType.java index 7f3f0019c..e93c92f51 100644 --- a/core/src/main/java/com/dtstack/flink/sql/enums/ColumnType.java +++ b/core/src/main/java/com/dtstack/flink/sql/enums/ColumnType.java @@ -102,6 +102,10 @@ public enum ColumnType { * timestamp */ TIMESTAMP, + /** + * time eg: 23:59:59 + */ + TIME, /** * decimal */ diff --git a/core/src/main/java/com/dtstack/flink/sql/exception/ExceptionTrace.java b/core/src/main/java/com/dtstack/flink/sql/exception/ExceptionTrace.java new file mode 100644 index 000000000..07db131d9 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/exception/ExceptionTrace.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * 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.exception; + +import java.util.Objects; + +/** + * @author tiezhu + * @date 2021/3/8 星期一 + * Company dtstack + */ +public class ExceptionTrace { + // 追溯当前异常的最原始异常信息 + public static String traceOriginalCause(Throwable e) { + String errorMsg; + if (Objects.nonNull(e.getCause())) { + errorMsg = traceOriginalCause(e.getCause()); + } else { + errorMsg = e.getMessage(); + } + return errorMsg; + } +} 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 5692acf1c..54927d531 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 @@ -81,9 +81,10 @@ import java.util.Set; /** - * 任务执行时的流程方法 + * 任务执行时的流程方法 * Date: 2020/2/17 * Company: www.dtstack.com + * * @author maqi */ public class ExecuteProcessHelper { @@ -130,13 +131,14 @@ public static ParamsInfo parseParams(String[] args) throws Exception { } /** - * 非local模式或者shipfile部署模式,remoteSqlPluginPath必填 + * 非local模式或者shipfile部署模式,remoteSqlPluginPath必填 + * * @param remoteSqlPluginPath * @param deployMode * @param pluginLoadMode * @return */ - public static boolean checkRemoteSqlPluginPath(String remoteSqlPluginPath, String deployMode, String pluginLoadMode) { + private static boolean checkRemoteSqlPluginPath(String remoteSqlPluginPath, String deployMode, String pluginLoadMode) { if (StringUtils.isEmpty(remoteSqlPluginPath)) { return StringUtils.equalsIgnoreCase(pluginLoadMode, EPluginLoadMode.SHIPFILE.name()) || StringUtils.equalsIgnoreCase(deployMode, ClusterMode.local.name()); @@ -159,7 +161,7 @@ public static StreamExecutionEnvironment getStreamExecution(ParamsInfo paramsInf Map registerTableCache = Maps.newHashMap(); //register udf - ExecuteProcessHelper.registerUserDefinedFunction(sqlTree, paramsInfo.getJarUrlList(), tableEnv); + ExecuteProcessHelper.registerUserDefinedFunction(sqlTree, paramsInfo.getJarUrlList(), tableEnv, paramsInfo.isGetPlan()); //register table schema Set classPathSets = ExecuteProcessHelper.registerTable(sqlTree, env, tableEnv, paramsInfo.getLocalSqlPluginPath(), paramsInfo.getRemoteSqlPluginPath(), paramsInfo.getPluginLoadMode(), sideTableMap, registerTableCache); @@ -175,7 +177,7 @@ public static StreamExecutionEnvironment getStreamExecution(ParamsInfo paramsInf } - public static List getExternalJarUrls(String addJarListStr) throws java.io.IOException { + private static List getExternalJarUrls(String addJarListStr) throws java.io.IOException { List jarUrlList = Lists.newArrayList(); if (Strings.isNullOrEmpty(addJarListStr)) { return jarUrlList; @@ -191,7 +193,7 @@ public static List getExternalJarUrls(String addJarListStr) throws java.io. private static void sqlTranslation(String localSqlPluginPath, StreamTableEnvironment tableEnv, - SqlTree sqlTree,Map sideTableMap, + SqlTree sqlTree, Map sideTableMap, Map registerTableCache, StreamQueryConfig queryConfig) throws Exception { @@ -245,29 +247,26 @@ private static void sqlTranslation(String localSqlPluginPath, } } - public static void registerUserDefinedFunction(SqlTree sqlTree, List jarUrlList, TableEnvironment tableEnv) + public static void registerUserDefinedFunction(SqlTree sqlTree, List jarUrlList, TableEnvironment tableEnv, boolean isGetPlan) throws IllegalAccessException, InvocationTargetException { // udf和tableEnv须由同一个类加载器加载 - ClassLoader levelClassLoader = tableEnv.getClass().getClassLoader(); - URLClassLoader classLoader = null; + ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader(); + URLClassLoader classLoader = ClassLoaderManager.loadExtraJar(jarUrlList, (URLClassLoader) currentClassLoader); List funcList = sqlTree.getFunctionList(); for (CreateFuncParser.SqlParserResult funcInfo : funcList) { - //classloader - if (classLoader == null) { - classLoader = ClassLoaderManager.loadExtraJar(jarUrlList, (URLClassLoader) levelClassLoader); - } FunctionManager.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName(), tableEnv, classLoader); } } /** - * 向Flink注册源表和结果表,返回执行时插件包的全路径 + * 向Flink注册源表和结果表,返回执行时插件包的全路径 + * * @param sqlTree * @param env * @param tableEnv * @param localSqlPluginPath * @param remoteSqlPluginPath - * @param pluginLoadMode 插件加载模式 classpath or shipfile + * @param pluginLoadMode 插件加载模式 classpath or shipfile * @param sideTableMap * @param registerTableCache * @return @@ -298,7 +297,23 @@ public static Set registerTable(SqlTree sqlTree, StreamExecutionEnvironment if (waterMarkerAssigner.checkNeedAssignWaterMarker(sourceTableInfo)) { adaptStream = waterMarkerAssigner.assignWaterMarker(adaptStream, typeInfo, sourceTableInfo); - fields += ",ROWTIME.ROWTIME"; + String eventTimeField = sourceTableInfo.getEventTimeField(); + boolean hasEventTimeField = false; + if (!Strings.isNullOrEmpty(eventTimeField)) { + String[] fieldArray = fields.split(","); + for (int i = 0; i < fieldArray.length; i++) { + if (fieldArray[i].equals(eventTimeField)) { + fieldArray[i] = eventTimeField + ".ROWTIME"; + hasEventTimeField = true; + break; + } + } + if (hasEventTimeField) { + fields = String.join(",", fieldArray); + } else { + fields += ",ROWTIME.ROWTIME"; + } + } } else { fields += ",PROCTIME.PROCTIME"; } @@ -315,8 +330,13 @@ public static Set registerTable(SqlTree sqlTree, StreamExecutionEnvironment } else if (tableInfo instanceof AbstractTargetTableInfo) { TableSink tableSink = StreamSinkFactory.getTableSink((AbstractTargetTableInfo) tableInfo, localSqlPluginPath); - TypeInformation[] flinkTypes = FunctionManager.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 = FunctionManager.transformTypes(tableInfo.getFieldClasses()); + tableEnv.registerTableSink(tableInfo.getName(), tableInfo.getFields(), flinkTypes, tableSink); + } URL sinkTablePathUrl = PluginUtil.buildSourceAndSinkPathByLoadMode(tableInfo.getType(), AbstractTargetTableInfo.TARGET_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode); pluginClassPathSets.add(sinkTablePathUrl); @@ -334,11 +354,12 @@ public static Set registerTable(SqlTree sqlTree, StreamExecutionEnvironment } /** - * perjob模式将job依赖的插件包路径存储到cacheFile,在外围将插件包路径传递给jobgraph + * perjob模式将job依赖的插件包路径存储到cacheFile,在外围将插件包路径传递给jobgraph + * * @param env * @param classPathSet */ - public static void registerPluginUrlToCachedFile(StreamExecutionEnvironment env, Set classPathSet) { + private static void registerPluginUrlToCachedFile(StreamExecutionEnvironment env, Set classPathSet) { int i = 0; for (URL url : classPathSet) { String classFileName = String.format(CLASS_FILE_NAME_FMT, i); @@ -347,7 +368,7 @@ public static void registerPluginUrlToCachedFile(StreamExecutionEnvironment env, } } - public static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws Exception { + private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws Exception { StreamExecutionEnvironment env = !ClusterMode.local.name().equals(deployMode) ? StreamExecutionEnvironment.getExecutionEnvironment() : new MyLocalStreamEnvironment(); @@ -356,4 +377,4 @@ public static StreamExecutionEnvironment getStreamExeEnv(Properties confProperti return env; } -} \ No newline at end of file +} 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 27cc7702d..9619c75c1 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 @@ -21,6 +21,7 @@ import java.net.URL; import java.util.List; +import java.util.Objects; import java.util.Properties; /** @@ -39,6 +40,7 @@ public class ParamsInfo { private String pluginLoadMode; private String deployMode; private Properties confProp; + private boolean getPlan = false; public ParamsInfo(String sql, String name, List jarUrlList, String localSqlPluginPath, String remoteSqlPluginPath, String pluginLoadMode, String deployMode, Properties confProp) { @@ -52,6 +54,14 @@ public ParamsInfo(String sql, String name, List jarUrlList, String localSql this.confProp = confProp; } + public boolean isGetPlan() { + return getPlan; + } + + public void setGetPlan(boolean getPlan) { + this.getPlan = getPlan; + } + public String getSql() { return sql; } @@ -114,10 +124,8 @@ public static class Builder { private String remoteSqlPluginPath; private String pluginLoadMode; private String deployMode; - private String logLevel; private Properties confProp; - public ParamsInfo.Builder setSql(String sql) { this.sql = sql; return this; 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 96ebe09ae..a4addc025 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 @@ -31,6 +31,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.nio.charset.StandardCharsets; /** * add metric for source @@ -91,7 +92,7 @@ public void initMetric() { public Row deserialize(byte[] message) throws IOException { try { if (numInRecord.getCount() % dataPrintFrequency == 0) { - LOG.info("receive source data:" + new String(message, "UTF-8")); + LOG.info("receive source data:" + new String(message, StandardCharsets.UTF_8)); } numInRecord.inc(); numInBytes.inc(message.length); diff --git a/core/src/main/java/com/dtstack/flink/sql/format/dtnest/DtNestRowDeserializationSchema.java b/core/src/main/java/com/dtstack/flink/sql/format/dtnest/DtNestRowDeserializationSchema.java index 13cc261ad..8b2ca5f49 100644 --- a/core/src/main/java/com/dtstack/flink/sql/format/dtnest/DtNestRowDeserializationSchema.java +++ b/core/src/main/java/com/dtstack/flink/sql/format/dtnest/DtNestRowDeserializationSchema.java @@ -42,6 +42,8 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; /** * source data parse to json format @@ -63,6 +65,15 @@ public class DtNestRowDeserializationSchema extends AbstractDeserializationSchem private final List fieldExtraInfos; private final String charsetName; + private static final Pattern TIMESTAMP_PATTERN = Pattern.compile("^\\d+$"); + private static final Pattern TIMESTAMP_FORMAT_PATTERN = Pattern.compile("(\\d+-\\d+-\\d+\\s)(\\d+:\\d+:\\d+)"); + private static final Pattern TIME_FORMAT_PATTERN = Pattern.compile("\\w+\\d+:\\d+:\\d+"); + private static final Pattern DATE_FORMAT_PATTERN = Pattern.compile("\\w+\\d+-\\d+-\\d+"); + + private static final String TIMESTAMP_TYPE = "timestamp"; + private static final String TIME_TYPE = "time"; + private static final String DATE_TYPE = "date"; + public DtNestRowDeserializationSchema(TypeInformation typeInfo, Map rowAndFieldMapping, List fieldExtraInfos, String charsetName) { @@ -141,14 +152,14 @@ private Object convert(JsonNode node, TypeInformation info) { return node.asText(); } } else if (info.getTypeClass().equals(Types.SQL_DATE.getTypeClass())) { - return Date.valueOf(node.asText()); + return convertToTimestamp(node.asText(), DATE_TYPE); } else if (info.getTypeClass().equals(Types.SQL_TIME.getTypeClass())) { // local zone - return Time.valueOf(node.asText()); + return convertToTimestamp(node.asText(), TIME_TYPE); } else if (info.getTypeClass().equals(Types.SQL_TIMESTAMP.getTypeClass())) { // local zone - return Timestamp.valueOf(node.asText()); - } else if (info instanceof RowTypeInfo) { + return convertToTimestamp(node.asText(), TIMESTAMP_TYPE); + } else if (info instanceof RowTypeInfo) { return convertRow(node, (RowTypeInfo) info); } else if (info instanceof ObjectArrayTypeInfo) { return convertObjectArray(node, ((ObjectArrayTypeInfo) info).getComponentInfo()); @@ -163,6 +174,47 @@ private Object convert(JsonNode node, TypeInformation info) { } } + /** + * 将 2020-09-07 14:49:10.0 和 1598446699685 两种格式都转化为 Timestamp、Time、Date + */ + private static Object convertToTimestamp(String timestamp, String type) { + if (TIMESTAMP_PATTERN.matcher(timestamp).find()) { + switch (type.toLowerCase()) { + case (TIMESTAMP_TYPE): + return new Timestamp(Long.parseLong(timestamp)); + case (DATE_TYPE): + return new Date(new Timestamp(Long.parseLong(timestamp)).getTime()); + case (TIME_TYPE): + return new Time(new Timestamp(Long.parseLong(timestamp)).getTime()); + default: + throw new RuntimeException(String.format("%s transform to %s error!", timestamp, type)); + } + } + + Matcher matcher = TIMESTAMP_FORMAT_PATTERN.matcher(timestamp); + if (matcher.find()) { + switch (type.toLowerCase()) { + case TIME_TYPE: + return Time.valueOf(String.valueOf(matcher.group(2)).trim()); + case DATE_TYPE: + return Date.valueOf(String.valueOf(matcher.group(1)).trim()); + case TIMESTAMP_TYPE: + return Timestamp.valueOf(timestamp); + default: + } + } + + if (TIME_FORMAT_PATTERN.matcher(timestamp).find() && TIME_TYPE.equalsIgnoreCase(type)) { + return Time.valueOf(timestamp); + } + + if (DATE_FORMAT_PATTERN.matcher(timestamp).find() && DATE_TYPE.equalsIgnoreCase(type)) { + return Date.valueOf(timestamp); + } + + throw new IllegalArgumentException("Incorrect time format of timestamp, input: " + timestamp); + } + private Row convertTopRow() { Row row = new Row(fieldNames.length); try { @@ -173,7 +225,7 @@ private Row convertTopRow() { if (node == null) { if (fieldExtraInfo != null && fieldExtraInfo.getNotNull()) { throw new IllegalStateException("Failed to find field with name '" - + fieldNames[i] + "'."); + + fieldNames[i] + "'."); } else { row.setField(i, 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 360cc82ab..549276dc2 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 @@ -122,13 +122,12 @@ public static void registerAggregateUDF(String classPath, String funcName, Table } } - public static TypeInformation[] transformTypes(Class[] fieldTypes) { TypeInformation[] types = new TypeInformation[fieldTypes.length]; for (int i = 0; i < fieldTypes.length; i++) { types[i] = TypeInformation.of(fieldTypes[i]); } - return types; } + } diff --git a/core/src/main/java/com/dtstack/flink/sql/krb/KerberosTable.java b/core/src/main/java/com/dtstack/flink/sql/krb/KerberosTable.java new file mode 100644 index 000000000..ce6691294 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/krb/KerberosTable.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flink.sql.krb; + +import com.google.common.base.Strings; + +/** + * @program: flinkStreamSQL + * @author: wuren + * @create: 2020/09/15 + **/ +public interface KerberosTable { + + String getPrincipal(); + + void setPrincipal(String principal); + + String getKeytab(); + + void setKeytab(String keytab); + + String getKrb5conf(); + + void setKrb5conf(String krb5conf); + + boolean isEnableKrb(); + + void setEnableKrb(boolean enableKrb); + + default void judgeKrbEnable() { + boolean allSet = + !Strings.isNullOrEmpty(getPrincipal()) && + !Strings.isNullOrEmpty(getKeytab()) && + !Strings.isNullOrEmpty(getKrb5conf()); + + boolean allNotSet = + Strings.isNullOrEmpty(getPrincipal()) && + Strings.isNullOrEmpty(getKeytab()) && + Strings.isNullOrEmpty(getKrb5conf()); + + if (allSet) { + setEnableKrb(true); + } else if (allNotSet) { + setEnableKrb(false); + } else { + throw new RuntimeException("Missing kerberos parameter! all kerberos params must be set, or all kerberos params are not set"); + } + } +} 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 91b1fd2ac..1486e8bfa 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 @@ -16,12 +16,11 @@ * limitations under the License. */ - - package com.dtstack.flink.sql.parser; import com.dtstack.flink.sql.util.DtStringUtil; import com.google.common.collect.Maps; +import org.apache.commons.lang3.StringUtils; import java.util.List; import java.util.Map; @@ -41,8 +40,6 @@ public class CreateTableParser implements IParser { private static final Pattern PATTERN = Pattern.compile(PATTERN_STR); - private static final Pattern PROP_PATTERN = Pattern.compile("^'\\s*(.+)\\s*'$"); - public static CreateTableParser newInstance(){ return new CreateTableParser(); } @@ -70,28 +67,19 @@ public void parseSql(String sql, SqlTree sqlTree) { } } - private Map parseProp(String propsStr){ - propsStr = propsStr.replaceAll("'\\s*,", "'|"); - String[] strs = propsStr.trim().split("\\|"); + private Map parseProp(String propsStr){ + List strings = DtStringUtil.splitIgnoreQuota(propsStr.trim(), ','); Map propMap = Maps.newHashMap(); - for(int i=0; i ss = DtStringUtil.splitIgnoreQuota(strs[i], '='); + for (String str : strings) { + List ss = DtStringUtil.splitIgnoreQuota(str, '='); String key = ss.get(0).trim(); - String value = extractValue(ss.get(1).trim()); + String value = DtStringUtil.removeStartAndEndQuota(ss.get(1).trim()); propMap.put(key, value); } return propMap; } - private String extractValue(String value) { - Matcher matcher = PROP_PATTERN.matcher(value); - if (matcher.find()) { - return matcher.group(1); - } - throw new RuntimeException("[" + value + "] format is invalid"); - } - public static class SqlParserResult{ private String tableName; 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 7c76ec2cd..3f48d5049 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 @@ -36,12 +36,8 @@ private FlinkPlanner() { } public static FlinkPlannerImpl createFlinkPlanner(FrameworkConfig frameworkConfig, RelOptPlanner relOptPlanner, FlinkTypeFactory typeFactory) { - if (flinkPlanner == null) { - synchronized (FlinkPlanner.class) { - if (flinkPlanner == null) { - flinkPlanner = new FlinkPlannerImpl(frameworkConfig, relOptPlanner, typeFactory); - } - } + synchronized (FlinkPlanner.class) { + flinkPlanner = new FlinkPlannerImpl(frameworkConfig, relOptPlanner, typeFactory); } return flinkPlanner; } 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 23f8e4942..f91aa63c4 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,7 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.parser; @@ -30,6 +30,8 @@ import java.util.List; import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; /** * Reason: @@ -51,6 +53,8 @@ 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 (); @@ -74,6 +78,7 @@ public static SqlTree parseSql(String sql) throws Exception { .replace("\t", " ").trim(); List sqlArr = DtStringUtil.splitIgnoreQuota(sql, SQL_DELIMITER); + sqlArr = removeAddFileAndJarStmt(sqlArr); SqlTree sqlTree = new SqlTree(); AbstractTableInfoParser tableInfoParser = new AbstractTableInfoParser(); for(String childSql : sqlArr){ @@ -154,4 +159,18 @@ public static SqlTree parseSql(String sql) throws Exception { return sqlTree; } + + /** + * remove add file and jar with statment etc. add file /etc/krb5.conf, add jar xxx.jar; + */ + 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()) { + cleanedStmts.add(stmt); + } + } + return cleanedStmts; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/BaseAsyncReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/BaseAsyncReqRow.java index ae8c75f7e..b89eb5615 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/BaseAsyncReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/BaseAsyncReqRow.java @@ -181,7 +181,7 @@ public void asyncInvoke(CRow row, ResultFuture resultFuture) throws Except } private Map parseInputParam(CRow input){ - Map inputParams = Maps.newHashMap(); + Map inputParams = Maps.newLinkedHashMap(); for (int i = 0; i < sideInfo.getEqualValIndex().size(); i++) { Integer conValIndex = sideInfo.getEqualValIndex().get(i); Object equalObj = input.row().getField(conValIndex); @@ -243,12 +243,7 @@ protected ScheduledFuture registerTimer(CRow input, ResultFuture result long timeoutTimestamp = sideInfo.getSideTableInfo().getAsyncTimeout() + getProcessingTimeService().getCurrentProcessingTime(); return getProcessingTimeService().registerTimer( timeoutTimestamp, - new ProcessingTimeCallback() { - @Override - public void onProcessingTime(long timestamp) throws Exception { - timeout(input, resultFuture); - } - }); + timestamp -> timeout(input, resultFuture)); } protected void cancelTimerWhenComplete(ResultFuture resultFuture, ScheduledFuture timerFuture){ 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 57104bd04..a65a53235 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 @@ -285,7 +285,7 @@ private JoinInfo dealNestJoin(SqlJoin joinNode, SqlBasicCall buildAs = TableUtils.buildAsNodeByJoinInfo(joinInfo, null, null); if(rightIsSide){ - addSideInfoToExeQueue(queueInfo, joinInfo, joinNode, parentSelectList, parentGroupByList, parentWhere, tableRef); + addSideInfoToExeQueue(queueInfo, joinInfo, joinNode, parentSelectList, parentGroupByList, parentWhere, tableRef, fieldRef); } SqlNode newLeftNode = joinNode.getLeft(); @@ -298,7 +298,7 @@ private JoinInfo dealNestJoin(SqlJoin joinNode, //替换leftNode 为新的查询 joinNode.setLeft(buildAs); - replaceSelectAndWhereField(buildAs, leftJoinNode, tableRef, parentSelectList, parentGroupByList, parentWhere); + replaceSelectAndWhereField(buildAs, leftJoinNode, tableRef, fieldRef, parentSelectList, parentGroupByList, parentWhere); } return joinInfo; @@ -321,7 +321,8 @@ public void addSideInfoToExeQueue(Queue queueInfo, SqlNodeList parentSelectList, SqlNodeList parentGroupByList, SqlNode parentWhere, - Map tableRef){ + Map tableRef, + Map fieldRef){ //只处理维表 if(!joinInfo.isRightIsSideTable()){ return; @@ -333,7 +334,7 @@ public void addSideInfoToExeQueue(Queue queueInfo, //替换左表为新的表名称 joinNode.setLeft(buildAs); - replaceSelectAndWhereField(buildAs, leftJoinNode, tableRef, parentSelectList, parentGroupByList, parentWhere); + replaceSelectAndWhereField(buildAs, leftJoinNode, tableRef, fieldRef, parentSelectList, parentGroupByList, parentWhere); } /** @@ -348,6 +349,7 @@ public void addSideInfoToExeQueue(Queue queueInfo, public void replaceSelectAndWhereField(SqlBasicCall buildAs, SqlNode leftJoinNode, Map tableRef, + Map fieldRef, SqlNodeList parentSelectList, SqlNodeList parentGroupByList, SqlNode parentWhere){ @@ -361,23 +363,22 @@ public void replaceSelectAndWhereField(SqlBasicCall buildAs, } //替换select field 中的对应字段 - HashBiMap fieldReplaceRef = HashBiMap.create(); for(SqlNode sqlNode : parentSelectList.getList()){ for(String tbTmp : fromTableNameSet) { - TableUtils.replaceSelectFieldTable(sqlNode, tbTmp, newLeftTableName, fieldReplaceRef); + TableUtils.replaceSelectFieldTable(sqlNode, tbTmp, newLeftTableName, fieldRef); } } //TODO 应该根据上面的查询字段的关联关系来替换 //替换where 中的条件相关 for(String tbTmp : fromTableNameSet){ - TableUtils.replaceWhereCondition(parentWhere, tbTmp, newLeftTableName, fieldReplaceRef); + TableUtils.replaceWhereCondition(parentWhere, tbTmp, newLeftTableName, fieldRef); } if(parentGroupByList != null){ for(SqlNode sqlNode : parentGroupByList.getList()){ for(String tbTmp : fromTableNameSet) { - TableUtils.replaceSelectFieldTable(sqlNode, tbTmp, newLeftTableName, fieldReplaceRef); + TableUtils.replaceSelectFieldTable(sqlNode, tbTmp, newLeftTableName, fieldRef); } } } @@ -489,7 +490,7 @@ private Set extractSelectFieldFromJoinCondition(Set extractSelectFieldFromJoinCondition(Set fromTableNameSet, String checkTableName, Map mappingTableName) { + for (int i = 0; i < mappingTableName.size() + 1; i++) { + if (fromTableNameSet.contains(checkTableName)) { + return true; + } + + checkTableName = mappingTableName.get(checkTableName); + if (checkTableName == null) { + return false; + } + } + return true; + } + private Set extractFieldFromGroupByList(SqlNodeList parentGroupByList, Set fromTableNameSet, Map tableRef){ diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index bb8faf8ee..6dad92061 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -60,12 +60,7 @@ import org.slf4j.LoggerFactory; import java.sql.Timestamp; -import java.util.Arrays; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Queue; -import java.util.Set; +import java.util.*; import static org.apache.calcite.sql.SqlKind.*; @@ -351,7 +346,7 @@ protected void dealAsSourceTable(StreamTableEnvironment tableEnv, FieldReplaceInfo fieldReplaceInfo = parseAsQuery((SqlBasicCall) pollSqlNode, tableCache); if(fieldReplaceInfo == null){ - return; + return; } //as 的源表 @@ -361,6 +356,77 @@ protected void dealAsSourceTable(StreamTableEnvironment tableEnv, } + /** + * check whether all table fields exist in join condition. + * @param conditionNode + * @param joinScope + */ + public void checkConditionFieldsInTable(SqlNode conditionNode, JoinScope joinScope, AbstractSideTableInfo sideTableInfo) { + List sqlNodeList = Lists.newArrayList(); + ParseUtils.parseAnd(conditionNode, sqlNodeList); + for (SqlNode sqlNode : sqlNodeList) { + if (!SqlKind.COMPARISON.contains(sqlNode.getKind())) { + throw new RuntimeException("It is not comparison operator."); + } + + SqlNode leftNode = ((SqlBasicCall) sqlNode).getOperands()[0]; + SqlNode rightNode = ((SqlBasicCall) sqlNode).getOperands()[1]; + + if (leftNode.getKind() == SqlKind.IDENTIFIER) { + checkFieldInTable((SqlIdentifier) leftNode, joinScope, conditionNode, sideTableInfo); + } + + if (rightNode.getKind() == SqlKind.IDENTIFIER) { + checkFieldInTable((SqlIdentifier) rightNode, joinScope, conditionNode, sideTableInfo); + } + + } + } + + /** + * check whether table exists and whether field is in table. + * @param sqlNode + * @param joinScope + * @param conditionNode + */ + private void checkFieldInTable(SqlIdentifier sqlNode, JoinScope joinScope, SqlNode conditionNode, AbstractSideTableInfo sideTableInfo) { + String tableName = sqlNode.getComponent(0).getSimple(); + String fieldName = sqlNode.getComponent(1).getSimple(); + JoinScope.ScopeChild scopeChild = joinScope.getScope(tableName); + String tableErrorMsg = "Table [%s] is not exist. Error condition is [%s]. If you find [%s] is exist. Please check AS statement."; + Preconditions.checkState( + scopeChild != null, + tableErrorMsg, + tableName, + conditionNode.toString(), + tableName + ); + + String[] fieldNames = scopeChild.getRowTypeInfo().getFieldNames(); + ArrayList allFieldNames = new ArrayList( + Arrays.asList(fieldNames) + ); + // HBase、Redis这种NoSQL Primary Key不在字段列表中,所以要加进去。 + if (sideTableInfo != null) { + List pks = sideTableInfo.getPrimaryKeys(); + if (pks != null) { + pks.stream() + .filter(pk -> !allFieldNames.contains(pk)) + .forEach(pk -> allFieldNames.add(pk)); + } + } + + boolean hasField = allFieldNames.contains(fieldName); + String fieldErrorMsg = "Table [%s] has not [%s] field. Error join condition is [%s]. If you find it is exist. Please check AS statement."; + Preconditions.checkState( + hasField, + fieldErrorMsg, + tableName, + fieldName, + conditionNode.toString() + ); + } + private void joinFun(Object pollObj, Map localTableCache, Map sideTableMap, @@ -395,6 +461,9 @@ private void joinFun(Object pollObj, HashBasedTable mappingTable = ((JoinInfo) pollObj).getTableFieldRef(); + // verify whether join's columns exists in table. + checkConditionFieldsInTable(joinInfo.getCondition(), joinScope, sideTableInfo); + //获取两个表的所有字段 List sideJoinFieldInfo = ParserJoinField.getRowTypeInfo(joinInfo.getSelectNode(), joinScope, true); //通过join的查询字段信息过滤出需要的字段信息 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 6b6f9fe1b..f825180e4 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) throws Exception { BaseAllReqRow allReqRow = loadFlatMap(sideType, sqlRootDir, rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); - 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 40b7a7e82..f8fa9c3ef 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 @@ -49,7 +49,7 @@ public abstract class AbstractTableInfo implements Serializable { private final List fieldList = Lists.newArrayList(); /**key:别名, value: realField */ - private Map physicalFields = Maps.newHashMap(); + private Map physicalFields = Maps.newLinkedHashMap(); private final List fieldTypeList = Lists.newArrayList(); 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 b5c463cb6..3df278d5e 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 @@ -24,11 +24,14 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.java.tuple.Tuple2; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; /** * Reason: @@ -43,14 +46,16 @@ 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)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 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(); - private Map handlerMap = Maps.newHashMap(); + private final Map patternMap = Maps.newHashMap(); + + private final Map handlerMap = Maps.newHashMap(); public AbstractTableParser() { addParserHandler(PRIMARY_KEY, primaryKeyPattern, this::dealPrimaryKey); @@ -93,35 +98,28 @@ public void parseFieldsInfo(String fieldsInfo, AbstractTableInfo tableInfo) { throw new RuntimeException(String.format("table [%s],exists field empty.", tableInfo.getName())); } - String[] fieldInfoArr = fieldRow.split("\\s+"); - - String errorMsg = String.format("table [%s] field [%s] format error.", tableInfo.getName(), fieldRow); - Preconditions.checkState(fieldInfoArr.length >= 2, errorMsg); - boolean isMatcherKey = dealKeyPattern(fieldRow, tableInfo); if (isMatcherKey) { continue; } - //Compatible situation may arise in space in the fieldName - String[] filedNameArr = new String[fieldInfoArr.length - 1]; - System.arraycopy(fieldInfoArr, 0, filedNameArr, 0, fieldInfoArr.length - 1); - String fieldName = String.join(" ", filedNameArr); - String fieldType = fieldInfoArr[fieldInfoArr.length - 1 ].trim(); + Tuple2 t = extractType(fieldRow, tableInfo.getName()); + String fieldName = t.f0; + String fieldType = t.f1; - Class fieldClass = null; + 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.valueOf(matcher.group(1))); + fieldExtraInfo.setLength(Integer.parseInt(matcher.group(1))); } else { fieldClass = dbTypeConvertToJavaType(fieldType); } - tableInfo.addPhysicalMappings(fieldInfoArr[0], fieldInfoArr[0]); + tableInfo.addPhysicalMappings(fieldName, fieldName); tableInfo.addField(fieldName); tableInfo.addFieldClass(fieldClass); tableInfo.addFieldType(fieldType); @@ -131,11 +129,25 @@ public void parseFieldsInfo(String fieldsInfo, AbstractTableInfo tableInfo) { tableInfo.finish(); } + private Tuple2 extractType(String fieldRow, String tableName) { + Matcher matcher = typePattern.matcher(fieldRow); + if (matcher.matches()) { + String fieldName = matcher.group(1); + String fieldType = matcher.group(2); + return Tuple2.of(fieldName, fieldType); + } else { + String errorMsg = String.format("table [%s] field [%s] format error.", tableName, fieldRow); + throw new RuntimeException(errorMsg); + } + } + public void dealPrimaryKey(Matcher matcher, AbstractTableInfo tableInfo) { - String primaryFields = matcher.group(1).trim(); - String[] splitArry = primaryFields.split(","); - List primaryKes = Lists.newArrayList(splitArry); - tableInfo.setPrimaryKeys(primaryKes); + String primaryFields = matcher.group(2).trim(); + List primaryKeys = Arrays + .stream(primaryFields.split(",")) + .map(String::trim) + .collect(Collectors.toList()); + tableInfo.setPrimaryKeys(primaryKeys); } /** 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..149cd9cc3 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": @@ -48,8 +54,10 @@ public static Class stringConvertClass(String str) { return Boolean.class; case "smallint": + return Short.class; case "smallintunsigned": case "tinyint": + return Byte.class; case "tinyintunsigned": case "mediumint": case "mediumintunsigned": @@ -68,9 +76,10 @@ public static Class stringConvertClass(String str) { case "varchar": case "char": - case "text": case "string": return String.class; + case "text": + throw new IllegalArgumentException(str + " type is not support, please use STRING. "); case "real": case "float": 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 72a8b4b66..bb15b9c45 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 @@ -20,6 +20,7 @@ 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; @@ -30,6 +31,8 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; +import static org.apache.commons.lang3.StringUtils.split; + /** * @program: flink.sql * @author: wuren @@ -39,28 +42,13 @@ 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 = ' '; 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和其他基本类型 * @param arrayTypeString @@ -86,6 +74,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里只支持基本类型 * @param rowTypeString @@ -104,6 +116,7 @@ public static RowTypeInfo convertToRow(String rowTypeString) { return new RowTypeInfo(info.f0, info.f1); } + 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/DateUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DateUtil.java index b05de0f16..bb48b0678 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DateUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DateUtil.java @@ -59,6 +59,7 @@ public class DateUtil { private static final Pattern DATETIME = Pattern.compile("^\\d{4}-(?:0[0-9]|1[0-2])-[0-9]{2}T\\d{2}:\\d{2}:\\d{2}(\\.\\d{3,9})?Z$"); private static final Pattern DATE = Pattern.compile("^\\d{4}-(?:0[0-9]|1[0-2])-[0-9]{2}$"); + private static final Pattern TIME = Pattern.compile("^\\d{2}:\\d{2}:\\d{2}(\\.\\d{3,9})?Z$"); private static final int MILLIS_PER_SECOND = 1000; @@ -832,4 +833,16 @@ public static java.sql.Date getDateFromStr(String dateStr) { return null == date ? null : new java.sql.Date(date.getTime()); } + public static java.sql.Time getTimeFromStr(String dateStr) { + if (TIME.matcher(dateStr).matches()) { + dateStr = dateStr.substring(0,dateStr.length()-1); + Instant instant = LocalTime.parse(dateStr).atDate(LocalDate.now()).toInstant(ZoneOffset.UTC); + return new java.sql.Time(instant.toEpochMilli()); + } else if (DATETIME.matcher(dateStr).matches()) { + Instant instant = Instant.from(ISO_INSTANT.parse(dateStr)); + return new java.sql.Time(instant.toEpochMilli()); + } + Date date = stringToDate(dateStr); + return null == date ? null : new java.sql.Time(date.getTime()); + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtFileUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/DtFileUtils.java new file mode 100644 index 000000000..eb1c974df --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtFileUtils.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flink.sql.util; + +import org.apache.flink.util.Preconditions; + +import java.io.File; + +/** + * @program: flinkStreamSQL + * @author: wuren + * @create: 2020/09/21 + **/ +public class DtFileUtils { + public static void checkExists(String path) { + File file = new File(path); + String errorMsg = "%s file is not exist!"; + Preconditions.checkState(file.exists(), errorMsg, path); + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index 3e2ebb82a..bfa0c7ca5 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -21,15 +21,17 @@ package com.dtstack.flink.sql.util; import com.dtstack.flink.sql.enums.ColumnType; -import org.apache.commons.lang3.StringUtils; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Strings; import com.google.common.collect.Maps; -import com.fasterxml.jackson.databind.ObjectMapper; -import java.sql.Timestamp; +import org.apache.commons.lang3.StringUtils; + import java.math.BigDecimal; +import java.sql.Timestamp; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -167,26 +169,20 @@ public static String replaceIgnoreQuota(String str, String oriStr, String replac public static String dealSqlComment(String sql) { boolean inQuotes = false; boolean inSingleQuotes = false; - int bracketLeftNum = 0; StringBuilder b = new StringBuilder(sql.length()); char[] chars = sql.toCharArray(); for (int index = 0; index < chars.length; index ++) { - if (index == chars.length) { - return b.toString(); - } StringBuilder tempSb = new StringBuilder(2); - if (index > 1) { + if (index >= 1) { tempSb.append(chars[index - 1]); tempSb.append(chars[index]); } - if (tempSb.toString().equals("--")) { + if ("--".equals(tempSb.toString())) { if (inQuotes) { b.append(chars[index]); } else if (inSingleQuotes) { b.append(chars[index]); - } else if (bracketLeftNum > 0) { - b.append(chars[index]); } else { b.deleteCharAt(b.length() - 1); while (chars[index] != '\n') { @@ -247,6 +243,9 @@ public static String col2string(Object column, String type) { case DATE: result = DateUtil.dateToString((java.util.Date)column); break; + case TIME: + result = DateUtil.getTimeFromStr(String.valueOf(column)); + break; case TIMESTAMP: result = DateUtil.timestampToString((java.util.Date)column); break; @@ -379,8 +378,7 @@ public static String getTableFullPath(String schema, String tableName) { return addQuoteForStr(tableName); } - String schemaAndTabName = addQuoteForStr(schema) + "." + addQuoteForStr(tableName); - return schemaAndTabName; + return addQuoteForStr(schema) + "." + addQuoteForStr(tableName); } /** @@ -413,4 +411,19 @@ public static String getStartQuote() { public static String getEndQuote() { return "\""; } + + public static String removeStartAndEndQuota(String str) { + String removeStart = StringUtils.removeStart(str, "'"); + return StringUtils.removeEnd(removeStart, "'"); + } + + /** + * 判断当前对象是null 还是空 + * + * @param obj 需要判断的对象 + * @return 返回true 如果对象是空或者为null + */ + public static boolean isEmptyOrNull(Object obj) { + return Objects.isNull(obj) || obj.toString().isEmpty(); + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/JDBCUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/JDBCUtils.java index faab22172..42d7b9356 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/JDBCUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/JDBCUtils.java @@ -19,9 +19,19 @@ package com.dtstack.flink.sql.util; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +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) { @@ -44,4 +54,74 @@ public synchronized static void forName(String clazz) { throw new RuntimeException(e); } } + + /** + * 关闭连接资源 + * + * @param rs ResultSet + * @param stmt Statement + * @param conn Connection + * @param commit + */ + public static void closeConnectionResource(ResultSet rs, Statement stmt, Connection conn, boolean commit) { + if (Objects.nonNull(rs)) { + try { + rs.close(); + } catch (SQLException e) { + LOG.warn("Close resultSet error: {}", e.getMessage()); + } + } + + if (Objects.nonNull(stmt)) { + try { + stmt.close(); + } catch (SQLException e) { + LOG.warn("Close statement error:{}", e.getMessage()); + } + } + + if (Objects.nonNull(conn)) { + try { + if (commit) { + commit(conn); + } else { + rollBack(conn); + } + + conn.close(); + } catch (SQLException e) { + LOG.warn("Close connection error:{}", e.getMessage()); + } + } + } + + /** + * 手动提交事物 + * + * @param conn Connection + */ + public static void commit(Connection conn) { + try { + if (!conn.isClosed() && !conn.getAutoCommit()) { + conn.commit(); + } + } catch (SQLException e) { + LOG.warn("commit error:{}", e.getMessage()); + } + } + + /** + * 手动回滚事物 + * + * @param conn Connection + */ + public static void rollBack(Connection conn) { + try { + if (!conn.isClosed() && !conn.getAutoCommit()) { + conn.rollback(); + } + } catch (SQLException e) { + LOG.warn("rollBack error:{}", e.getMessage()); + } + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/KrbUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/KrbUtils.java new file mode 100644 index 000000000..00f2f538d --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/util/KrbUtils.java @@ -0,0 +1,63 @@ +/* + * 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 org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authentication.util.KerberosName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import sun.security.krb5.Config; +import sun.security.krb5.KrbException; + +import java.io.IOException; + +/** + * @program: flinkStreamSQL + * @author: wuren + * @create: 2020/09/14 + **/ +public class KrbUtils { + + private static final Logger LOG = LoggerFactory.getLogger(KrbUtils.class); + + public static final String KRB5_CONF_KEY = "java.security.krb5.conf"; + public static final String HADOOP_AUTH_KEY = "hadoop.security.authentication"; + public static final String KRB_STR = "Kerberos"; +// public static final String FALSE_STR = "false"; +// public static final String SUBJECT_ONLY_KEY = "javax.security.auth.useSubjectCredsOnly"; + + public static UserGroupInformation loginAndReturnUgi(String principal, String keytabPath, String krb5confPath) throws IOException { LOG.info("Kerberos login with principal: {} and keytab: {}", principal, keytabPath); + System.setProperty(KRB5_CONF_KEY, krb5confPath); + // 不刷新会读/etc/krb5.conf + try { + Config.refresh(); + KerberosName.resetDefaultRealm(); + } catch (KrbException e) { + LOG.warn("resetting default realm failed, current default realm will still be used.", e); + } + // TODO 尚未探索出此选项的意义,以后研究明白方可打开 +// System.setProperty(SUBJECT_ONLY_KEY, FALSE_STR); + Configuration configuration = new Configuration(); + configuration.set(HADOOP_AUTH_KEY , KRB_STR); + UserGroupInformation.setConfiguration(configuration); + return UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytabPath); + } + +} diff --git a/core/src/main/java/com/dtstack/flink/sql/util/MathUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/MathUtil.java index c5584b5d8..844b87033 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/MathUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/MathUtil.java @@ -22,7 +22,9 @@ import java.math.BigDecimal; import java.math.BigInteger; import java.sql.Date; +import java.sql.Time; import java.sql.Timestamp; +import java.util.Objects; /** * Convert val to specified numeric type @@ -234,7 +236,19 @@ public static Date getDate(Object obj) { throw new RuntimeException("not support type of " + obj.getClass() + " convert to Date."); } - + public static Time getTime(Object obj) { + if (Objects.isNull(obj)) { + return null; + } + if (obj instanceof String) { + return DateUtil.getTimeFromStr((String) obj); + } else if (obj instanceof Timestamp) { + return new Time(((Timestamp) obj).getTime()); + } else if (obj instanceof Time) { + return (Time) obj; + } + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Time."); + } public static Timestamp getTimestamp(Object obj) { if (obj == null) { diff --git a/core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java index 2aeb06cf8..28678cfb4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java @@ -232,6 +232,13 @@ public static String dealSelectResultWithJoinInfo(JoinInfo joinInfo, SqlSelect s queueInfo.offer(joinInfo.getLeftNode()); } + if (joinInfo.getLeftNode().getKind() == AS) { + SqlNode leftSqlNode = ((SqlBasicCall) joinInfo.getLeftNode()).getOperands()[0]; + if (leftSqlNode.getKind() == UNION) { + queueInfo.offer(joinInfo.getLeftNode()); + } + } + queueInfo.offer(joinInfo); } replaceFromNodeForJoin(joinInfo, sqlNode); @@ -301,7 +308,7 @@ public static void getFromTableInfo(SqlNode fromTable, Set tableNameSet) public static void replaceSelectFieldTable(SqlNode selectNode, String oldTbName, String newTbName, - HashBiMap fieldReplaceRef) { + Map fieldReplaceRef) { if (selectNode.getKind() == AS) { SqlNode leftNode = ((SqlBasicCall) selectNode).getOperands()[0]; replaceSelectFieldTable(leftNode, oldTbName, newTbName, fieldReplaceRef); @@ -395,22 +402,13 @@ public static void replaceSelectFieldTable(SqlNode selectNode, private static void replaceOneSelectField(SqlIdentifier sqlIdentifier, String newTbName, String oldTbName, - HashBiMap fieldReplaceRef){ + Map fieldReplaceRef){ SqlIdentifier newField = sqlIdentifier.setName(0, newTbName); String fieldName = sqlIdentifier.names.get(1); - String fieldKey = oldTbName + "_" + fieldName; - - if(!fieldReplaceRef.containsKey(fieldKey)){ - if(fieldReplaceRef.inverse().get(fieldName) != null){ - //换一个名字 - String mappingFieldName = ParseUtils.dealDuplicateFieldName(fieldReplaceRef, fieldName); - newField = newField.setName(1, mappingFieldName); - fieldReplaceRef.put(fieldKey, mappingFieldName); - } else { - fieldReplaceRef.put(fieldKey, fieldName); - } - }else { - newField = newField.setName(1, fieldReplaceRef.get(fieldKey)); + String fieldKey = oldTbName + "." + fieldName; + if(fieldReplaceRef.get(fieldKey) != null){ + String newFieldName = fieldReplaceRef.get(fieldKey).split("\\.")[1]; + newField = newField.setName(1, newFieldName); } sqlIdentifier.assignNamesFrom(newField); @@ -511,7 +509,7 @@ public static String getTargetRefField(Map refFieldMap, String c return preFieldName; } - public static void replaceWhereCondition(SqlNode parentWhere, String oldTbName, String newTbName, HashBiMap fieldReplaceRef){ + public static void replaceWhereCondition(SqlNode parentWhere, String oldTbName, String newTbName, Map fieldReplaceRef){ if(parentWhere == null){ return; @@ -527,7 +525,7 @@ public static void replaceWhereCondition(SqlNode parentWhere, String oldTbName, } } - private static void replaceConditionNode(SqlNode selectNode, String oldTbName, String newTbName, HashBiMap fieldReplaceRef) { + private static void replaceConditionNode(SqlNode selectNode, String oldTbName, String newTbName, Map fieldReplaceRef) { if(selectNode.getKind() == IDENTIFIER){ SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; @@ -537,11 +535,25 @@ private static void replaceConditionNode(SqlNode selectNode, String oldTbName, S String tableName = sqlIdentifier.names.asList().get(0); String tableField = sqlIdentifier.names.asList().get(1); - String fieldKey = tableName + "_" + tableField; + String fieldKey = tableName + "." + tableField; if(tableName.equalsIgnoreCase(oldTbName)){ - String newFieldName = fieldReplaceRef.get(fieldKey) == null ? tableField : fieldReplaceRef.get(fieldKey); + /* + * ****Before replace:***** + * fieldKey: b.department + * fieldReplaceRef : b.department -> a_b_0.department0 + * oldFieldRef: a_b_0.department0 + * oldTbName: b + * oldFieldName: department + * ****After replace:***** + * newTbName: a_b_0 + * newFieldName: department0 + */ + String oldFieldRef = fieldReplaceRef.get(fieldKey); + String newFieldName = (oldFieldRef != null && !StringUtils.substringAfter(oldFieldRef, ".").isEmpty()) ? + StringUtils.substringAfter(oldFieldRef, ".") : tableField; + SqlIdentifier newField = ((SqlIdentifier)selectNode).setName(0, newTbName); newField = newField.setName(1, newFieldName); ((SqlIdentifier)selectNode).assignNamesFrom(newField); diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/AbstractCustomerWaterMarker.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/AbstractCustomerWaterMarker.java index d75d26a61..9991c7021 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/AbstractCustomerWaterMarker.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/AbstractCustomerWaterMarker.java @@ -92,7 +92,6 @@ public IterationRuntimeContext getIterationRuntimeContext() { public void setRuntimeContext(RuntimeContext t) { this.runtimeContext = t; eventDelayGauge = new EventDelayGauge(); - t.getMetricGroup().getAllVariables().put("", fromSourceTag); t.getMetricGroup().gauge(MetricConstant.DT_EVENT_DELAY_GAUGE, eventDelayGauge); } diff --git a/core/src/test/java/com/dtstack/flink/sql/parser/SqlParserTest.java b/core/src/test/java/com/dtstack/flink/sql/parser/SqlParserTest.java new file mode 100644 index 000000000..fdbb03a75 --- /dev/null +++ b/core/src/test/java/com/dtstack/flink/sql/parser/SqlParserTest.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flink.sql.parser; + +import org.junit.Assert; +import org.junit.Test; +import org.powermock.reflect.Whitebox; + +import java.util.ArrayList; +import java.util.List; + +/** + * @program: flink.sql + * @author: wuren + * @create: 2020/09/15 + **/ +public class SqlParserTest { + + @Test + public void testRemoveAddFileStmt() throws Exception { + List rawStmts = new ArrayList<>(); + String sql1 = " add file asdasdasd "; + String sql2 = " aDd fIle With asdasdasd "; + String sql3 = " INSERT INTO dwd_foo SELECT id, name FROM ods_foo"; + String sql4 = " ADD FILE asb "; + rawStmts.add(sql1); + rawStmts.add(sql2); + rawStmts.add(sql3); + rawStmts.add(sql4); + + List stmts = Whitebox.invokeMethod(SqlParser.class, "removeAddFileStmt", rawStmts); + Assert.assertEquals(stmts.get(0), sql3); + } + +} \ No newline at end of file diff --git a/core/src/test/java/com/dtstack/flink/sql/util/KrbUtilsTest.java b/core/src/test/java/com/dtstack/flink/sql/util/KrbUtilsTest.java new file mode 100644 index 000000000..5d65158c7 --- /dev/null +++ b/core/src/test/java/com/dtstack/flink/sql/util/KrbUtilsTest.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flink.sql.util; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +/** + * @program: flinkStreamSQL + * @author: wuren + * @create: 2020/09/14 + **/ +public class KrbUtilsTest { + @Test + public void testGetUgi() throws IOException { + String principal = ""; + String keytabPath = ""; + String krb5confPath = ""; + try { + KrbUtils.loginAndReturnUgi(principal, keytabPath, krb5confPath); + } catch (IllegalArgumentException e) { + Assert.assertEquals(e.getMessage(), "Can't get Kerberos realm"); + } + + } +} \ No newline at end of file diff --git a/docs/plugin/hbaseSink.md b/docs/plugin/hbaseSink.md index 5006f11a2..a5ad12067 100644 --- a/docs/plugin/hbaseSink.md +++ b/docs/plugin/hbaseSink.md @@ -7,7 +7,7 @@ CREATE TABLE MyResult( type ='hbase', zookeeperQuorum ='ip:port[,ip:port]', tableName ='tableName', - rowKey ='colName[,colName]', + rowKey ='colName[+colName]', parallelism ='1', zookeeperParent ='/hbase' ) @@ -34,7 +34,7 @@ hbase2.0 |zookeeperQuorum | hbase zk地址,多个直接用逗号隔开|是|| |zookeeperParent | zkParent 路径|是|| |tableName | 关联的hbase表名称|是|| -|rowkey | hbase的rowkey关联的列信息,多个值以逗号隔开|是|| +|rowkey | hbase的rowkey关联的列信息,多个值以'+'连接|是|| |updateMode|APPEND:不回撤数据,只下发增量数据,UPSERT:先删除回撤数据,然后更新|否|APPEND| |parallelism | 并行度设置|否|1| |kerberosAuthEnable | 是否开启kerberos认证|否|false| @@ -76,7 +76,7 @@ CREATE TABLE MyResult( tableName ='myresult', partitionedJoin ='false', parallelism ='1', - rowKey='name,channel' + rowKey='name+channel' ); insert @@ -141,7 +141,7 @@ into ## 6.hbase数据 ### 数据内容说明 -hbase的rowkey 构建规则:以描述的rowkey字段值作为key,多个字段以'-'连接 +hbase的rowkey 构建规则:以描述的rowkey字段值作为key,多个字段以'+'连接 ### 数据内容示例 hbase(main):007:0> scan 'myresult' ROW COLUMN+CELL 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/kuduSide.md b/docs/plugin/kuduSide.md index a2636d4aa..3075c3f35 100644 --- a/docs/plugin/kuduSide.md +++ b/docs/plugin/kuduSide.md @@ -60,6 +60,11 @@ | isFaultTolerant |查询是否容错 查询失败是否扫描第二个副本 默认false 容错 | 否|| | cache | 维表缓存策略(NONE/LRU/ALL)|否|NONE| | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| +| principal |kerberos用于登录的principal | 否|| +| keytab |keytab文件的路径 | 否|| +| krb5conf |conf文件路径 | 否|| +Kerberos三个参数全部设置则开启Kerberos认证,如果缺少任何一个则会提示缺少参数错误。 +如果全部未设置则不开启Kerberos连接Kudu集群。 -------------- ## 5.样例 @@ -163,3 +168,20 @@ into on t1.id = t2.id; ``` +## 7.kerberos示例 +``` +create table dim ( + name varchar, + id int, + PERIOD FOR SYSTEM_TIME +) WITH ( + type='kudu', + kuduMasters='host1', + tableName='foo', + parallelism ='1', + cache ='ALL', + keytab='foo/foobar.keytab', + krb5conf='bar/krb5.conf', + principal='kudu/host1@DTSTACK.COM' +); +``` \ No newline at end of file diff --git a/docs/plugin/kuduSink.md b/docs/plugin/kuduSink.md index cc8f8535a..7eb907770 100644 --- a/docs/plugin/kuduSink.md +++ b/docs/plugin/kuduSink.md @@ -11,7 +11,154 @@ CREATE TABLE tableName( writeMode='upsert', workerCount='1', defaultOperationTimeoutMs='600000', - defaultSocketReadTimeoutMs='6000000', + defaultSocketReadT## 1.格式: + ``` + CREATE TABLE tableName( + colName colType, + ... + colNameX colType + )WITH( + type ='kudu', + kuduMasters ='ip1,ip2,ip3', + tableName ='impala::default.test', + writeMode='upsert', + workerCount='1', + defaultOperationTimeoutMs='600000', + defaultSocketReadTimeoutMs='6000000', + parallelism ='parllNum' + ); + + + ``` + + ## 2.支持版本 + kudu 1.9.0+cdh6.2.0 + + ## 3.表结构定义 + + |参数名称|含义| + |----|---| + | tableName | 在 sql 中使用的名称;即注册到flink-table-env上的名称 + | colName | 列名称,redis中存储为 表名:主键名:主键值:列名]| + | colType | 列类型 [colType支持的类型](../colType.md)| + + + ## 4.参数: + + |参数名称|含义|是否必填|默认值| + |----|---|---|-----| + | type | 表名 输出表类型[mysq|hbase|elasticsearch|redis|kudu]|是|| + | kuduMasters | kudu master节点的地址;格式ip[ip,ip2]|是|| + | tableName | kudu 的表名称|是|| + | writeMode | 写入kudu的模式 insert|update|upsert |否 |upsert + | workerCount | 工作线程数 |否| + | defaultOperationTimeoutMs | 操作超时时间 |否| + | defaultSocketReadTimeoutMs | socket读取超时时间 |否| + | parallelism | 并行度设置|否|1| + | principal |kerberos用于登录的principal | 否|| + | keytab |keytab文件的路径 | 否|| + | krb5conf |conf文件路径 | 否|| + Kerberos三个参数全部设置则开启Kerberos认证,如果缺少任何一个则会提示缺少参数错误。 + 如果全部未设置则不开启Kerberos连接Kudu集群。 + + ## 5.样例: + ``` + CREATE TABLE MyTable( + channel varchar, + name varchar, + pv varchar, + a varchar, + b varchar + )WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='es_test', + timezone='Asia/Shanghai', + updateMode ='append', + enableKeyPartitions ='false', + topicIsPattern ='false', + parallelism ='1' + ); + + CREATE TABLE MyResult( + a string, + b string, + c string, + d string + )WITH( + type ='kudu', + kuduMasters ='cdh03.cdhsite:7051', + tableName ='myresult', + writeMode='insert', + parallelism ='1' + ); + + CREATE TABLE sideTable( + c string, + d string, + PRIMARY KEY(c) , + PERIOD FOR SYSTEM_TIME + )WITH( + type ='kudu', + kuduMasters ='cdh03.cdhsite:7051', + tableName ='sidetest4', + partitionedJoin ='false', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + primaryKey ='c', + isFaultTolerant ='false' + ); + + insert + into + MyResult + select + MyTable.a, + MyTable.b, + s.c, + s.d + from + MyTable + join + sideTable s + on MyTable.a = s.c + where + MyTable.a='2' + and s.d='2' + + ``` + + ## 6.数据示例 + ### 输入数据 + ``` + {"channel":"daishuyun","name":"roc","pv":"10","a":"2","b":"2"} + ``` + ### 结果数据 + ``` + {"a":"2","b":"2","c":"3","d":"4"} + ``` + + ## 7.kerberos示例 + ``` + create table dwd ( + name varchar, + id int + ) WITH ( + type='kudu', + kuduMasters='host1', + tableName='foo', + writeMode='insert', + parallelism ='1', + keytab='foo/foobar.keytab', + krb5conf='bar/krb5.conf', + principal='kudu/host1@DTSTACK.COM' + ); + ``` +imeoutMs='6000000', parallelism ='parllNum' ); 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 515410c6e..0ad18338a 100644 --- a/elasticsearch5/elasticsearch5-sink/pom.xml +++ b/elasticsearch5/elasticsearch5-sink/pom.xml @@ -19,24 +19,19 @@ 5.3.3 - - ch.qos.logback - logback-core - 1.1.7 - - - - ch.qos.logback - logback-classic - 1.1.7 - - org.elasticsearch.client x-pack-transport 5.3.3 + + + org.slf4j + slf4j-log4j12 + 1.6.1 + + org.apache.logging.log4j log4j-to-slf4j diff --git a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java index 23213e721..9c68ffea7 100644 --- a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java +++ b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.sink.elasticsearch; +import com.dtstack.flink.sql.util.DateUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.java.tuple.Tuple2; @@ -32,6 +33,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.sql.Date; +import java.sql.Timestamp; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -107,6 +110,14 @@ private IndexRequest createIndexRequest(Row element) { Map dataMap = EsUtil.rowToJsonMap(element,fieldNames,fieldTypes); int length = Math.min(element.getArity(), fieldNames.size()); for(int i=0; i, IStreamSi private TypeInformation[] fieldTypes; - private int parallelism = -1; + private int parallelism = 1; + + private String registerTableName; private ElasticsearchTableInfo esTableInfo; @@ -149,7 +152,7 @@ private RichSinkFunction createEsSinkFunction(){ @Override public void emitDataStream(DataStream> dataStream) { RichSinkFunction richSinkFunction = createEsSinkFunction(); - DataStreamSink streamSink = dataStream.addSink(richSinkFunction); + DataStreamSink streamSink = dataStream.addSink(richSinkFunction).name(registerTableName); if(parallelism > 0){ streamSink.setParallelism(parallelism); } @@ -176,6 +179,9 @@ public ElasticsearchSink genStreamSink(AbstractTargetTableInfo targetTableInfo) String id = elasticsearchTableInfo.getId(); String[] idField = StringUtils.split(id, ","); idIndexList = new ArrayList<>(); + parallelism = Objects.isNull(elasticsearchTableInfo.getParallelism()) ? + parallelism : elasticsearchTableInfo.getParallelism(); + registerTableName = elasticsearchTableInfo.getName(); for(int i = 0; i < idField.length; ++i) { idIndexList.add(Integer.valueOf(idField[i])); diff --git a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java index 7988e597c..4117ad961 100644 --- a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java +++ b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java @@ -50,6 +50,8 @@ public class ElasticsearchSinkParser extends AbstractTableParser { private static final String KEY_ES_PASSWORD = "password"; + private static final String KEY_ES_PARALLELISM = "parallelism"; + @Override protected boolean fieldNameNeedsUpperCase() { return false; @@ -65,6 +67,7 @@ public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map sqlJoinCompareOperate = Lists.newArrayList(); + private static final Integer MAX_ROW_NUM = 50000; public Elasticsearch6AsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new Elasticsearch6AsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); @@ -92,7 +93,7 @@ public void handleAsyncInvoke(Map inputParams, CRow input, Resul String key = buildCacheKey(inputParams); BoolQueryBuilder boolQueryBuilder = Es6Util.setPredicateclause(sideInfo); boolQueryBuilder = setInputParams(inputParams, boolQueryBuilder); - SearchSourceBuilder searchSourceBuilder = initConfiguration(); + SearchSourceBuilder searchSourceBuilder = initConfiguration(inputParams); searchSourceBuilder.query(boolQueryBuilder); searchRequest.source(searchSourceBuilder); @@ -116,6 +117,11 @@ public void onResponse(SearchResponse searchResponse) { if (searchHits.length < getFetchSize()) { break; } + //protect memory + if (rowList.size() >= MAX_ROW_NUM) { + LOG.warn("row size beyond limit"); + break; + } if (tableInfo == null && tmpRhlClient == null) { // create new connection to fetch data tableInfo = (Elasticsearch6SideTableInfo) sideInfo.getSideTableInfo(); @@ -225,10 +231,11 @@ public void close() throws Exception { } - private SearchSourceBuilder initConfiguration() { + private SearchSourceBuilder initConfiguration(Map inputParams) { SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); searchSourceBuilder.size(getFetchSize()); searchSourceBuilder.sort("_id", SortOrder.DESC); + inputParams.keySet().stream().forEach(k -> searchSourceBuilder.sort(k, SortOrder.DESC)); String[] sideFieldNames = StringUtils.split(sideInfo.getSideSelectFields().trim(), ","); searchSourceBuilder.fetchSource(sideFieldNames, null); diff --git a/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/ClassUtil.java b/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/ClassUtil.java index 90ac58229..4dd1c4300 100644 --- a/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/ClassUtil.java +++ b/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/ClassUtil.java @@ -15,12 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - - package com.dtstack.flink.sql.side.elasticsearch6.util; import com.dtstack.flink.sql.util.DateUtil; +import java.lang.reflect.Array; import java.math.BigDecimal; import java.sql.Date; import java.sql.Time; @@ -30,13 +29,19 @@ * Reason: TODO ADD REASON(可选) * Date: 2017年03月10日 下午1:16:37 * Company: www.dtstack.com - * * @author sishu.yss */ public class ClassUtil { public static Class stringConvertClass(String str) { - switch (str.toLowerCase()) { + + // 这部分主要是告诉Class转TypeInfomation的方法,字段是Array类型 + String lowerStr = str.toLowerCase().trim(); + if (lowerStr.startsWith("array")) { + return Array.newInstance(Integer.class, 0).getClass(); + } + + switch (lowerStr) { case "boolean": case "bit": return Boolean.class; @@ -54,7 +59,6 @@ public static Class stringConvertClass(String str) { case "blob": return Byte.class; - case "long": case "bigint": case "intunsigned": case "integerunsigned": @@ -63,8 +67,10 @@ public static Class stringConvertClass(String str) { case "varchar": case "char": - case "text": + case "string": return String.class; + case "text": + throw new IllegalArgumentException(str + " type is not support, please use STRING. "); case "real": case "float": @@ -90,9 +96,10 @@ public static Class stringConvertClass(String str) { case "decimalunsigned": return BigDecimal.class; default: - throw new RuntimeException("不支持 " + str + " 类型"); + break; } + throw new RuntimeException("不支持 " + str + " 类型"); } public static Object convertType(Object field, String fromType, String toType) { diff --git a/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/Es6Util.java b/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/Es6Util.java index bc5637515..4a25b43d9 100644 --- a/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/Es6Util.java +++ b/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/Es6Util.java @@ -171,6 +171,9 @@ public static BoolQueryBuilder buildFilterCondition(BoolQueryBuilder boolQueryBu return boolQueryBuilder.must(QueryBuilders.existsQuery(info.getFieldName())); case "=": case "EQUALS": + if(StringUtils.isBlank(info.getCondition())){ + return boolQueryBuilder; + } return boolQueryBuilder.must(QueryBuilders.termQuery(textConvertToKeyword(info.getFieldName(), sideInfo), removeSpaceAndApostrophe(info.getCondition())[0])); case "<>": case "NOT_EQUALS": diff --git a/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/MathUtil.java b/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/MathUtil.java index d72530a56..faf256145 100644 --- a/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/MathUtil.java +++ b/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/MathUtil.java @@ -19,22 +19,23 @@ package com.dtstack.flink.sql.side.elasticsearch6.util; +import com.dtstack.flink.sql.util.DateUtil; + import java.math.BigDecimal; import java.math.BigInteger; import java.sql.Date; +import java.sql.Time; import java.sql.Timestamp; -import java.text.ParseException; -import java.text.SimpleDateFormat; +import java.util.Objects; /** + * Convert val to specified numeric type * Date: 2017/4/21 * Company: www.dtstack.com - * * @author xuchao */ public class MathUtil { - public static Long getLongVal(Object obj) { if (obj == null) { return null; @@ -103,8 +104,6 @@ public static Float getFloatVal(Object obj) { return (Float) obj; } else if (obj instanceof BigDecimal) { return ((BigDecimal) obj).floatValue(); - } else if (obj instanceof Double) { - return ((Double) obj).floatValue(); } throw new RuntimeException("not support type of " + obj.getClass() + " convert to Float."); @@ -126,9 +125,13 @@ public static Double getDoubleVal(Object obj) { if (obj instanceof String) { return Double.valueOf((String) obj); } else if (obj instanceof Float) { + return ((Float) obj).doubleValue(); + } else if (obj instanceof Double) { return (Double) obj; } else if (obj instanceof BigDecimal) { return ((BigDecimal) obj).doubleValue(); + } else if (obj instanceof Integer) { + return ((Integer) obj).doubleValue(); } throw new RuntimeException("not support type of " + obj.getClass() + " convert to Double."); @@ -226,12 +229,7 @@ public static Date getDate(Object obj) { return null; } if (obj instanceof String) { - SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); - try { - return new Date(format.parse((String) obj).getTime()); - } catch (ParseException e) { - throw new RuntimeException("String convert to Date fail."); - } + return DateUtil.getDateFromStr((String) obj); } else if (obj instanceof Timestamp) { return new Date(((Timestamp) obj).getTime()); } else if (obj instanceof Date) { @@ -240,6 +238,20 @@ public static Date getDate(Object obj) { throw new RuntimeException("not support type of " + obj.getClass() + " convert to Date."); } + public static Time getTime(Object obj) { + if (Objects.isNull(obj)) { + return null; + } + if (obj instanceof String) { + return DateUtil.getTimeFromStr((String) obj); + } else if (obj instanceof Timestamp) { + return new Time(((Timestamp) obj).getTime()); + } else if (obj instanceof Time) { + return (Time) obj; + } + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Time."); + } + public static Timestamp getTimestamp(Object obj) { if (obj == null) { return null; @@ -249,7 +261,7 @@ public static Timestamp getTimestamp(Object obj) { } else if (obj instanceof Date) { return new Timestamp(((Date) obj).getTime()); } else if (obj instanceof String) { - return new Timestamp(getDate(obj).getTime()); + return DateUtil.getTimestampFromStr(obj.toString()); } throw new RuntimeException("not support type of " + obj.getClass() + " convert to Date."); } diff --git a/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/SwitchUtil.java b/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/SwitchUtil.java index e0aaa123e..76165a4a7 100644 --- a/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/SwitchUtil.java +++ b/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/util/SwitchUtil.java @@ -39,7 +39,6 @@ public static Object getTarget(Object obj, String targetType) { case "int": return MathUtil.getIntegerVal(obj); - case "long": case "bigint": case "bigintunsigned": case "intunsigned": @@ -77,6 +76,8 @@ public static Object getTarget(Object obj, String targetType) { case "timestamp": case "datetime": return MathUtil.getTimestamp(obj); + case "time": + return MathUtil.getTime(obj); default: return obj; } diff --git a/elasticsearch6/elasticsearch6-sink/pom.xml b/elasticsearch6/elasticsearch6-sink/pom.xml index cb5fb753c..8caa7eeb8 100644 --- a/elasticsearch6/elasticsearch6-sink/pom.xml +++ b/elasticsearch6/elasticsearch6-sink/pom.xml @@ -31,27 +31,34 @@ - ch.qos.logback - logback-core - 1.1.7 + log4j + log4j + 1.2.17 - ch.qos.logback - logback-classic - 1.1.7 + slf4j-api + org.slf4j + 1.7.15 + - org.apache.logging.log4j - log4j-to-slf4j - 2.7 + org.slf4j + slf4j-log4j12 + 1.6.1 org.apache.flink flink-connector-elasticsearch6_2.11 ${flink.version} + + + log4j-to-slf4j + org.apache.logging.log4j + + diff --git a/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java b/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java index d594ddd93..d0b0d314b 100644 --- a/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java +++ b/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java @@ -33,6 +33,8 @@ import org.slf4j.LoggerFactory; import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -115,6 +117,10 @@ private IndexRequest createIndexRequest(Row element) { dataMap.put(fieldNames.get(i), DateUtil.transformSqlDateToUtilDate((Date) element.getField(i))); continue; } + if (element.getField(i) instanceof Timestamp) { + dataMap.put(fieldNames.get(i), ((Timestamp) element.getField(i)).getTime()); + continue; + } dataMap.put(fieldNames.get(i), element.getField(i)); } diff --git a/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java b/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java index b7d9de6fc..398f03751 100644 --- a/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java +++ b/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java @@ -38,6 +38,7 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.stream.Collectors; /** @@ -67,7 +68,9 @@ public class ElasticsearchSink implements RetractStreamTableSink, IStreamSi private TypeInformation[] fieldTypes; - private int parallelism = -1; + private int parallelism = 1; + + private String registerTableName; private ElasticsearchTableInfo esTableInfo; @@ -121,7 +124,7 @@ private RichSinkFunction createEsSinkFunction() { @Override public void emitDataStream(DataStream> dataStream) { RichSinkFunction richSinkFunction = createEsSinkFunction(); - DataStreamSink streamSink = dataStream.addSink(richSinkFunction); + DataStreamSink streamSink = dataStream.addSink(richSinkFunction).name(registerTableName); if (parallelism > 0) { streamSink.setParallelism(parallelism); } @@ -135,7 +138,11 @@ public ElasticsearchSink genStreamSink(AbstractTargetTableInfo targetTableInfo) type = esTableInfo.getEsType(); columnTypes = esTableInfo.getFieldTypes(); esAddressList = Arrays.asList(esTableInfo.getAddress().split(",")); + this.bulkFlushMaxActions = esTableInfo.getBatchSize(); String id = esTableInfo.getId(); + parallelism = Objects.isNull(esTableInfo.getParallelism()) ? + parallelism : esTableInfo.getParallelism(); + registerTableName = esTableInfo.getName(); if (!StringUtils.isEmpty(id)) { idIndexList = Arrays.stream(StringUtils.split(id, ",")).map(Integer::valueOf).collect(Collectors.toList()); diff --git a/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java b/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java index 22c2b72bc..5c12a2da8 100644 --- a/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java +++ b/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java @@ -47,8 +47,12 @@ public class ElasticsearchSinkParser extends AbstractTableParser { private static final String KEY_ES6_PASSWORD = "password"; + public static final String BATCH_SIZE_KEY = "batchSize"; + private static final String KEY_TRUE = "true"; + private static final String KEY_PARALLELISM = "parallelism"; + @Override protected boolean fieldNameNeedsUpperCase() { return false; @@ -64,6 +68,8 @@ public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map 0) { + logger.debug(buildWarningMessage(request, host, warnings)); + } + } + if (tracer.isTraceEnabled()) { + String requestLine; + try { + requestLine = buildTraceRequest(request, host); + } catch(IOException e) { + requestLine = ""; + tracer.trace("error while reading request for trace purposes", e); + } + String responseLine; + try { + responseLine = buildTraceResponse(httpResponse); + } catch(IOException e) { + responseLine = ""; + tracer.trace("error while reading response for trace purposes", e); + } + tracer.trace(requestLine + '\n' + responseLine); + } + } + + /** + * Logs a request that failed + */ + static void logFailedRequest(Log logger, HttpUriRequest request, Node node, Exception e) { + if (logger.isDebugEnabled()) { + logger.debug("request [" + request.getMethod() + " " + node.getHost() + getUri(request.getRequestLine()) + "] failed", e); + } + if (tracer.isTraceEnabled()) { + String traceRequest; + try { + traceRequest = buildTraceRequest(request, node.getHost()); + } catch (IOException e1) { + tracer.trace("error while reading request for trace purposes", e); + traceRequest = ""; + } + tracer.trace(traceRequest); + } + } + + static String buildWarningMessage(HttpUriRequest request, HttpHost host, Header[] warnings) { + StringBuilder message = new StringBuilder("request [").append(request.getMethod()).append(" ").append(host) + .append(getUri(request.getRequestLine())).append("] returned ").append(warnings.length).append(" warnings: "); + for (int i = 0; i < warnings.length; i++) { + if (i > 0) { + message.append(","); + } + message.append("[").append(warnings[i].getValue()).append("]"); + } + return message.toString(); + } + + /** + * Creates curl output for given request + */ + static String buildTraceRequest(HttpUriRequest request, HttpHost host) throws IOException { + String requestLine = "curl -iX " + request.getMethod() + " '" + host + getUri(request.getRequestLine()) + "'"; + if (request instanceof HttpEntityEnclosingRequest) { + HttpEntityEnclosingRequest enclosingRequest = (HttpEntityEnclosingRequest) request; + if (enclosingRequest.getEntity() != null) { + requestLine += " -d '"; + HttpEntity entity = enclosingRequest.getEntity(); + if (entity.isRepeatable() == false) { + entity = new BufferedHttpEntity(enclosingRequest.getEntity()); + enclosingRequest.setEntity(entity); + } + requestLine += EntityUtils.toString(entity, StandardCharsets.UTF_8) + "'"; + } + } + return requestLine; + } + + /** + * Creates curl output for given response + */ + static String buildTraceResponse(HttpResponse httpResponse) throws IOException { + StringBuilder responseLine = new StringBuilder(); + responseLine.append("# ").append(httpResponse.getStatusLine()); + for (Header header : httpResponse.getAllHeaders()) { + responseLine.append("\n# ").append(header.getName()).append(": ").append(header.getValue()); + } + responseLine.append("\n#"); + HttpEntity entity = httpResponse.getEntity(); + if (entity != null) { + if (entity.isRepeatable() == false) { + entity = new BufferedHttpEntity(entity); + } + httpResponse.setEntity(entity); + ContentType contentType = ContentType.get(entity); + Charset charset = StandardCharsets.UTF_8; + if (contentType != null && contentType.getCharset() != null) { + charset = contentType.getCharset(); + } + try (BufferedReader reader = new BufferedReader(new InputStreamReader(entity.getContent(), charset))) { + String line; + while( (line = reader.readLine()) != null) { + responseLine.append("\n# ").append(line); + } + } + } + return responseLine.toString(); + } + + private static String getUri(RequestLine requestLine) { + if (requestLine.getUri().charAt(0) != '/') { + return "/" + requestLine.getUri(); + } + return requestLine.getUri(); + } +} diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java index 0250decae..c36c4affb 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java @@ -30,11 +30,18 @@ import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo; import com.dtstack.flink.sql.factory.DTThreadFactory; import com.dtstack.flink.sql.side.hbase.utils.HbaseConfigUtils; +import com.dtstack.flink.sql.util.DtFileUtils; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; import com.stumbleupon.async.Deferred; +import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.security.DynamicConfiguration; +import org.apache.flink.runtime.security.KerberosUtils; import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; @@ -45,7 +52,12 @@ import org.slf4j.LoggerFactory; import sun.security.krb5.KrbException; +import javax.security.auth.login.AppConfigurationEntry; +import java.io.File; +import java.lang.reflect.Field; +import java.security.PrivilegedExceptionAction; import java.sql.Timestamp; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.TimeZone; @@ -100,23 +112,33 @@ public void open(Configuration parameters) throws Exception { ExecutorService executorService =new ThreadPoolExecutor(DEFAULT_POOL_SIZE, DEFAULT_POOL_SIZE, 0L, TimeUnit.MILLISECONDS, - new LinkedBlockingQueue<>(), new DTThreadFactory("hbase-aysnc")); + new LinkedBlockingQueue<>(), new DTThreadFactory("hbase-async")); Config config = new Config(); config.overrideConfig(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_QUORUM, hbaseSideTableInfo.getHost()); config.overrideConfig(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_ZNODE_QUORUM, hbaseSideTableInfo.getParent()); - HbaseConfigUtils.loadKrb5Conf(hbaseConfig); hbaseConfig.entrySet().forEach(entity -> { config.overrideConfig(entity.getKey(), (String) entity.getValue()); }); - if (HbaseConfigUtils.asyncOpenKerberos(hbaseConfig)) { - String jaasStr = HbaseConfigUtils.buildJaasStr(hbaseConfig); - String jaasFilePath = HbaseConfigUtils.creatJassFile(jaasStr); - System.setProperty(HbaseConfigUtils.KEY_JAVA_SECURITY_AUTH_LOGIN_CONF, jaasFilePath); - config.overrideConfig(HbaseConfigUtils.KEY_JAVA_SECURITY_AUTH_LOGIN_CONF, jaasFilePath); + if (HbaseConfigUtils.isEnableKerberos(hbaseConfig)) { + HbaseConfigUtils.loadKrb5Conf(hbaseConfig); + String principal = MapUtils.getString(hbaseConfig, HbaseConfigUtils.KEY_PRINCIPAL); + HbaseConfigUtils.checkOpt(principal, HbaseConfigUtils.KEY_PRINCIPAL); + String regionserverPrincipal = MapUtils.getString(hbaseConfig, HbaseConfigUtils.KEY_HBASE_KERBEROS_REGIONSERVER_PRINCIPAL); + HbaseConfigUtils.checkOpt(regionserverPrincipal, HbaseConfigUtils.KEY_HBASE_KERBEROS_REGIONSERVER_PRINCIPAL); + String keytab = MapUtils.getString(hbaseConfig, HbaseConfigUtils.KEY_KEY_TAB); + HbaseConfigUtils.checkOpt(keytab, HbaseConfigUtils.KEY_KEY_TAB); + String keytabPath = System.getProperty("user.dir") + File.separator + keytab; + DtFileUtils.checkExists(keytabPath); + + LOG.info("Kerberos login with keytab: {} and principal: {}", keytab, principal); + String name = "HBaseClient"; + config.overrideConfig("hbase.sasl.clientconfig", name); + appendJaasConf(name, keytab, principal); refreshConfig(); } + hBaseClient = new HBaseClient(config, executorService); try { @@ -144,11 +166,21 @@ public void open(Configuration parameters) throws Exception { } } + private void appendJaasConf(String name, String keytab, String principal) { + javax.security.auth.login.Configuration priorConfig = javax.security.auth.login.Configuration.getConfiguration(); + // construct a dynamic JAAS configuration + DynamicConfiguration currentConfig = new DynamicConfiguration(priorConfig); + // wire up the configured JAAS login contexts to use the krb5 entries + AppConfigurationEntry krb5Entry = KerberosUtils.keytabEntry(keytab, principal); + currentConfig.addAppConfigurationEntry(name, krb5Entry); + javax.security.auth.login.Configuration.setConfiguration(currentConfig); + } + private void refreshConfig() throws KrbException { sun.security.krb5.Config.refresh(); KerberosName.resetDefaultRealm(); //reload java.security.auth.login.config - javax.security.auth.login.Configuration.setConfiguration(null); +// javax.security.auth.login.Configuration.setConfiguration(null); } @Override diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncSideInfo.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncSideInfo.java index 135aec004..6601458d4 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncSideInfo.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncSideInfo.java @@ -1,3 +1,21 @@ +/* + * 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.side.hbase; import com.dtstack.flink.sql.side.FieldInfo; 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 90ee289bd..6b5d6a24f 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 @@ -93,9 +93,6 @@ protected Row fillData(Row input, Object sideInput){ Row row = new Row(outFieldInfoList.size()); for(Map.Entry entry : inFieldIndex.entrySet()){ Object obj = input.getField(entry.getValue()); - if(obj instanceof Timestamp){ - obj = ((Timestamp)obj).getTime(); - } row.setField(entry.getKey(), obj); } diff --git a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java index 6acfcb760..1269b2844 100644 --- a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java +++ b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java @@ -101,10 +101,6 @@ public static String[] splitIgnoreQuotaBrackets(String str, String delimiter){ public ReplaceInfo getReplaceInfo(String field){ field = field.trim(); - if(field.length() <= 2){ - throw new RuntimeException(field + " \n" + - "Format defined exceptions"); - } //判断是不是常量==>''包裹的标识 if(field.startsWith("'") && field.endsWith("'")){ diff --git a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseConfigUtils.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseConfigUtils.java index c6399e27a..cad4fccc9 100644 --- a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseConfigUtils.java +++ b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseConfigUtils.java @@ -18,6 +18,9 @@ package com.dtstack.flink.sql.side.hbase.utils; +import com.dtstack.flink.sql.util.DtFileUtils; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; @@ -26,14 +29,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedWriter; import java.io.File; -import java.io.FileWriter; import java.io.IOException; import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.UUID; /** * @@ -58,9 +58,9 @@ public class HbaseConfigUtils { // async side kerberos private final static String KEY_HBASE_SECURITY_AUTH_ENABLE = "hbase.security.auth.enable"; private final static String KEY_HBASE_SASL_CLIENTCONFIG = "hbase.sasl.clientconfig"; - private final static String KEY_HBASE_KERBEROS_REGIONSERVER_PRINCIPAL = "hbase.kerberos.regionserver.principal"; + public final static String KEY_HBASE_KERBEROS_REGIONSERVER_PRINCIPAL = "hbase.kerberos.regionserver.principal"; public static final String KEY_KEY_TAB = "hbase.keytab"; - private static final String KEY_PRINCIPAL = "hbase.principal"; + public static final String KEY_PRINCIPAL = "hbase.principal"; public final static String KEY_HBASE_ZOOKEEPER_QUORUM = "hbase.zookeeper.quorum"; public final static String KEY_HBASE_ZOOKEEPER_ZNODE_QUORUM = "hbase.zookeeper.znode.parent"; @@ -101,23 +101,29 @@ public static Configuration getConfig(Map hbaseConfigMap) { return hConfiguration; } - public static boolean openKerberos(Map hbaseConfigMap) { - if (!MapUtils.getBooleanValue(hbaseConfigMap, KEY_HBASE_SECURITY_AUTHORIZATION)) { - return false; + public static boolean isEnableKerberos(Map hbaseConfigMap) { + boolean hasAuthorization = AUTHENTICATION_TYPE.equalsIgnoreCase( + MapUtils.getString(hbaseConfigMap, KEY_HBASE_SECURITY_AUTHORIZATION) + ); + boolean hasAuthentication = AUTHENTICATION_TYPE.equalsIgnoreCase( + MapUtils.getString(hbaseConfigMap, KEY_HBASE_SECURITY_AUTHENTICATION) + ); + boolean hasAuthEnable = MapUtils.getBooleanValue(hbaseConfigMap, KEY_HBASE_SECURITY_AUTH_ENABLE); + + if(hasAuthentication || hasAuthorization || hasAuthEnable) { + LOG.info("Enable kerberos for hbase."); + setKerberosConf(hbaseConfigMap); + return true; } - return AUTHENTICATION_TYPE.equalsIgnoreCase(MapUtils.getString(hbaseConfigMap, KEY_HBASE_SECURITY_AUTHENTICATION)); + return false; } - public static boolean asyncOpenKerberos(Map hbaseConfigMap) { - if (!MapUtils.getBooleanValue(hbaseConfigMap, KEY_HBASE_SECURITY_AUTH_ENABLE)) { - return false; - } - return AUTHENTICATION_TYPE.equalsIgnoreCase(MapUtils.getString(hbaseConfigMap, KEY_HBASE_SECURITY_AUTHENTICATION)); + private static void setKerberosConf(Map hbaseConfigMap) { + hbaseConfigMap.put(KEY_HBASE_SECURITY_AUTHORIZATION, AUTHENTICATION_TYPE); + hbaseConfigMap.put(KEY_HBASE_SECURITY_AUTHENTICATION, AUTHENTICATION_TYPE); + hbaseConfigMap.put(KEY_HBASE_SECURITY_AUTH_ENABLE, true); } - - - public static Configuration getHadoopConfiguration(Map hbaseConfigMap) { for (String key : KEYS_KERBEROS_REQUIRED) { if (StringUtils.isEmpty(MapUtils.getString(hbaseConfigMap, key))) { @@ -158,46 +164,20 @@ public static String getKeytab(Map hbaseConfigMap) { throw new IllegalArgumentException(""); } - public static void loadKrb5Conf(Map kerberosConfig) { - String krb5FilePath = System.getProperty("user.dir") + File.separator + MapUtils.getString(kerberosConfig, KEY_JAVA_SECURITY_KRB5_CONF); - if (!org.apache.commons.lang.StringUtils.isEmpty(krb5FilePath)) { - System.setProperty(KEY_JAVA_SECURITY_KRB5_CONF, krb5FilePath);; - } + public static void loadKrb5Conf(Map config) { + String krb5conf = MapUtils.getString(config, KEY_JAVA_SECURITY_KRB5_CONF); + checkOpt(krb5conf, KEY_JAVA_SECURITY_KRB5_CONF); + String krb5FilePath = System.getProperty("user.dir") + File.separator + MapUtils.getString(config, KEY_JAVA_SECURITY_KRB5_CONF); + DtFileUtils.checkExists(krb5FilePath); + System.setProperty(KEY_JAVA_SECURITY_KRB5_CONF, krb5FilePath); + LOG.info("{} is set to {}", KEY_JAVA_SECURITY_KRB5_CONF, krb5FilePath); } - public static String creatJassFile(String configStr) throws IOException { - String fileName = System.getProperty("user.dir"); - File krbConf = new File(fileName); - File temp = File.createTempFile("JAAS", ".conf", krbConf); - temp.deleteOnExit(); - BufferedWriter out = new BufferedWriter(new FileWriter(temp, false)); - out.write(configStr + "\n"); - out.close(); - return temp.getAbsolutePath(); + // TODO 日后改造可以下沉到Core模块 + public static void checkOpt(String opt, String key) { + Preconditions.checkState(!Strings.isNullOrEmpty(opt), "%s must be set!", key); } - public static String buildJaasStr(Map kerberosConfig) { - for (String key : ASYNC_KEYS_KERBEROS_REQUIRED) { - if (StringUtils.isEmpty(MapUtils.getString(kerberosConfig, key))) { - throw new IllegalArgumentException(String.format("Must provide [%s] when authentication is Kerberos", key)); - } - } - - String keyTab = System.getProperty("user.dir") + File.separator + MapUtils.getString(kerberosConfig, KEY_KEY_TAB); - String principal = MapUtils.getString(kerberosConfig, KEY_PRINCIPAL); - - StringBuilder jaasSB = new StringBuilder("Client {\n" + - " com.sun.security.auth.module.Krb5LoginModule required\n" + - " useKeyTab=true\n" + - " useTicketCache=false\n"); - jaasSB.append(" keyTab=\"").append(keyTab).append("\"").append("\n"); - jaasSB.append(" principal=\"").append(principal).append("\"").append(";\n"); - jaasSB.append("};"); - return jaasSB.toString(); - } - - - public static UserGroupInformation loginAndReturnUGI(Configuration conf, String principal, String keytab) throws IOException { if (conf == null) { throw new IllegalArgumentException("kerberos conf can not be null"); 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 904c75a1b..42a796403 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 @@ -27,7 +27,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; -import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.AuthUtil; +import org.apache.hadoop.hbase.ChoreService; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.ScheduledChore; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; @@ -39,9 +43,9 @@ import java.io.File; import java.io.IOException; import java.security.PrivilegedAction; +import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Set; /** * @author: jingzhen@dtstack.com @@ -185,14 +189,13 @@ private Put getPutByRow(Row record) { Put put = new Put(rowKey.getBytes()); for (int i = 0; i < record.getArity(); ++i) { Object fieldVal = record.getField(i); - byte[] val = null; if (fieldVal != null) { - val = HbaseUtil.toByte(fieldVal); - } - byte[] cf = families[i].getBytes(); - byte[] qualifier = qualifiers[i].getBytes(); + byte[] val = fieldVal.toString().getBytes(); + byte[] cf = families[i].getBytes(); + byte[] qualifier = qualifiers[i].getBytes(); - put.addColumn(cf, qualifier, val); + put.addColumn(cf, qualifier, val); + } } return put; } @@ -327,8 +330,8 @@ public HbaseOutputFormat finish() { String[] qualifiers = new String[format.columnNames.length]; if (format.columnNameFamily != null) { - Set keySet = format.columnNameFamily.keySet(); - String[] columns = keySet.toArray(new String[keySet.size()]); + List keyList = new LinkedList<>(format.columnNameFamily.keySet()); + String[] columns = keyList.toArray(new String[0]); for (int i = 0; i < columns.length; ++i) { String col = columns[i]; String[] part = col.split(":"); 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 09f5944b4..ea98c67c3 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 @@ -62,7 +62,7 @@ public class HbaseSink implements RetractStreamTableSink, IStreamSinkGener< private String clientPrincipal; private String clientKeytabFile; - private int parallelism = -1; + private int parallelism = 1; public HbaseSink() { @@ -115,7 +115,7 @@ public void emitDataStream(DataStream> dataStream) { HbaseOutputFormat outputFormat = builder.finish(); RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(outputFormat); - dataStream.addSink(richSinkFunction); + dataStream.addSink(richSinkFunction).setParallelism(parallelism).name(registerTabName); } @Override diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/RowKeyBuilder.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/RowKeyBuilder.java index 7af51be3d..13329faed 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/RowKeyBuilder.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/RowKeyBuilder.java @@ -110,10 +110,6 @@ public static String[] splitIgnoreQuotaBrackets(String str, String delimiter){ public ReplaceInfo getReplaceInfo(String field){ field = field.trim(); - if(field.length() <= 2){ - throw new RuntimeException(field + " \n" + - "Format defined exceptions"); - } //判断是不是常量==>''包裹的标识 if(field.startsWith("'") && field.endsWith("'")){ diff --git a/images/IMG_1573.JPG b/images/IMG_1573.JPG new file mode 100644 index 000000000..4a32461c1 Binary files /dev/null and b/images/IMG_1573.JPG differ diff --git a/impala/impala-side/impala-all-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAllReqRow.java b/impala/impala-side/impala-all-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAllReqRow.java index 10938308a..85b1f1b1f 100644 --- a/impala/impala-side/impala-all-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAllReqRow.java +++ b/impala/impala-side/impala-all-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAllReqRow.java @@ -18,21 +18,23 @@ package com.dtstack.flink.sql.side.impala; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; 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.impala.table.ImpalaSideTableInfo; import com.dtstack.flink.sql.side.rdb.all.AbstractRdbAllReqRow; import com.dtstack.flink.sql.util.JDBCUtils; +import com.dtstack.flink.sql.util.KrbUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.hadoop.conf.Configuration; 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.Connection; import java.sql.DriverManager; +import java.sql.SQLException; import java.util.List; /** @@ -61,10 +63,26 @@ public ImpalaAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List) () -> { + try { + return DriverManager.getConnection(url); + } catch (SQLException e) { + e.printStackTrace(); + } + return null; + }); + } else { + connection = DriverManager.getConnection(url); + } connection.setAutoCommit(false); return connection; } catch (Exception e) { @@ -78,14 +96,11 @@ public String getUrl() throws IOException { String newUrl = ""; Integer authMech = impalaSideTableInfo.getAuthMech(); - StringBuffer urlBuffer = new StringBuffer(impalaSideTableInfo.getUrl()); + StringBuilder urlBuffer = new StringBuilder(impalaSideTableInfo.getUrl()); if (authMech == 0) { newUrl = urlBuffer.toString(); } else if (authMech == 1) { - String keyTabFilePath = impalaSideTableInfo.getKeyTabFilePath(); - String krb5FilePath = impalaSideTableInfo.getKrb5FilePath(); - String principal = impalaSideTableInfo.getPrincipal(); String krbRealm = impalaSideTableInfo.getKrbRealm(); String krbHostFQDN = impalaSideTableInfo.getKrbHostFQDN(); String krbServiceName = impalaSideTableInfo.getKrbServiceName(); @@ -96,11 +111,7 @@ public String getUrl() throws IOException { .concat("KrbServiceName=").concat(krbServiceName).concat(";") ); newUrl = urlBuffer.toString(); - System.setProperty("java.security.krb5.conf", krb5FilePath); - Configuration configuration = new Configuration(); - configuration.set("hadoop.security.authentication" , "Kerberos"); - UserGroupInformation.setConfiguration(configuration); - UserGroupInformation.loginUserFromKeytab(principal, keyTabFilePath); + } else if (authMech == 2) { String uName = impalaSideTableInfo.getUserName(); diff --git a/impala/impala-side/impala-async-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAsyncReqRow.java b/impala/impala-side/impala-async-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAsyncReqRow.java index 3f4817580..3e7da110a 100644 --- a/impala/impala-side/impala-async-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAsyncReqRow.java +++ b/impala/impala-side/impala-async-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAsyncReqRow.java @@ -18,27 +18,31 @@ package com.dtstack.flink.sql.side.impala; -import com.dtstack.flink.sql.factory.DTThreadFactory; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; 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.impala.table.ImpalaSideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncReqRow; +import com.dtstack.flink.sql.util.KrbUtils; import io.vertx.core.Vertx; import io.vertx.core.VertxOptions; import io.vertx.core.json.JsonObject; import io.vertx.ext.jdbc.JDBCClient; +import io.vertx.ext.sql.SQLClient; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.runtime.types.CRow; import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; +import java.security.PrivilegedAction; import java.util.List; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; /** * Date: 2019/11/12 @@ -53,6 +57,7 @@ public class ImpalaAsyncReqRow extends RdbAsyncReqRow { private final static String IMPALA_DRIVER = "com.cloudera.impala.jdbc41.Driver"; + protected UserGroupInformation ugi = null; public ImpalaAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new ImpalaAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); @@ -60,9 +65,19 @@ public ImpalaAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List inputParams, + CRow input, + ResultFuture resultFuture, + SQLClient rdbSqlClient, + AtomicLong failCounter, + AtomicBoolean finishFlag, + CountDownLatch latch) { + if (ugi == null) { + doAsyncQueryData(inputParams, + input, resultFuture, + rdbSqlClient, + failCounter, + finishFlag, + latch); + } else { + // Kerberos + ugi.doAs((PrivilegedAction) () -> { + doAsyncQueryData(inputParams, + input, resultFuture, + rdbSqlClient, + failCounter, + finishFlag, + latch); + return null; + }); + } + } } 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 new file mode 100644 index 000000000..8723254bf --- /dev/null +++ b/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/ImpalaOutputFormat.java @@ -0,0 +1,756 @@ +/* + * 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.sink.impala; + +import com.dtstack.flink.sql.exception.ExceptionTrace; +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.DtStringUtil; +import com.dtstack.flink.sql.util.JDBCUtils; +import com.dtstack.flink.sql.util.KrbUtils; +import com.google.common.collect.Maps; +import org.apache.commons.collections.CollectionUtils; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +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; +import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static com.dtstack.flink.sql.sink.rdb.JDBCTypeConvertUtils.setRecordToStatement; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Date: 2020/10/14 + * Company: www.dtstack.com + * + * @author tiezhu + */ +public class ImpalaOutputFormat extends AbstractDtRichOutputFormat> { + + private static final Logger LOG = LoggerFactory.getLogger(ImpalaOutputFormat.class); + + private static final long serialVersionUID = 1L; + + // ${field} + private static final Pattern STATIC_PARTITION_PATTERN = Pattern.compile("\\$\\{([^}]*)}"); + //specific type which values need to be quoted + private static final String[] NEED_QUOTE_TYPE = {"string", "timestamp", "varchar"}; + + private static final Integer DEFAULT_CONN_TIME_OUT = 60; + private static final int RECEIVE_DATA_PRINT_FREQUENCY = 1000; + private static final int DIRTY_DATA_PRINT_FREQUENCY = 1000; + + private static final String KUDU_TYPE = "kudu"; + private static final String UPDATE_MODE = "update"; + private static final String PARTITION_CONSTANT = "PARTITION"; + private static final String DRIVER_NAME = "com.cloudera.impala.jdbc41.Driver"; + + private static final String VALUES_CONDITION = "${valuesCondition}"; + private static final String PARTITION_CONDITION = "${partitionCondition}"; + private static final String TABLE_FIELDS_CONDITION = "${tableFieldsCondition}"; + private static final String NO_PARTITION = "noPartition"; + + 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; + protected Integer authMech; + protected String dbUrl; + protected String userName; + protected String password; + protected int batchSize = 100; + protected long batchWaitInterval = 60 * 1000L; + protected String tableName; + protected List primaryKeys; + protected String partitionFields; + protected Boolean enablePartition; + protected String schema; + protected String storeType; + protected String updateMode; + public List fieldNames; + public List fieldTypes; + public List fieldExtraInfoList; + + // partition field of static partition which matched by ${field} + private final List staticPartitionFields = new ArrayList<>(); + + // valueFieldsName -> 重组之后的fieldNames,为了重组row data字段值对应 + // 需要对partition字段做特殊处理,比如原来的字段顺序为(age, name, id),但是因为partition,写入的SQL为 + // INSERT INTO tableName(name, id) PARTITION(age) VALUES(?, ?, ?) + // 那么实际executeSql设置字段的顺序应该为(name, id, age),同时,字段对应的type顺序也需要重组 + private List valueFieldNames; + private transient AbstractDtRichOutputFormat metricOutputFormat; + private List rows; + + private transient ScheduledExecutorService scheduler; + private transient ScheduledFuture scheduledFuture; + + @Override + public void configure(Configuration parameters) { + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + try { + rowDataMap = new HashMap<>(); + rows = new ArrayList<>(); + metricOutputFormat = this; + openConnect(); + initScheduledTask(batchWaitInterval); + init(); + initMetric(); + } catch (Exception e) { + throw new RuntimeException("impala output format open error!", e); + } + } + + private void init() throws SQLException { + if (Objects.nonNull(partitionFields)) { + // match ${field} from partitionFields + Matcher matcher = STATIC_PARTITION_PATTERN.matcher(partitionFields); + while (matcher.find()) { + LOG.info("find static partition field: {}", matcher.group(1)); + staticPartitionFields.add(matcher.group(1)); + } + } + + if (updateMode.equalsIgnoreCase(UPDATE_MODE)) { + if (!storeType.equalsIgnoreCase(KUDU_TYPE)) { + throw new IllegalArgumentException("update mode not support for non-kudu table!"); + } + updateStatement = connection.prepareStatement(buildUpdateSql(schema, tableName, fieldNames, primaryKeys)); + } else { + valueFieldNames = rebuildFieldNameListAndTypeList(fieldNames, staticPartitionFields, fieldTypes, partitionFields); + } + } + + private void initScheduledTask(Long batchWaitInterval) { + try { + if (batchWaitInterval != 0) { + this.scheduler = new ScheduledThreadPoolExecutor(1, + new DTThreadFactory("impala-upsert-output-format")); + this.scheduledFuture = this.scheduler.scheduleWithFixedDelay(() -> { + synchronized (ImpalaOutputFormat.this) { + flush(); + } + }, batchWaitInterval, batchWaitInterval, TimeUnit.MILLISECONDS); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private void openConnect() throws IOException { + if (authMech == 1) { + UserGroupInformation ugi = KrbUtils.loginAndReturnUgi(principal, keytabPath, krb5confPath); + try { + ugi.doAs((PrivilegedExceptionAction) () -> { + openJdbc(); + return null; + }); + } catch (InterruptedException | IOException e) { + throw new IllegalArgumentException("connect impala error!", e); + } + } else { + openJdbc(); + } + } + + /** + * get jdbc connection + */ + private void openJdbc() { + JDBCUtils.forName(DRIVER_NAME, getClass().getClassLoader()); + try { + connection = DriverManager.getConnection(dbUrl, userName, password); + statement = connection.createStatement(); + connection.setAutoCommit(false); + } catch (SQLException sqlException) { + throw new RuntimeException("get impala jdbc connection failed!", sqlException); + } + } + + private synchronized void flush() { + try { + if (batchCount > 0) { + if (updateMode.equalsIgnoreCase(UPDATE_MODE)) { + executeUpdateBatch(); + } + if (!rowDataMap.isEmpty()) { + String templateSql = + "INSERT INTO tableName ${tableFieldsCondition} PARTITION ${partitionCondition} VALUES ${valuesCondition}"; + executeBatchSql( + templateSql, + schema, + tableName, + storeType, + enablePartition, + valueFieldNames, + partitionFields, + rowDataMap + ); + rowDataMap.clear(); + } + } + batchCount = 0; + } catch (Exception e) { + LOG.error("Writing records to impala jdbc failed.", e); + throw new RuntimeException("Writing records to impala jdbc failed.", e); + } + } + + /** + * execute batch update statement + * + * @throws SQLException throw sql exception + */ + private void executeUpdateBatch() throws SQLException { + try { + rows.forEach(row -> { + try { + JDBCTypeConvertUtils.setRecordToStatement( + updateStatement, + JDBCTypeConvertUtils.getSqlTypeFromFieldType(fieldTypes), + row, + primaryKeys.stream().mapToInt(fieldNames::indexOf).toArray() + ); + updateStatement.addBatch(); + } catch (Exception e) { + throw new RuntimeException("impala jdbc execute batch error!", e); + } + }); + updateStatement.executeBatch(); + connection.commit(); + rows.clear(); + } catch (Exception e) { + LOG.debug("impala jdbc execute batch error ", e); + JDBCUtils.rollBack(connection); + JDBCUtils.commit(connection); + updateStatement.clearBatch(); + executeUpdate(connection); + } + } + + public void executeUpdate(Connection connection) { + rows.forEach(row -> { + try { + setRecordToStatement(updateStatement, JDBCTypeConvertUtils.getSqlTypeFromFieldType(fieldTypes), row); + updateStatement.executeUpdate(); + JDBCUtils.commit(connection); + } catch (Exception e) { + JDBCUtils.rollBack(connection); + JDBCUtils.commit(connection); + if (metricOutputFormat.outDirtyRecords.getCount() % DIRTY_DATA_PRINT_FREQUENCY == 0 || LOG.isDebugEnabled()) { + LOG.error("record insert failed ,this row is {}", row.toString()); + LOG.error("", e); + } + metricOutputFormat.outDirtyRecords.inc(); + } + }); + rows.clear(); + } + + private void putRowIntoMap(Map> rowDataMap, Tuple2 rowData) { + Set keySet = rowDataMap.keySet(); + ArrayList tempRowArray; + if (keySet.contains(rowData.f0)) { + tempRowArray = rowDataMap.get(rowData.f0); + } else { + tempRowArray = new ArrayList<>(); + } + tempRowArray.add(rowData.f1); + rowDataMap.put(rowData.f0, tempRowArray); + } + + private List rebuildFieldNameListAndTypeList(List fieldNames, List staticPartitionFields, List fieldTypes, String partitionFields) { + if (partitionFields == null || partitionFields.isEmpty()) { + return fieldNames; + } + + List valueFields = new ArrayList<>(fieldNames); + + for (int i = valueFields.size() - 1; i >= 0; i--) { + if (staticPartitionFields.contains(fieldNames.get(i))) { + valueFields.remove(i); + fieldTypes.remove(i); + } + } + + for (int i = 0; i < valueFields.size(); i++) { + if (partitionFields.contains(fieldNames.get(i))) { + valueFields.add(valueFields.remove(i)); + fieldTypes.add(fieldTypes.remove(i)); + } + } + + return valueFields; + } + + @Override + public void writeRecord(Tuple2 record) throws IOException { + try { + if (!record.f0) { + return; + } + + if (outRecords.getCount() % RECEIVE_DATA_PRINT_FREQUENCY == 0 || LOG.isDebugEnabled()) { + LOG.info("Receive data : {}", record); + } + + if (updateMode.equalsIgnoreCase(UPDATE_MODE)) { + rows.add(Row.copy(record.f1)); + } else { + Map valueMap = Maps.newHashMap(); + Row row = Row.copy(record.f1); + + for (int i = 0; i < row.getArity(); i++) { + valueMap.put(fieldNames.get(i), row.getField(i)); + } + + Tuple2 rowTuple2 = new Tuple2<>(); + if (storeType.equalsIgnoreCase(KUDU_TYPE) || !enablePartition) { + rowTuple2.f0 = NO_PARTITION; + } else { + rowTuple2.f0 = buildPartitionCondition(valueMap, partitionFields, staticPartitionFields); + } + + // 根据字段名对 row data 重组, 比如,原始 row data : (1, xxx, 20) -> (id, name, age) + // 但是由于 partition,写入的field 顺序变成了 (name, id, age),则需要对 row data 重组变成 (xxx, 1, 20) + Row rowValue = new Row(fieldTypes.size()); + for (int i = 0; i < fieldTypes.size(); i++) { + rowValue.setField(i, valueMap.get(valueFieldNames.get(i))); + } + rowTuple2.f1 = buildValuesCondition(fieldTypes, rowValue); + putRowIntoMap(rowDataMap, rowTuple2); + } + + batchCount++; + + if (batchCount >= batchSize) { + flush(); + } + + // Receive data + outRecords.inc(); + } catch (Exception e) { + throw new IOException("Writing records to impala failed.", e); + } + } + + @Override + public void close() throws IOException { + if (closed) { + return; + } + // 将还未执行的SQL flush + if (batchCount > 0) { + flush(); + } + // cancel scheduled task + if (this.scheduledFuture != null) { + scheduledFuture.cancel(false); + this.scheduler.shutdown(); + } + // close connection + try { + if (connection != null && connection.isValid(DEFAULT_CONN_TIME_OUT)) { + connection.close(); + } + + if (statement != null && !statement.isClosed()) { + statement.close(); + } + + if (updateStatement != null && !updateStatement.isClosed()) { + updateStatement.close(); + } + } catch (SQLException e) { + throw new RuntimeException("impala connection close failed!", e); + } finally { + connection = null; + statement = null; + updateStatement = null; + } + closed = true; + } + + /** + * execute batch sql from row data map + * sql like 'insert into tableName(f1, f2, f3) ${partitionCondition} values(v1, v2, v3), (v4, v5, v6).... + * + * @param tempSql template sql + * @param storeType the store type of data + * @param enablePartition enable partition or not + * @param fieldNames field name list + * @param partitionFields partition fields + * @param rowDataMap row data map + */ + private void executeBatchSql(String tempSql, + String schema, + String tableName, + String storeType, + Boolean enablePartition, + List fieldNames, + String partitionFields, + Map> rowDataMap) { + StringBuilder partitionCondition = new StringBuilder(); + String tableFieldsCondition = buildTableFieldsCondition(fieldNames, partitionFields); + ArrayList rowData = new ArrayList<>(); + String tableNameInfo = Objects.isNull(schema) ? + tableName : quoteIdentifier(schema) + "." + tableName; + tempSql = tempSql.replace("tableName", tableNameInfo); + boolean isPartitioned = storeType.equalsIgnoreCase(KUDU_TYPE) || !enablePartition; + + try { + // kudu ${partitionCondition} is null + if (isPartitioned) { + tempSql = tempSql + .replace(PARTITION_CONDITION, partitionCondition.toString()) + .replace(PARTITION_CONSTANT, "") + .replace(TABLE_FIELDS_CONDITION, tableFieldsCondition); + rowData.addAll(rowDataMap.get(NO_PARTITION)); + String executeSql = tempSql.replace(VALUES_CONDITION, String.join(", ", rowData)); + statement.execute(executeSql); + rowData.clear(); + } else { + // partition sql + Set keySet = rowDataMap.keySet(); + for (String key : keySet) { + rowData.addAll(rowDataMap.get(key)); + partitionCondition.append(key); + tempSql = tempSql + .replace(PARTITION_CONDITION, partitionCondition.toString()) + .replace(TABLE_FIELDS_CONDITION, tableFieldsCondition); + String executeSql = tempSql + .replace(VALUES_CONDITION, String.join(", ", rowData)); + statement.execute(executeSql); + partitionCondition.delete(0, partitionCondition.length()); + } + } + } catch (Exception e) { + if (e instanceof SQLException) { + dealBatchSqlError(rowData, connection, statement, tempSql); + } else { + throw new RuntimeException("Insert into impala error!", e); + } + } finally { + rowData.clear(); + } + } + + /** + * 当批量写入失败时,把批量的sql拆解为单条sql提交,对于单条写入的sql记做脏数据 + * + * @param rowData 批量的values + * @param connection 当前数据库connect + * @param statement 当前statement + * @param templateSql 模版sql,例如insert into tableName(f1, f2, f3) [partition] values $valueCondition + */ + private void dealBatchSqlError(List rowData, + Connection connection, + Statement statement, + String templateSql) { + String errorMsg = "Insert into impala error. \nCause: [%s]\nRow: [%s]"; + JDBCUtils.rollBack(connection); + JDBCUtils.commit(connection); + for (String rowDatum : rowData) { + String executeSql = templateSql.replace(VALUES_CONDITION, rowDatum); + try { + statement.execute(executeSql); + JDBCUtils.commit(connection); + } catch (SQLException e) { + JDBCUtils.rollBack(connection); + JDBCUtils.commit(connection); + if (metricOutputFormat.outDirtyRecords.getCount() % DIRTY_DATA_PRINT_FREQUENCY == 0 || LOG.isDebugEnabled()) { + LOG.error( + String.format( + errorMsg, + ExceptionTrace.traceOriginalCause(e), + rowDatum) + ); + } + metricOutputFormat.outDirtyRecords.inc(); + } + } + } + + /** + * build partition condition with row data + * + * @param rowData row data + * @param partitionFields partition fields + * @param staticPartitionField static partition fields + * @return condition like '(ptOne, ptTwo=v2)' + */ + private String buildPartitionCondition(Map rowData, String partitionFields, List staticPartitionField) { + for (String key : staticPartitionField) { + StringBuilder sb = new StringBuilder(); + Object value = rowData.get(key); + sb.append(key).append("=").append(value); + partitionFields = partitionFields.replace("${" + key + "}", sb.toString()); + } + return "(" + partitionFields + ")"; + } + + /** + * build field condition according to field names + * replace ${tableFieldCondition} + * + * @param fieldNames the selected field names + * @param partitionFields the partition fields + * @return condition like '(id, name, age)' + */ + private String buildTableFieldsCondition(List fieldNames, String partitionFields) { + return "(" + fieldNames.stream() + .filter(f -> !partitionFields.contains(f)) + .map(this::quoteIdentifier) + .collect(Collectors.joining(", ")) + ")"; + } + + /** + * according to field types, build the values condition + * replace ${valuesCondition} + * + * @param fieldTypes field types + * @return condition like '(?, ?, cast('?' as string))' and '?' will be replaced with row data + */ + private String buildValuesCondition(List fieldTypes, Row row) { + String valuesCondition = fieldTypes.stream().map( + f -> { + for (String item : NEED_QUOTE_TYPE) { + if (f.toLowerCase().contains(item)) { + return String.format("cast('?' as %s)", f.toLowerCase()); + } + } + return "?"; + }).collect(Collectors.joining(", ")); + for (int i = 0; i < row.getArity(); i++) { + Object rowField = row.getField(i); + if (DtStringUtil.isEmptyOrNull(rowField)) { + valuesCondition = valuesCondition.replaceFirst("'\\?'", "null"); + } else { + valuesCondition = valuesCondition.replaceFirst("\\?", Matcher.quoteReplacement(rowField.toString())); + } + } + return "(" + valuesCondition + ")"; + } + + /** + * impala update mode SQL + * + * @return UPDATE tableName SET setCondition WHERE whereCondition + */ + private String buildUpdateSql(String schema, String tableName, List fieldNames, List primaryKeys) { + //跳过primary key字段 + String setClause = fieldNames.stream() + .filter(f -> !CollectionUtils.isNotEmpty(primaryKeys) || !primaryKeys.contains(f)) + .map(f -> quoteIdentifier(f) + "=?") + .collect(Collectors.joining(", ")); + + String conditionClause = primaryKeys.stream() + .map(f -> quoteIdentifier(f) + "=?") + .collect(Collectors.joining(" AND ")); + + return "UPDATE " + (Objects.isNull(schema) ? "" : quoteIdentifier(schema) + ".") + + quoteIdentifier(tableName) + " SET " + setClause + " WHERE " + conditionClause; + } + + private String quoteIdentifier(String identifier) { + return "`" + identifier + "`"; + } + + public static Builder getImpalaBuilder() { + return new Builder(); + } + + public static class Builder { + private final ImpalaOutputFormat format = new ImpalaOutputFormat(); + + public Builder setDbUrl(String dbUrl) { + format.dbUrl = dbUrl; + return this; + } + + public Builder setUserName(String userName) { + format.userName = userName; + return this; + } + + public Builder setPassword(String password) { + format.password = password; + return this; + } + + public Builder setBatchSize(Integer batchSize) { + format.batchSize = batchSize; + return this; + } + + public Builder setBatchWaitInterval(Long batchWaitInterval) { + format.batchWaitInterval = batchWaitInterval; + return this; + } + + public Builder setTableName(String tableName) { + format.tableName = tableName; + return this; + } + + public Builder setPartitionFields(String partitionFields) { + format.partitionFields = Objects.isNull(partitionFields) ? + "" : partitionFields; + return this; + } + + public Builder setPrimaryKeys(List primaryKeys) { + format.primaryKeys = primaryKeys; + return this; + } + + public Builder setSchema(String schema) { + format.schema = schema; + return this; + } + + public Builder setEnablePartition(Boolean enablePartition) { + format.enablePartition = enablePartition; + return this; + } + + public Builder setUpdateMode(String updateMode) { + format.updateMode = updateMode; + return this; + } + + public Builder setFieldList(List fieldList) { + format.fieldNames = fieldList; + return this; + } + + public Builder setFieldTypeList(List fieldTypeList) { + format.fieldTypes = fieldTypeList; + return this; + } + + public Builder setStoreType(String storeType) { + format.storeType = storeType; + return this; + } + + public Builder setFieldExtraInfoList(List fieldExtraInfoList) { + format.fieldExtraInfoList = fieldExtraInfoList; + return this; + } + + public Builder setKeyTabPath(String keyTabPath) { + format.keytabPath = keyTabPath; + return this; + } + + public Builder setKrb5ConfPath(String krb5ConfPath) { + format.krb5confPath = krb5ConfPath; + return this; + } + + public Builder setPrincipal(String principal) { + format.principal = principal; + return this; + } + + public Builder setAuthMech(Integer authMech) { + format.authMech = authMech; + return this; + } + + private boolean canHandle(String url) { + return url.startsWith("jdbc:impala:"); + } + + public ImpalaOutputFormat build() { + if (!canHandle(format.dbUrl)) { + throw new IllegalArgumentException("impala dbUrl is illegal, check url: " + format.dbUrl); + } + + if (format.authMech == EAuthMech.Kerberos.getType()) { + checkNotNull(format.krb5confPath, + "When kerberos authentication is enabled, krb5confPath is required!"); + checkNotNull(format.principal, + "When kerberos authentication is enabled, principal is required!"); + checkNotNull(format.keytabPath, + "When kerberos authentication is enabled, keytabPath is required!"); + } + + if (format.authMech == EAuthMech.UserName.getType()) { + checkNotNull(format.userName, "userName is required!"); + } + + if (format.authMech == EAuthMech.NameANDPassword.getType()) { + checkNotNull(format.userName, "userName is required!"); + checkNotNull(format.password, "password is required!"); + } + + checkNotNull(format.storeType, "storeType is required!"); + + return format; + } + } +} diff --git a/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/ImpalaSink.java b/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/ImpalaSink.java index 34101d410..6d28f0173 100644 --- a/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/ImpalaSink.java +++ b/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/ImpalaSink.java @@ -20,64 +20,115 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.impala.table.ImpalaTableInfo; -import com.dtstack.flink.sql.sink.rdb.JDBCOptions; -import com.dtstack.flink.sql.sink.rdb.AbstractRdbSink; -import com.dtstack.flink.sql.sink.rdb.format.JDBCUpsertOutputFormat; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.table.AbstractTargetTableInfo; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.UserGroupInformation; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.table.sinks.RetractStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; -import java.io.IOException; +import java.util.List; +import java.util.Objects; /** - * Date: 2019/11/11 + * Date: 2020/10/14 * Company: www.dtstack.com * - * @author xiuzhu + * @author tiezhu */ +public class ImpalaSink implements RetractStreamTableSink, IStreamSinkGener { -public class ImpalaSink extends AbstractRdbSink implements IStreamSinkGener { + private static final String DEFAULT_STORE_TYPE = "kudu"; - private ImpalaTableInfo impalaTableInfo; + protected String[] fieldNames; + TypeInformation[] fieldTypes; + protected String dbUrl; + protected String userName; + protected String password; + protected Integer authMech; + + protected String keytabPath; + protected String krb5confPath; + protected String principal; + + protected int batchSize = 100; + protected long batchWaitInterval = 60 * 1000L; + protected String tableName; + protected String registerTabName; + protected String storeType; + + protected List primaryKeys; + private int parallelism = 1; + protected String schema; + protected String updateMode; + protected Boolean enablePartition; + public List fieldList; + public List fieldTypeList; + public List fieldExtraInfoList; + protected String partitionFields; public ImpalaSink() { - super(null); + // do Nothing } @Override - public JDBCUpsertOutputFormat getOutputFormat() { - JDBCOptions jdbcOptions = JDBCOptions.builder() - .setDbUrl(getImpalaJdbcUrl()) - .setDialect(new ImpalaDialect(getFieldTypes(), primaryKeys)) - .setUsername(userName) - .setPassword(password) - .setTableName(tableName) - .build(); + public ImpalaSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { + ImpalaTableInfo impalaTableInfo = (ImpalaTableInfo) targetTableInfo; + this.dbUrl = getImpalaJdbcUrl(impalaTableInfo); + this.password = impalaTableInfo.getPassword(); + this.userName = impalaTableInfo.getUserName(); + this.authMech = impalaTableInfo.getAuthMech(); - return JDBCUpsertOutputFormat.builder() - .setOptions(jdbcOptions) - .setFieldNames(fieldNames) - .setFlushMaxSize(batchNum) - .setFlushIntervalMills(batchWaitInterval) - .setFieldTypes(sqlTypes) - .setKeyFields(primaryKeys) - .setPartitionFields(impalaTableInfo.getPartitionFields()) - .setAllReplace(allReplace) - .setUpdateMode(updateMode) - .build(); - } + this.principal = impalaTableInfo.getPrincipal(); + this.keytabPath = impalaTableInfo.getKeyTabFilePath(); + this.krb5confPath = impalaTableInfo.getKrb5FilePath(); + + this.updateMode = Objects.isNull(impalaTableInfo.getUpdateMode()) ? + "append" : impalaTableInfo.getUpdateMode(); + + this.batchSize = Objects.isNull(impalaTableInfo.getBatchSize()) ? + batchSize : impalaTableInfo.getBatchSize(); + this.batchWaitInterval = Objects.isNull(impalaTableInfo.getBatchWaitInterval()) ? + batchWaitInterval : impalaTableInfo.getBatchWaitInterval(); + this.parallelism = Objects.isNull(impalaTableInfo.getParallelism()) ? + parallelism : impalaTableInfo.getParallelism(); + this.registerTabName = impalaTableInfo.getTableName(); + this.fieldList = impalaTableInfo.getFieldList(); + this.fieldTypeList = impalaTableInfo.getFieldTypeList(); + this.fieldExtraInfoList = impalaTableInfo.getFieldExtraInfoList(); + this.tableName = impalaTableInfo.getTableName(); + this.schema = impalaTableInfo.getSchema(); + this.primaryKeys = impalaTableInfo.getPrimaryKeys(); + this.partitionFields = impalaTableInfo.getPartitionFields(); - public String getImpalaJdbcUrl() { + this.storeType = Objects.isNull(impalaTableInfo.getStoreType()) ? + DEFAULT_STORE_TYPE : impalaTableInfo.getStoreType(); + this.enablePartition = impalaTableInfo.isEnablePartition(); + + return this; + } + + /** + * build Impala Jdbc Url according to authMech + * + * @param impalaTableInfo impala table info + * @return jdbc url with auth mech info + */ + public String getImpalaJdbcUrl(ImpalaTableInfo impalaTableInfo) { Integer authMech = impalaTableInfo.getAuthMech(); - String newUrl = dbUrl; - StringBuffer urlBuffer = new StringBuffer(dbUrl); + String newUrl = impalaTableInfo.getUrl(); + StringBuilder urlBuffer = new StringBuilder(impalaTableInfo.getUrl()); if (authMech == EAuthMech.NoAuthentication.getType()) { return newUrl; } else if (authMech == EAuthMech.Kerberos.getType()) { - String keyTabFilePath = impalaTableInfo.getKeyTabFilePath(); - String krb5FilePath = impalaTableInfo.getKrb5FilePath(); - String principal = impalaTableInfo.getPrincipal(); String krbRealm = impalaTableInfo.getKrbRealm(); String krbHostFqdn = impalaTableInfo.getKrbHostFQDN(); String krbServiceName = impalaTableInfo.getKrbServiceName(); @@ -88,21 +139,10 @@ public String getImpalaJdbcUrl() { .concat("KrbServiceName=").concat(krbServiceName).concat(";") ); newUrl = urlBuffer.toString(); - - System.setProperty("java.security.krb5.conf", krb5FilePath); - Configuration configuration = new Configuration(); - configuration.set("hadoop.security.authentication", "Kerberos"); - UserGroupInformation.setConfiguration(configuration); - try { - UserGroupInformation.loginUserFromKeytab(principal, keyTabFilePath); - } catch (IOException e) { - throw new RuntimeException("loginUserFromKeytab error ..", e); - } - } else if (authMech == EAuthMech.UserName.getType()) { urlBuffer.append(";" .concat("AuthMech=3;") - .concat("UID=").concat(userName).concat(";") + .concat("UID=").concat(impalaTableInfo.getUserName()).concat(";") .concat("PWD=;") .concat("UseSasl=0") ); @@ -110,8 +150,8 @@ public String getImpalaJdbcUrl() { } else if (authMech == EAuthMech.NameANDPassword.getType()) { urlBuffer.append(";" .concat("AuthMech=3;") - .concat("UID=").concat(userName).concat(";") - .concat("PWD=").concat(password) + .concat("UID=").concat(impalaTableInfo.getUserName()).concat(";") + .concat("PWD=").concat(impalaTableInfo.getPassword()) ); newUrl = urlBuffer.toString(); } else { @@ -120,11 +160,69 @@ public String getImpalaJdbcUrl() { return newUrl; } + private ImpalaOutputFormat buildImpalaOutputFormat() { + + return ImpalaOutputFormat.getImpalaBuilder() + .setDbUrl(dbUrl) + .setPassword(password) + .setUserName(userName) + .setSchema(schema) + .setTableName(tableName) + .setUpdateMode(updateMode) + .setBatchSize(batchSize) + .setBatchWaitInterval(batchWaitInterval) + .setPrimaryKeys(primaryKeys) + .setPartitionFields(partitionFields) + .setFieldList(fieldList) + .setFieldTypeList(fieldTypeList) + .setFieldExtraInfoList(fieldExtraInfoList) + .setStoreType(storeType) + .setEnablePartition(enablePartition) + .setUpdateMode(updateMode) + .setAuthMech(authMech) + .setKeyTabPath(keytabPath) + .setKrb5ConfPath(krb5confPath) + .setPrincipal(principal) + .build(); + } + + @Override + public void emitDataStream(DataStream> dataStream) { + consumeDataStream(dataStream); + } + + public void consumeDataStream(DataStream> dataStream) { + ImpalaOutputFormat outputFormat = buildImpalaOutputFormat(); + RichSinkFunction> richSinkFunction = new OutputFormatSinkFunction(outputFormat); + dataStream.addSink(richSinkFunction) + .setParallelism(parallelism) + .name(tableName); + } + @Override - public AbstractRdbSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { - super.genStreamSink(targetTableInfo); - this.impalaTableInfo = (ImpalaTableInfo) targetTableInfo; + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; return this; } + @Override + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo<>(org.apache.flink.table.api.Types.BOOLEAN(), getRecordType()); + } + + @Override + public TypeInformation getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } } 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 0d4857809..003b5a6ab 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 @@ -18,109 +18,128 @@ package com.dtstack.flink.sql.sink.impala.table; -import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; import com.dtstack.flink.sql.table.AbstractTableInfo; +import com.dtstack.flink.sql.table.AbstractTableParser; import com.dtstack.flink.sql.util.MathUtil; -import java.math.BigDecimal; -import java.sql.Timestamp; import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Objects; /** - * Reason: - * Date: 2019/11/11 + * Date: 2020/10/14 * Company: www.dtstack.com * - * @author xiuzhu + * @author tiezhu */ +public class ImpalaSinkParser extends AbstractTableParser { -public class ImpalaSinkParser extends RdbSinkParser { + private static final String PARALLELISM_KEY = "parallelism"; + + private static final String AUTH_MECH_KEY = "authMech"; + + private static final String KRB5FILEPATH_KEY = "krb5FilePath"; + + private static final String PRINCIPAL_KEY = "principal"; + + private static final String KEY_TAB_FILE_PATH_KEY = "keyTabFilePath"; + + private static final String KRB_REALM_KEY = "krbRealm"; + + private static final String KRB_HOST_FQDN_KEY = "krbHostFQDN"; + + private static final String KRB_SERVICE_NAME_KEY = "krbServiceName"; + + private static final String ENABLE_PARTITION_KEY = "enablePartition"; + + private static final String PARTITION_FIELDS_KEY = "partitionFields"; + + private static final String URL_KEY = "url"; + + private static final String TABLE_NAME_KEY = "tableName"; + + private static final String USER_NAME_KEY = "userName"; + + private static final String PASSWORD_KEY = "password"; + + private static final String BATCH_SIZE_KEY = "batchSize"; + + private static final String BATCH_WAIT_INTERVAL_KEY = "batchWaitInterval"; + + private static final String BUFFER_SIZE_KEY = "bufferSize"; + + private static final String FLUSH_INTERVAL_MS_KEY = "flushIntervalMs"; + + private static final String SCHEMA_KEY = "schema"; + + private static final String UPDATE_KEY = "updateMode"; + + private static final String KUDU_TYPE = "kudu"; + + private static final String DEFAULT_STORE_TYPE = "kudu"; + + private static final String STORE_TYPE_KEY = "storeType"; + + private static final String KRB_DEFAULT_REALM = "HADOOP.COM"; + + private static final String CURRENT_TYPE = "impala"; - private static final String CURR_TYPE = "impala"; @Override - public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) throws Exception { ImpalaTableInfo impalaTableInfo = new ImpalaTableInfo(); impalaTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, impalaTableInfo); - impalaTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(ImpalaTableInfo.PARALLELISM_KEY.toLowerCase()))); - impalaTableInfo.setUrl(MathUtil.getString(props.get(ImpalaTableInfo.URL_KEY.toLowerCase()))); - impalaTableInfo.setTableName(MathUtil.getString(props.get(ImpalaTableInfo.TABLE_NAME_KEY.toLowerCase()))); - impalaTableInfo.setBatchSize(MathUtil.getIntegerVal(props.get(ImpalaTableInfo.BATCH_SIZE_KEY.toLowerCase()))); - impalaTableInfo.setBatchWaitInterval(MathUtil.getLongVal(props.get(ImpalaTableInfo.BATCH_WAIT_INTERVAL_KEY.toLowerCase()))); - impalaTableInfo.setBufferSize(MathUtil.getString(props.get(ImpalaTableInfo.BUFFER_SIZE_KEY.toLowerCase()))); - impalaTableInfo.setFlushIntervalMs(MathUtil.getString(props.get(ImpalaTableInfo.FLUSH_INTERVALMS_KEY.toLowerCase()))); - impalaTableInfo.setSchema(MathUtil.getString(props.get(ImpalaTableInfo.SCHEMA_KEY.toLowerCase()))); - impalaTableInfo.setUpdateMode(MathUtil.getString(props.get(ImpalaTableInfo.UPDATE_KEY.toLowerCase()))); - - Integer authMech = MathUtil.getIntegerVal(props.get(ImpalaTableInfo.AUTHMECH_KEY.toLowerCase())); + impalaTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); + impalaTableInfo.setUrl(MathUtil.getString(props.get(URL_KEY.toLowerCase()))); + impalaTableInfo.setTableName(MathUtil.getString(props.get(TABLE_NAME_KEY.toLowerCase()))); + impalaTableInfo.setBatchSize(MathUtil.getIntegerVal(props.get(BATCH_SIZE_KEY.toLowerCase()))); + impalaTableInfo.setBatchWaitInterval(MathUtil.getLongVal(props.get(BATCH_WAIT_INTERVAL_KEY.toLowerCase()))); + impalaTableInfo.setBufferSize(MathUtil.getString(props.get(BUFFER_SIZE_KEY.toLowerCase()))); + impalaTableInfo.setFlushIntervalMs(MathUtil.getString(props.get(FLUSH_INTERVAL_MS_KEY.toLowerCase()))); + impalaTableInfo.setSchema(MathUtil.getString(props.get(SCHEMA_KEY.toLowerCase()))); + impalaTableInfo.setUpdateMode(MathUtil.getString(props.get(UPDATE_KEY.toLowerCase()))); + + Integer authMech = MathUtil.getIntegerVal(props.get(AUTH_MECH_KEY.toLowerCase())); authMech = authMech == null ? 0 : authMech; impalaTableInfo.setAuthMech(authMech); - List authMechs = Arrays.asList(new Integer[]{0, 1, 2, 3}); + List authMechs = Arrays.asList(0, 1, 2, 3); if (!authMechs.contains(authMech)) { throw new IllegalArgumentException("The value of authMech is illegal, Please select 0, 1, 2, 3"); } else if (authMech == 1) { - impalaTableInfo.setPrincipal(MathUtil.getString(props.get(ImpalaTableInfo.PRINCIPAL_KEY.toLowerCase()))); - impalaTableInfo.setKeyTabFilePath(MathUtil.getString(props.get(ImpalaTableInfo.KEYTABFILEPATH_KEY.toLowerCase()))); - impalaTableInfo.setKrb5FilePath(MathUtil.getString(props.get(ImpalaTableInfo.KRB5FILEPATH_KEY.toLowerCase()))); - String krbRealm = MathUtil.getString(props.get(ImpalaTableInfo.KRBREALM_KEY.toLowerCase())); - krbRealm = krbRealm == null ? "HADOOP.COM" : krbRealm; + impalaTableInfo.setPrincipal(MathUtil.getString(props.get(PRINCIPAL_KEY.toLowerCase()))); + impalaTableInfo.setKeyTabFilePath(MathUtil.getString(props.get(KEY_TAB_FILE_PATH_KEY.toLowerCase()))); + impalaTableInfo.setKrb5FilePath(MathUtil.getString(props.get(KRB5FILEPATH_KEY.toLowerCase()))); + String krbRealm = MathUtil.getString(props.get(KRB_REALM_KEY.toLowerCase())); + krbRealm = krbRealm == null ? KRB_DEFAULT_REALM : krbRealm; impalaTableInfo.setKrbRealm(krbRealm); - impalaTableInfo.setKrbHostFQDN(MathUtil.getString(props.get(ImpalaTableInfo.KRBHOSTFQDN_KEY.toLowerCase()))); - impalaTableInfo.setKrbServiceName(MathUtil.getString(props.get(ImpalaTableInfo.KRBSERVICENAME_KEY.toLowerCase()))); + impalaTableInfo.setKrbHostFQDN(MathUtil.getString(props.get(KRB_HOST_FQDN_KEY.toLowerCase()))); + impalaTableInfo.setKrbServiceName(MathUtil.getString(props.get(KRB_SERVICE_NAME_KEY.toLowerCase()))); } else if (authMech == 2) { - impalaTableInfo.setUserName(MathUtil.getString(props.get(ImpalaTableInfo.USER_NAME_KEY.toLowerCase()))); + impalaTableInfo.setUserName(MathUtil.getString(props.get(USER_NAME_KEY.toLowerCase()))); } else if (authMech == 3) { - impalaTableInfo.setUserName(MathUtil.getString(props.get(ImpalaTableInfo.USER_NAME_KEY.toLowerCase()))); - impalaTableInfo.setPassword(MathUtil.getString(props.get(ImpalaTableInfo.PASSWORD_KEY.toLowerCase()))); + impalaTableInfo.setUserName(MathUtil.getString(props.get(USER_NAME_KEY.toLowerCase()))); + impalaTableInfo.setPassword(MathUtil.getString(props.get(PASSWORD_KEY.toLowerCase()))); } - String enablePartitionStr = (String) props.get(ImpalaTableInfo.ENABLEPARITION_KEY.toLowerCase()); + String storeType = MathUtil.getString(props.get(STORE_TYPE_KEY.toLowerCase())); + impalaTableInfo.setStoreType(Objects.isNull(storeType) ? DEFAULT_STORE_TYPE : storeType); + + String enablePartitionStr = (String) props.get(ENABLE_PARTITION_KEY.toLowerCase()); boolean enablePartition = MathUtil.getBoolean(enablePartitionStr == null ? "false" : enablePartitionStr); impalaTableInfo.setEnablePartition(enablePartition); - if (enablePartition) { - String partitionFields = MathUtil.getString(props.get(ImpalaTableInfo.PARTITIONFIELDS_KEY.toLowerCase())); + + if (!impalaTableInfo.getStoreType().equalsIgnoreCase(KUDU_TYPE) && enablePartition) { + String partitionFields = MathUtil.getString(props.get(PARTITION_FIELDS_KEY.toLowerCase())); impalaTableInfo.setPartitionFields(partitionFields); } + impalaTableInfo.setType(CURRENT_TYPE); + impalaTableInfo.check(); return impalaTableInfo; } - - @Override - public Class dbTypeConvertToJavaType(String fieldType) { - switch (fieldType.toLowerCase()) { - case "boolean": - return Boolean.class; - case "char": - return Character.class; - case "double": - return Double.class; - case "float": - return Float.class; - case "tinyint": - return Byte.class; - case "smallint": - return Short.class; - case "int": - return Integer.class; - case "bigint": - return Long.class; - case "decimal": - return BigDecimal.class; - case "string": - case "varchar": - return String.class; - case "timestamp": - return Timestamp.class; - default: - break; - } - - throw new RuntimeException("不支持 " + fieldType + " 类型"); - } } diff --git a/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/table/ImpalaTableInfo.java b/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/table/ImpalaTableInfo.java index 032d9ea18..bc0dbac23 100644 --- a/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/table/ImpalaTableInfo.java +++ b/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/table/ImpalaTableInfo.java @@ -16,43 +16,46 @@ * limitations under the License. */ - package com.dtstack.flink.sql.sink.impala.table; -import com.dtstack.flink.sql.sink.rdb.table.RdbTableInfo; +import com.dtstack.flink.sql.enums.EUpdateMode; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import com.google.common.base.Preconditions; import org.apache.commons.lang3.StringUtils; +import java.util.Objects; + /** - * Date: 2019/11/13 + * Date: 2020/10/14 * Company: www.dtstack.com * - * @author xiuzhu + * @author tiezhu */ +public class ImpalaTableInfo extends AbstractTargetTableInfo { -public class ImpalaTableInfo extends RdbTableInfo { + private static final int MAX_BATCH_SIZE = 10000; - public static final String AUTHMECH_KEY = "authMech"; + private String url; - public static final String KRB5FILEPATH_KEY = "krb5FilePath"; + private String tableName; - public static final String PRINCIPAL_KEY = "principal"; + private String userName; - public static final String KEYTABFILEPATH_KEY = "keyTabFilePath"; + private String password; - public static final String KRBREALM_KEY = "krbRealm"; + private Integer batchSize; - public static final String KRBHOSTFQDN_KEY = "krbHostFQDN"; + private Long batchWaitInterval; - public static final String KRBSERVICENAME_KEY = "krbServiceName"; + private String bufferSize; - public static final String ENABLEPARITION_KEY = "enablePartition"; + private String flushIntervalMs; - public static final String PARTITIONFIELDS_KEY = "partitionFields"; + private String schema; - private static final String CURR_TYPE = "impala"; + private boolean allReplace; - private static final String PARTITION_FIELD_SPLIT_REGEX = ","; + private String updateMode; private Integer authMech; @@ -68,12 +71,90 @@ public class ImpalaTableInfo extends RdbTableInfo { private String krbServiceName; - private boolean enablePartition; + private boolean enablePartition = false; + + private String partitionFields; + + private String storeType; + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } + + public Integer getBatchSize() { + return batchSize; + } + + public void setBatchSize(Integer batchSize) { + this.batchSize = batchSize; + } + + public Long getBatchWaitInterval() { + return batchWaitInterval; + } + + public void setBatchWaitInterval(Long batchWaitInterval) { + this.batchWaitInterval = batchWaitInterval; + } + + public String getBufferSize() { + return bufferSize; + } + + public void setBufferSize(String bufferSize) { + this.bufferSize = bufferSize; + } + + public String getFlushIntervalMs() { + return flushIntervalMs; + } + + public void setFlushIntervalMs(String flushIntervalMs) { + this.flushIntervalMs = flushIntervalMs; + } + + public String getSchema() { + return schema; + } + + public void setSchema(String schema) { + this.schema = schema; + } - private String[] partitionFields; + public String getUpdateMode() { + return updateMode; + } - public ImpalaTableInfo() { - setType(CURR_TYPE); + public void setUpdateMode(String updateMode) { + this.updateMode = updateMode; } public Integer getAuthMech() { @@ -140,38 +221,49 @@ public void setEnablePartition(boolean enablePartition) { this.enablePartition = enablePartition; } - public String[] getPartitionFields() { + public String getPartitionFields() { return partitionFields; } public void setPartitionFields(String partitionFields) { - this.partitionFields = StringUtils.split(partitionFields, PARTITION_FIELD_SPLIT_REGEX); + this.partitionFields = partitionFields; + } + + public String getStoreType() { + return storeType; + } + + public void setStoreType(String storeType) { + this.storeType = storeType; + } + + @Override + public String getType() { + return super.getType().toLowerCase(); } @Override public boolean check() { - Preconditions.checkNotNull(this.getUrl(), "impala field of url is required"); - Preconditions.checkNotNull(this.getTableName(), "impala field of tableName is required"); - Preconditions.checkNotNull(this.getAuthMech(), "impala field of authMech is required"); - Integer authMech = getAuthMech(); - - if (authMech == 1) { - Preconditions.checkNotNull(this.getKrb5FilePath(), "impala field of krb5FilePath is required"); - Preconditions.checkNotNull(this.getPrincipal(), "impala field of principal is required"); - Preconditions.checkNotNull(this.getKeyTabFilePath(), "impala field of keyTabFilePath is required"); - Preconditions.checkNotNull(this.getKrbHostFQDN(), "impala field of krbHostFQDN is required"); - Preconditions.checkNotNull(this.getKrbServiceName(), "impala field of krbServiceName is required"); - } else if (authMech == 2) { - Preconditions.checkNotNull(this.getUserName(), "impala field of userName is required"); - }else if (authMech == 3) { - Preconditions.checkNotNull(this.getUserName(), "impala field of userName is required"); - Preconditions.checkNotNull(this.getPassword(), "impala field of password is required"); + Preconditions.checkNotNull(url, "impala field of URL is required"); + Preconditions.checkNotNull(tableName, "impala field of tableName is required"); + + if (Objects.nonNull(batchSize)) { + Preconditions.checkArgument(batchSize <= MAX_BATCH_SIZE, "batchSize must be less than " + MAX_BATCH_SIZE); } - if (isEnablePartition()) { - Preconditions.checkArgument(this.getPartitionFields().length > 0, "impala field of partitionFields is required"); + if (StringUtils.equalsIgnoreCase(updateMode, EUpdateMode.UPSERT.name())) { + Preconditions.checkArgument(Objects.nonNull(getPrimaryKeys()) && getPrimaryKeys().size() > 0, "updateMode mode primary is required"); } + if (Objects.nonNull(getPrimaryKeys())) { + getPrimaryKeys().forEach(pk -> { + Preconditions.checkArgument(getFieldList().contains(pk), "primary key " + pk + " not found in sink table field"); + }); + } + + + Preconditions.checkArgument(getFieldList().size() == getFieldExtraInfoList().size(), + "fields and fieldExtraInfoList attributes must be the same length"); return true; } } diff --git a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaSink.java b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaSink.java index 7234216a7..67c351274 100644 --- a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaSink.java +++ b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaSink.java @@ -18,8 +18,10 @@ package com.dtstack.flink.sql.sink.kafka; +import com.dtstack.flink.sql.enums.EUpdateMode; import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; +import com.dtstack.flink.sql.util.DataTypeUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; @@ -37,6 +39,8 @@ import org.apache.flink.util.Preconditions; import org.apache.kafka.clients.consumer.ConsumerConfig; +import java.util.Arrays; +import java.util.HashMap; import java.util.Optional; import java.util.Properties; import java.util.stream.IntStream; @@ -56,9 +60,10 @@ public abstract class AbstractKafkaSink implements RetractStreamTableSink, protected String[] partitionKeys; protected String sinkOperatorName; protected Properties properties; - protected int parallelism; + protected int parallelism = 1; protected String topic; protected String tableName; + protected String updateMode; protected TableSchema schema; protected SinkFunction kafkaProducer; @@ -75,11 +80,21 @@ protected Properties getKafkaProperties(KafkaSinkTableInfo KafkaSinkTableInfo) { } return props; } - + // TODO Source有相同的方法日后可以合并 protected TypeInformation[] getTypeInformations(KafkaSinkTableInfo kafka11SinkTableInfo) { + String[] fieldTypes = kafka11SinkTableInfo.getFieldTypes(); Class[] 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; } @@ -98,6 +113,9 @@ protected TableSchema buildTableSchema(String[] fieldNames, TypeInformation[] @Override public void emitDataStream(DataStream> dataStream) { + if (updateMode.equalsIgnoreCase(EUpdateMode.APPEND.name())) { + dataStream = dataStream.filter((Tuple2 record) -> record.f0); + } DataStream mapDataStream = dataStream .map((Tuple2 record) -> new CRow(record.f1, record.f0)) .returns(getRowTypeInfo()) @@ -111,8 +129,13 @@ public CRowTypeInfo getRowTypeInfo() { } 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-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 4ad8947a8..58bf8bc78 100644 --- a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -57,7 +57,7 @@ public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map()); this.partitionKeys = getPartitionKeys(kafkaSinkTableInfo); diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index d22be3d59..402b6ed9b 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -38,6 +38,7 @@ public KafkaSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { Properties kafkaProperties = getKafkaProperties(kafka09SinkTableInfo); this.tableName = kafka09SinkTableInfo.getName(); + this.updateMode = kafka09SinkTableInfo.getUpdateMode(); this.topic = kafka09SinkTableInfo.getTopic(); this.partitioner = Optional.of(new CustomerFlinkPartition<>()); this.partitionKeys = getPartitionKeys(kafka09SinkTableInfo); diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index eea78e121..09a1bd781 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -40,6 +40,7 @@ public KafkaSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { Properties kafkaProperties = getKafkaProperties(kafka10SinkTableInfo); this.tableName = kafka10SinkTableInfo.getName(); + this.updateMode = kafka10SinkTableInfo.getUpdateMode(); this.topic = kafka10SinkTableInfo.getTopic(); this.partitioner = Optional.of(new CustomerFlinkPartition<>()); this.partitionKeys = getPartitionKeys(kafka10SinkTableInfo); diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index ea45280c7..eee1a4ce5 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -41,6 +41,7 @@ public KafkaSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { Properties kafkaProperties = getKafkaProperties(kafka11SinkTableInfo); this.tableName = kafka11SinkTableInfo.getName(); + this.updateMode = kafka11SinkTableInfo.getUpdateMode(); this.topic = kafka11SinkTableInfo.getTopic(); this.partitioner = Optional.of(new CustomerFlinkPartition<>()); this.partitionKeys = getPartitionKeys(kafka11SinkTableInfo); 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 cf7f0dea7..01ee1cd9b 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 @@ -7,6 +7,7 @@ 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; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -18,6 +19,7 @@ import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.apache.flink.util.Collector; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.kudu.ColumnSchema; import org.apache.kudu.Schema; import org.apache.kudu.client.KuduClient; @@ -31,6 +33,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.security.PrivilegedAction; import java.sql.SQLException; import java.sql.Timestamp; import java.util.Arrays; @@ -218,24 +222,8 @@ private String buildKey(Map val, List equalFieldList) { private KuduScanner getConn(KuduSideTableInfo tableInfo) { try { if (client == null) { - String kuduMasters = tableInfo.getKuduMasters(); String tableName = tableInfo.getTableName(); - Integer workerCount = tableInfo.getWorkerCount(); - Integer defaultSocketReadTimeoutMs = tableInfo.getDefaultSocketReadTimeoutMs(); - Integer defaultOperationTimeoutMs = tableInfo.getDefaultOperationTimeoutMs(); - - Preconditions.checkNotNull(kuduMasters, "kuduMasters could not be null"); - - KuduClient.KuduClientBuilder kuduClientBuilder = new KuduClient.KuduClientBuilder(kuduMasters); - if (null != workerCount) { - kuduClientBuilder.workerCount(workerCount); - } - - if (null != defaultOperationTimeoutMs) { - kuduClientBuilder.defaultOperationTimeoutMs(defaultOperationTimeoutMs); - } - client = kuduClientBuilder.build(); - + client = getClient(tableInfo); if (!client.tableExists(tableName)) { throw new IllegalArgumentException("Table Open Failed , please check table exists"); } @@ -250,6 +238,36 @@ private KuduScanner getConn(KuduSideTableInfo tableInfo) { } } + private KuduClient getClient(KuduSideTableInfo tableInfo) throws IOException { + String kuduMasters = tableInfo.getKuduMasters(); + Integer workerCount = tableInfo.getWorkerCount(); + Integer defaultOperationTimeoutMs = tableInfo.getDefaultOperationTimeoutMs(); + + Preconditions.checkNotNull(kuduMasters, "kuduMasters could not be null"); + + KuduClient.KuduClientBuilder kuduClientBuilder = new KuduClient.KuduClientBuilder(kuduMasters); + + if (null != workerCount) { + kuduClientBuilder.workerCount(workerCount); + } + + if (null != defaultOperationTimeoutMs) { + kuduClientBuilder.defaultOperationTimeoutMs(defaultOperationTimeoutMs); + } + + if (tableInfo.isEnableKrb()) { + UserGroupInformation ugi = KrbUtils.loginAndReturnUgi(tableInfo.getPrincipal(), tableInfo.getKeytab(), tableInfo.getKrb5conf()); + return ugi.doAs(new PrivilegedAction() { + @Override + public KuduClient run() { + return kuduClientBuilder.build(); + } + }); + } else { + return kuduClientBuilder.build(); + } + } + /** * @param builder 创建AsyncKuduScanner对象 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 edb49814f..84b37ab67 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 @@ -10,11 +10,11 @@ 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; +import com.dtstack.flink.sql.util.KrbUtils; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.stumbleupon.async.Callback; import com.stumbleupon.async.Deferred; -import io.vertx.core.json.JsonArray; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; @@ -23,6 +23,7 @@ import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.kudu.ColumnSchema; import org.apache.kudu.Schema; import org.apache.kudu.client.AsyncKuduClient; @@ -35,6 +36,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.security.PrivilegedAction; import java.sql.Timestamp; import java.util.*; @@ -80,25 +83,10 @@ public void open(Configuration parameters) throws Exception { * * @throws KuduException */ - private void connKuDu() throws KuduException { + private void connKuDu() throws IOException { if (null == table) { - String kuduMasters = kuduSideTableInfo.getKuduMasters(); String tableName = kuduSideTableInfo.getTableName(); - Integer workerCount = kuduSideTableInfo.getWorkerCount(); - Integer defaultSocketReadTimeoutMs = kuduSideTableInfo.getDefaultSocketReadTimeoutMs(); - Integer defaultOperationTimeoutMs = kuduSideTableInfo.getDefaultOperationTimeoutMs(); - - Preconditions.checkNotNull(kuduMasters, "kuduMasters could not be null"); - - AsyncKuduClient.AsyncKuduClientBuilder asyncKuduClientBuilder = new AsyncKuduClient.AsyncKuduClientBuilder(kuduMasters); - if (null != workerCount) { - asyncKuduClientBuilder.workerCount(workerCount); - } - - if (null != defaultOperationTimeoutMs) { - asyncKuduClientBuilder.defaultOperationTimeoutMs(defaultOperationTimeoutMs); - } - asyncClient = asyncKuduClientBuilder.build(); + asyncClient = getClient(); if (!asyncClient.syncClient().tableExists(tableName)) { throw new IllegalArgumentException("Table Open Failed , please check table exists"); } @@ -128,6 +116,40 @@ private void connKuDu() throws KuduException { scannerBuilder.setProjectedColumnNames(projectColumns); } + private AsyncKuduClient getClient() throws IOException { + String kuduMasters = kuduSideTableInfo.getKuduMasters(); + Integer workerCount = kuduSideTableInfo.getWorkerCount(); + Integer defaultOperationTimeoutMs = kuduSideTableInfo.getDefaultOperationTimeoutMs(); + + Preconditions.checkNotNull(kuduMasters, "kuduMasters could not be null"); + + AsyncKuduClient.AsyncKuduClientBuilder asyncKuduClientBuilder = new AsyncKuduClient.AsyncKuduClientBuilder(kuduMasters); + if (null != workerCount) { + asyncKuduClientBuilder.workerCount(workerCount); + } + + if (null != defaultOperationTimeoutMs) { + asyncKuduClientBuilder.defaultOperationTimeoutMs(defaultOperationTimeoutMs); + } + + if (kuduSideTableInfo.isEnableKrb()) { + UserGroupInformation ugi = KrbUtils.loginAndReturnUgi( + kuduSideTableInfo.getPrincipal(), + kuduSideTableInfo.getKeytab(), + kuduSideTableInfo.getKrb5conf() + ); + return ugi.doAs( + new PrivilegedAction() { + @Override + public AsyncKuduClient run() { + return asyncKuduClientBuilder.build(); + } + }); + } else { + return asyncKuduClientBuilder.build(); + } + } + @Override public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { CRow inputCopy = new CRow(input.row(), input.change()); diff --git a/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideParser.java b/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideParser.java index 62d215d87..f350dd6c3 100644 --- a/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideParser.java +++ b/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideParser.java @@ -1,5 +1,6 @@ package com.dtstack.flink.sql.side.kudu.table; +import com.dtstack.flink.sql.constrant.PluginParamConsts; import com.dtstack.flink.sql.table.AbstractSideTableParser; import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; @@ -71,6 +72,18 @@ public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map { 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 AsyncKuduClient client; + private KuduClient client; private KuduTable table; + private volatile KuduSession session; + private Integer workerCount; private Integer defaultOperationTimeoutMs; private Integer defaultSocketReadTimeoutMs; + /** + * kerberos + */ + private String principal; + private String keytab; + private String krb5conf; private KuduOutputFormat() { } + public static KuduOutputFormatBuilder buildKuduOutputFormat() { + return new KuduOutputFormatBuilder(); + } + @Override public void configure(Configuration parameters) { @@ -92,25 +93,41 @@ public void open(int taskNumber, int numTasks) throws IOException { initMetric(); } - - private void establishConnection() throws KuduException { - AsyncKuduClient.AsyncKuduClientBuilder asyncKuduClientBuilder = new AsyncKuduClient.AsyncKuduClientBuilder(kuduMasters); + private void establishConnection() throws IOException { + KuduClient.KuduClientBuilder kuduClientBuilder = new KuduClient.KuduClientBuilder(kuduMasters); if (null != workerCount) { - asyncKuduClientBuilder.workerCount(workerCount); + kuduClientBuilder.workerCount(workerCount); } if (null != defaultSocketReadTimeoutMs) { - asyncKuduClientBuilder.workerCount(defaultSocketReadTimeoutMs); + kuduClientBuilder.defaultSocketReadTimeoutMs(defaultSocketReadTimeoutMs); } if (null != defaultOperationTimeoutMs) { - asyncKuduClientBuilder.workerCount(defaultOperationTimeoutMs); + kuduClientBuilder.defaultOperationTimeoutMs(defaultOperationTimeoutMs); + } + + if (enableKrb) { + UserGroupInformation ugi = KrbUtils.loginAndReturnUgi( + principal, + keytab, + krb5conf + ); + client = ugi.doAs( + (PrivilegedAction) kuduClientBuilder::build); + } else { + client = kuduClientBuilder.build(); } - client = asyncKuduClientBuilder.build(); - KuduClient syncClient = client.syncClient(); - if (syncClient.tableExists(tableName)) { - table = syncClient.openTable(tableName); + 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 @@ -122,26 +139,22 @@ public void writeRecord(Tuple2 record) throws IOException { } Row row = tupleTrans.getField(1); if (row.getArity() != fieldNames.length) { - if(outDirtyRecords.getCount() % DIRTY_PRINT_FREQUENCY == 0) { + if (outDirtyRecords.getCount() % DIRTY_PRINT_FREQUENCY == 0) { LOG.error("record insert failed ..{}", row.toString()); LOG.error("cause by row.getArity() != fieldNames.length"); } outDirtyRecords.inc(); return; } - Operation operation = toOperation(writeMode, row); - AsyncKuduSession session = client.newSession(); try { if (outRecords.getCount() % ROW_PRINT_FREQUENCY == 0) { LOG.info("Receive data : {}", row); } - - session.apply(operation); - session.close(); + session.apply(toOperation(writeMode, row)); outRecords.inc(); } catch (KuduException e) { - if(outDirtyRecords.getCount() % DIRTY_PRINT_FREQUENCY == 0){ + if (outDirtyRecords.getCount() % DIRTY_PRINT_FREQUENCY == 0) { LOG.error("record insert failed, total dirty record:{} current row:{}", outDirtyRecords.getCount(), row.toString()); LOG.error("", e); } @@ -151,88 +164,24 @@ public void writeRecord(Tuple2 record) throws IOException { @Override public void close() { - if (null != client) { + if (Objects.nonNull(session) && !session.isClosed()) { try { - client.close(); + session.close(); } catch (Exception e) { - throw new IllegalArgumentException("[closeKudu]:" + e.getMessage()); - } - } - } - - public static KuduOutputFormatBuilder buildKuduOutputFormat() { - return new KuduOutputFormatBuilder(); - } - - public static class KuduOutputFormatBuilder { - private final KuduOutputFormat kuduOutputFormat; - - protected KuduOutputFormatBuilder() { - this.kuduOutputFormat = new KuduOutputFormat(); - } - - public KuduOutputFormatBuilder setKuduMasters(String kuduMasters) { - kuduOutputFormat.kuduMasters = kuduMasters; - return this; - } - - public KuduOutputFormatBuilder setTableName(String tableName) { - kuduOutputFormat.tableName = tableName; - return this; - } - - - public KuduOutputFormatBuilder setFieldNames(String[] fieldNames) { - kuduOutputFormat.fieldNames = fieldNames; - return this; - } - - public KuduOutputFormatBuilder setFieldTypes(TypeInformation[] fieldTypes) { - kuduOutputFormat.fieldTypes = fieldTypes; - return this; - } - - public KuduOutputFormatBuilder setWriteMode(WriteMode writeMode) { - if (null == writeMode) { - kuduOutputFormat.writeMode = WriteMode.UPSERT; + throw new IllegalArgumentException("[closeKuduSession]: " + e.getMessage()); } - kuduOutputFormat.writeMode = writeMode; - return this; - } - - public KuduOutputFormatBuilder setWorkerCount(Integer workerCount) { - kuduOutputFormat.workerCount = workerCount; - return this; } - public KuduOutputFormatBuilder setDefaultOperationTimeoutMs(Integer defaultOperationTimeoutMs) { - kuduOutputFormat.defaultOperationTimeoutMs = defaultOperationTimeoutMs; - return this; - } - - public KuduOutputFormatBuilder setDefaultSocketReadTimeoutMs(Integer defaultSocketReadTimeoutMs) { - kuduOutputFormat.defaultSocketReadTimeoutMs = defaultSocketReadTimeoutMs; - return this; - } - - - public KuduOutputFormat finish() { - if (kuduOutputFormat.kuduMasters == null) { - throw new IllegalArgumentException("No kuduMasters supplied."); - } - - if (kuduOutputFormat.tableName == null) { - throw new IllegalArgumentException("No tablename supplied."); + if (null != client) { + try { + client.shutdown(); + } catch (Exception e) { + throw new IllegalArgumentException("[closeKuduClient]:" + e.getMessage()); } - - return kuduOutputFormat; } } 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(); @@ -311,11 +260,102 @@ private Operation toOperation(WriteMode writeMode) { return table.newInsert(); case UPDATE: return table.newUpdate(); - case UPSERT: - return table.newUpsert(); default: return table.newUpsert(); } } + public enum WriteMode { + // insert + INSERT, + // update + UPDATE, + // update or insert + UPSERT + } + + public static class KuduOutputFormatBuilder { + private final KuduOutputFormat kuduOutputFormat; + + protected KuduOutputFormatBuilder() { + this.kuduOutputFormat = new KuduOutputFormat(); + } + + public KuduOutputFormatBuilder setKuduMasters(String kuduMasters) { + kuduOutputFormat.kuduMasters = kuduMasters; + return this; + } + + public KuduOutputFormatBuilder setTableName(String tableName) { + kuduOutputFormat.tableName = tableName; + return this; + } + + + public KuduOutputFormatBuilder setFieldNames(String[] fieldNames) { + kuduOutputFormat.fieldNames = fieldNames; + return this; + } + + public KuduOutputFormatBuilder setFieldTypes(TypeInformation[] fieldTypes) { + kuduOutputFormat.fieldTypes = fieldTypes; + return this; + } + + public KuduOutputFormatBuilder setWriteMode(WriteMode writeMode) { + if (null == writeMode) { + kuduOutputFormat.writeMode = WriteMode.UPSERT; + } + kuduOutputFormat.writeMode = writeMode; + return this; + } + + public KuduOutputFormatBuilder setWorkerCount(Integer workerCount) { + kuduOutputFormat.workerCount = workerCount; + return this; + } + + public KuduOutputFormatBuilder setDefaultOperationTimeoutMs(Integer defaultOperationTimeoutMs) { + kuduOutputFormat.defaultOperationTimeoutMs = defaultOperationTimeoutMs; + return this; + } + + public KuduOutputFormatBuilder setDefaultSocketReadTimeoutMs(Integer defaultSocketReadTimeoutMs) { + kuduOutputFormat.defaultSocketReadTimeoutMs = defaultSocketReadTimeoutMs; + return this; + } + + public KuduOutputFormatBuilder setPrincipal(String principal) { + kuduOutputFormat.principal = principal; + return this; + } + + public KuduOutputFormatBuilder setKeytab(String keytab) { + kuduOutputFormat.keytab = keytab; + return this; + } + + public KuduOutputFormatBuilder setKrb5conf(String krb5conf) { + kuduOutputFormat.krb5conf = krb5conf; + return this; + } + + public KuduOutputFormatBuilder setEnableKrb(boolean enableKrb) { + kuduOutputFormat.enableKrb = enableKrb; + return this; + } + + public KuduOutputFormat finish() { + if (kuduOutputFormat.kuduMasters == null) { + throw new IllegalArgumentException("No kuduMasters supplied."); + } + + if (kuduOutputFormat.tableName == null) { + throw new IllegalArgumentException("No tablename supplied."); + } + + return kuduOutputFormat; + } + } + } \ 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 4c7c3bea8..68c092445 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 @@ -20,23 +20,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; @Override public KuduSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { @@ -47,28 +43,41 @@ public KuduSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { this.defaultSocketReadTimeoutMs = kuduTableInfo.getDefaultSocketReadTimeoutMs(); this.workerCount = kuduTableInfo.getWorkerCount(); this.writeMode = kuduTableInfo.getWriteMode(); + this.principal = kuduTableInfo.getPrincipal(); + this.keytab = kuduTableInfo.getKeytab(); + this.krb5conf = kuduTableInfo.getKrb5conf(); + this.enableKrb = kuduTableInfo.isEnableKrb(); return this; } @Override public void emitDataStream(DataStream> dataStream) { + consumeDataStream(dataStream); + } + + 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) .setDefaultOperationTimeoutMs(this.defaultOperationTimeoutMs) .setDefaultSocketReadTimeoutMs(this.defaultSocketReadTimeoutMs) .setFieldNames(this.fieldNames) - .setFieldTypes(this.fieldTypes); - KuduOutputFormat kuduOutputFormat = builder.finish(); + .setFieldTypes(this.fieldTypes) + .setPrincipal(this.principal) + .setKeytab(this.keytab) + .setKrb5conf(this.krb5conf) + .setEnableKrb(this.enableKrb) + .finish(); RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(kuduOutputFormat); DataStreamSink dataStreamSink = dataStream.addSink(richSinkFunction); dataStreamSink.name(tableName); if (parallelism > 0) { dataStreamSink.setParallelism(parallelism); } + return dataStreamSink; } @Override @@ -98,6 +107,4 @@ public String[] getFieldNames() { public TypeInformation[] getFieldTypes() { return fieldTypes; } - - } diff --git a/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduSinkParser.java b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduSinkParser.java index 20302d44f..80e9cc628 100644 --- a/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduSinkParser.java +++ b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduSinkParser.java @@ -1,5 +1,6 @@ package com.dtstack.flink.sql.sink.kudu.table; +import com.dtstack.flink.sql.constrant.PluginParamConsts; import com.dtstack.flink.sql.sink.kudu.KuduOutputFormat; import com.dtstack.flink.sql.table.AbstractTableParser; import com.dtstack.flink.sql.table.AbstractTableInfo; @@ -39,6 +40,18 @@ public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map, IStreamSinkGener< protected String userName; protected String password; protected String database; + protected Integer parallelism = 1; + protected String registerTableName; public MongoSink() { // TO DO NOTHING @@ -61,6 +65,9 @@ public MongoSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { this.userName = mongoTableInfo.getUserName(); this.password = mongoTableInfo.getPassword(); this.database = mongoTableInfo.getDatabase(); + this.parallelism = Objects.isNull(mongoTableInfo.getParallelism()) ? + parallelism : mongoTableInfo.getParallelism(); + this.registerTableName = mongoTableInfo.getName(); return this; } @@ -77,7 +84,9 @@ public void emitDataStream(DataStream> dataStream) { MongoOutputFormat outputFormat = builder.finish(); RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(outputFormat); - dataStream.addSink(richSinkFunction); + dataStream.addSink(richSinkFunction) + .setParallelism(parallelism) + .name(registerTableName); } @Override diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleDialect.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleDialect.java index 3a320658c..590ecc7a1 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleDialect.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleDialect.java @@ -104,7 +104,7 @@ private String buildConnectionByAllReplace(boolean allReplace, String col) { private String buildConnectionConditions(String[] uniqueKeyFields) { - return Arrays.stream(uniqueKeyFields).map(col -> "T1." + quoteIdentifier(col) + "=T2." + quoteIdentifier(col)).collect(Collectors.joining(",")); + return Arrays.stream(uniqueKeyFields).map(col -> "T1." + quoteIdentifier(col) + "=T2." + quoteIdentifier(col)).collect(Collectors.joining(" AND ")); } /** diff --git a/pom.xml b/pom.xml index 374ec3c31..1b797690d 100644 --- a/pom.xml +++ b/pom.xml @@ -39,11 +39,56 @@ + + UTF-8 + 1.8.3 + 2.7.3 + 4.12 + 2.21.0 + 2.0.4 + 0.7.8 + - - UTF-8 - 1.8.1 - + + + + junit + junit + ${junit.version} + test + + + org.mockito + mockito-core + ${mockito.version} + test + + + org.powermock + powermock-module-junit4 + ${powermock.version} + test + + + org.powermock + powermock-api-mockito2 + ${powermock.version} + test + + + org.mockito + mockito-core + + + + + org.jacoco + org.jacoco.agent + runtime + test + ${jacoco.version} + + @@ -92,6 +137,21 @@ + + org.apache.maven.plugins + maven-clean-plugin + 3.1.0 + + + + ${basedir}/plugins + + + ${basedir}/sqlplugins + + + + 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 0530e57e2..853060755 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,6 +18,7 @@ package com.dtstack.flink.sql.side.rdb.all; +import com.dtstack.flink.sql.util.JDBCUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.runtime.types.CRow; @@ -167,42 +168,32 @@ protected Object dealTimeAttributeType(Class entry, O boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(entry); if (obj instanceof Timestamp && 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; } private void loadData(Map>> tmpCache) throws SQLException { - RdbSideTableInfo tableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); - Connection connection = null; + queryAndFillData(tmpCache, getConnectionWithRetry((RdbSideTableInfo) sideInfo.getSideTableInfo())); + } - try { - for (int i = 0; i < CONN_RETRY_NUM; i++) { + private Connection getConnectionWithRetry(RdbSideTableInfo tableInfo) throws SQLException { + String connInfo = "url:" + tableInfo.getUrl() + "; userName:" + tableInfo.getUserName(); + String errorMsg = null; + for (int i = 0; i < CONN_RETRY_NUM; i++) { + try { + return getConn(tableInfo.getUrl(), tableInfo.getUserName(), tableInfo.getPassword()); + } catch (Exception e) { try { - connection = getConn(tableInfo.getUrl(), tableInfo.getUserName(), tableInfo.getPassword()); - break; - } catch (Exception e) { - if (i == CONN_RETRY_NUM - 1) { - throw new RuntimeException("", e); - } - try { - String connInfo = "url:" + tableInfo.getUrl() + ";userName:" + tableInfo.getUserName() + ",pwd:" + tableInfo.getPassword(); - LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + connInfo); - Thread.sleep(5 * 1000); - } catch (InterruptedException e1) { - LOG.error("", e1); - } + LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + connInfo); + errorMsg = e.getCause().toString(); + Thread.sleep(5 * 1000); + } catch (InterruptedException e1) { + LOG.error("", e1); } } - queryAndFillData(tmpCache, connection); - } catch (Exception e) { - LOG.error("", e); - throw new SQLException(e); - } finally { - if (connection != null) { - connection.close(); - } } + throw new SQLException("get conn fail. connInfo: " + connInfo + "\ncause by: " + errorMsg); } private void queryAndFillData(Map>> tmpCache, Connection connection) throws SQLException { @@ -213,24 +204,30 @@ private void queryAndFillData(Map>> tmpCache, C ResultSet resultSet = statement.executeQuery(sql); String[] sideFieldNames = StringUtils.split(sideInfo.getSideSelectFields(), ","); - String[] fields = sideInfo.getSideTableInfo().getFieldTypes(); + String[] sideFieldTypes = sideInfo.getSideTableInfo().getFieldTypes(); + String[] fields = sideInfo.getSideTableInfo().getFields(); + Map sideFieldNamesAndTypes = Maps.newHashMap(); + for (int i = 0; i < fields.length; i++) { + sideFieldNamesAndTypes.put(fields[i], sideFieldTypes[i]); + } + while (resultSet.next()) { Map oneRow = Maps.newHashMap(); for (String fieldName : sideFieldNames) { Object object = resultSet.getObject(fieldName.trim()); - int fieldIndex = sideInfo.getSideTableInfo().getFieldList().indexOf(fieldName.trim()); - object = SwitchUtil.getTarget(object, fields[fieldIndex]); + object = SwitchUtil.getTarget(object, sideFieldNamesAndTypes.get(fieldName)); oneRow.put(fieldName.trim(), object); } String cacheKey = sideInfo.getEqualFieldList().stream() .map(oneRow::get) - .map(Object::toString) + .map(String::valueOf) .collect(Collectors.joining("_")); tmpCache.computeIfAbsent(cacheKey, key -> Lists.newArrayList()) .add(oneRow); } + JDBCUtils.closeConnectionResource(resultSet, statement, connection, false); } public int getFetchSize() { diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java index 82ec95ca0..cafc715d5 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java @@ -36,6 +36,8 @@ import java.util.Arrays; import java.util.List; +import java.util.Map; +import java.util.Objects; import java.util.stream.Collectors; /** @@ -59,7 +61,18 @@ public RdbAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List selectFields = Lists.newArrayList(); + Map physicalFields = rdbSideTableInfo.getPhysicalFields(); + physicalFields.keySet().forEach( + item -> { + if (Objects.isNull(physicalFields.get(item))) { + selectFields.add(quoteIdentifier(item)); + } else { + selectFields.add(quoteIdentifier(physicalFields.get(item)) + " AS " + quoteIdentifier(item)); + } + } + ); + sqlCondition = getSelectFromStatement(getTableName(rdbSideTableInfo), selectFields, sideTableInfo.getPredicateInfoes()); LOG.info("--------dimension sql query-------\n{}" + sqlCondition); } @@ -68,7 +81,7 @@ public String getAdditionalWhereClause() { } private String getSelectFromStatement(String tableName, List selectFields, List predicateInfoes) { - String fromClause = selectFields.stream().map(this::quoteIdentifier).collect(Collectors.joining(", ")); + String fromClause = String.join(", ", selectFields); String predicateClause = predicateInfoes.stream().map(this::buildFilterCondition).collect(Collectors.joining(" AND ")); String whereClause = buildWhereClause(predicateClause); return "SELECT " + fromClause + " FROM " + tableName + whereClause; 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 ba37c4843..6693d7976 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 @@ -49,7 +49,10 @@ import java.util.List; import java.util.Map; import java.util.TimeZone; -import java.util.concurrent.*; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -74,7 +77,7 @@ public class RdbAsyncReqRow extends BaseAsyncReqRow { 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; @@ -112,7 +115,7 @@ protected void init(BaseSideInfo sideInfo) { } @Override - protected void preInvoke(CRow input, ResultFuture resultFuture){ + protected void preInvoke(CRow input, ResultFuture resultFuture) { } @@ -120,8 +123,8 @@ protected void preInvoke(CRow input, ResultFuture resultFuture){ public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { AtomicLong networkLogCounter = new AtomicLong(0L); - while (!connectionStatus.get()){//network is unhealth - if(networkLogCounter.getAndIncrement() % 1000 == 0){ + while (!connectionStatus.get()) {//network is unhealth + if (networkLogCounter.getAndIncrement() % 1000 == 0) { LOG.info("network unhealth to block task"); } Thread.sleep(100); @@ -133,53 +136,84 @@ public void handleAsyncInvoke(Map inputParams, CRow input, Resul private void connectWithRetry(Map inputParams, CRow input, ResultFuture resultFuture, SQLClient rdbSqlClient) { AtomicLong failCounter = new AtomicLong(0); AtomicBoolean finishFlag = new AtomicBoolean(false); - while(!finishFlag.get()){ - try{ + while (!finishFlag.get()) { + try { CountDownLatch latch = new CountDownLatch(1); - rdbSqlClient.getConnection(conn -> { - try { - if(conn.failed()){ - connectionStatus.set(false); - if(failCounter.getAndIncrement() % 1000 == 0){ - LOG.error("getConnection error", conn.cause()); - } - if(failCounter.get() >= sideInfo.getSideTableInfo().getConnectRetryMaxNum(100)){ - resultFuture.completeExceptionally(conn.cause()); - finishFlag.set(true); - } - return; - } - connectionStatus.set(true); - ScheduledFuture timerFuture = registerTimer(input, resultFuture); - cancelTimerWhenComplete(resultFuture, timerFuture); - handleQuery(conn.result(), inputParams, input, resultFuture); - finishFlag.set(true); - } catch (Exception e) { - dealFillDataError(input, resultFuture, e); - } finally { - latch.countDown(); - } - }); + asyncQueryData(inputParams, + input, resultFuture, + rdbSqlClient, + failCounter, + finishFlag, + latch); try { latch.await(); } catch (InterruptedException e) { LOG.error("", e); } - } catch (Exception e){ + } catch (Exception e) { //数据源队列溢出情况 connectionStatus.set(false); } - if(!finishFlag.get()){ + if (!finishFlag.get()) { try { Thread.sleep(3000); - } catch (Exception e){ + } catch (Exception e) { LOG.error("", e); } } } } + protected void asyncQueryData(Map inputParams, + CRow input, + ResultFuture resultFuture, + SQLClient rdbSqlClient, + AtomicLong failCounter, + AtomicBoolean finishFlag, + CountDownLatch latch) { + doAsyncQueryData(inputParams, + input, resultFuture, + rdbSqlClient, + failCounter, + finishFlag, + latch); + } + + final protected void doAsyncQueryData( + Map inputParams, + CRow input, + ResultFuture resultFuture, + SQLClient rdbSqlClient, + AtomicLong failCounter, + AtomicBoolean finishFlag, + CountDownLatch latch) { + rdbSqlClient.getConnection(conn -> { + try { + if (conn.failed()) { + connectionStatus.set(false); + if (failCounter.getAndIncrement() % 1000 == 0) { + LOG.error("getConnection error", conn.cause()); + } + if (failCounter.get() >= sideInfo.getSideTableInfo().getConnectRetryMaxNum(100)) { + resultFuture.completeExceptionally(conn.cause()); + finishFlag.set(true); + } + return; + } + connectionStatus.set(true); + preInvoke(input, resultFuture); + + handleQuery(conn.result(), inputParams, input, resultFuture); + finishFlag.set(true); + } catch (Exception e) { + dealFillDataError(input, resultFuture, e); + } finally { + latch.countDown(); + } + }); + } + private Object convertDataType(Object val) { if (val == null) { @@ -219,7 +253,7 @@ private Object convertDataType(Object val) { @Override public String buildCacheKey(Map inputParam) { - return StringUtils.join(inputParam.values(),"_"); + return StringUtils.join(inputParam.values(), "_"); } @Override @@ -231,7 +265,7 @@ public Row fillData(Row input, Object line) { boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { //去除上一层OutputRowtimeProcessFunction 调用时区导致的影响 - obj = ((Timestamp) obj).getTime() + (long)LOCAL_TZ.getOffset(((Timestamp) obj).getTime()); + obj = ((Timestamp) obj).getTime() + (long) LOCAL_TZ.getOffset(((Timestamp) obj).getTime()); } row.setField(entry.getKey(), obj); @@ -258,7 +292,7 @@ public void close() throws Exception { rdbSqlClient.close(); } - if(executor != null){ + if (executor != null) { executor.shutdown(); } @@ -268,7 +302,7 @@ public void setRdbSqlClient(SQLClient rdbSqlClient) { this.rdbSqlClient = rdbSqlClient; } - private void handleQuery(SQLConnection connection, Map inputParams, CRow input, ResultFuture resultFuture){ + private void handleQuery(SQLConnection connection, Map inputParams, CRow input, ResultFuture resultFuture) { String key = buildCacheKey(inputParams); JsonArray params = new JsonArray(Lists.newArrayList(inputParams.values())); connection.queryWithParams(sideInfo.getSqlCondition(), params, rs -> { @@ -312,9 +346,9 @@ private void handleQuery(SQLConnection connection, Map inputPara }); } - private Map formatInputParam(Map inputParam){ - Map result = Maps.newHashMap(); - inputParam.forEach((k,v) -> { + private Map formatInputParam(Map inputParam) { + Map result = Maps.newLinkedHashMap(); + inputParam.forEach((k, v) -> { result.put(k, convertDataType(v)); }); return result; diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java index 6bd9bece5..339e037ba 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java @@ -91,6 +91,7 @@ public void dealOneEqualCon(SqlNode sqlNode, String sideTableName) { SqlIdentifier left = (SqlIdentifier) ((SqlBasicCall) sqlNode).getOperands()[0]; SqlIdentifier right = (SqlIdentifier) ((SqlBasicCall) sqlNode).getOperands()[1]; + Map physicalFields = sideTableInfo.getPhysicalFields(); String leftTableName = left.getComponent(0).getSimple(); String leftField = left.getComponent(1).getSimple(); @@ -99,7 +100,7 @@ public void dealOneEqualCon(SqlNode sqlNode, String sideTableName) { String rightField = right.getComponent(1).getSimple(); if (leftTableName.equalsIgnoreCase(sideTableName)) { - equalFieldList.add(leftField); + equalFieldList.add(physicalFields.get(leftField)); int equalFieldIndex = -1; for (int i = 0; i < rowTypeInfo.getFieldNames().length; i++) { String fieldName = rowTypeInfo.getFieldNames()[i]; @@ -115,7 +116,7 @@ public void dealOneEqualCon(SqlNode sqlNode, String sideTableName) { } else if (rightTableName.equalsIgnoreCase(sideTableName)) { - equalFieldList.add(rightField); + equalFieldList.add(physicalFields.get(rightField)); int equalFieldIndex = -1; for (int i = 0; i < rowTypeInfo.getFieldNames().length; i++) { String fieldName = rowTypeInfo.getFieldNames()[i]; diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java index 49d163d4d..62390b47d 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java @@ -78,6 +78,8 @@ public static Object getTarget(Object obj, String targetType) { case "timestamp": case "datetime": return MathUtil.getTimestamp(obj); + case "time": + return MathUtil.getTime(obj); default: } return obj; diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/JDBCTypeConvertUtils.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/JDBCTypeConvertUtils.java index ded966c68..f06ecba74 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/JDBCTypeConvertUtils.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/JDBCTypeConvertUtils.java @@ -204,6 +204,8 @@ public static int[] buildSqlTypes(List fieldTypeArray) { tmpFieldsType[i] = Types.TINYINT; } else if (fieldType.equals(Short.class.getName())) { tmpFieldsType[i] = Types.SMALLINT; + } else if(fieldType.equals(Character.class.getName())){ + tmpFieldsType[i] = Types.CHAR; } else if (fieldType.equals(String.class.getName())) { tmpFieldsType[i] = Types.CHAR; } else if (fieldType.equals(Byte.class.getName())) { @@ -226,4 +228,59 @@ public static int[] buildSqlTypes(List fieldTypeArray) { return tmpFieldsType; } + /** + * according to Java type, get the corresponding SQL type + * + * @param fieldTypeList the Java type + * @return the type number of the corresponding type + */ + public static int[] getSqlTypeFromFieldType(List fieldTypeList) { + int[] tmpFieldsType = new int[fieldTypeList.size()]; + for (int i = 0; i < fieldTypeList.size(); i++) { + String fieldType = fieldTypeList.get(i).toUpperCase(); + switch (fieldType) { + case "INT": + tmpFieldsType[i] = Types.INTEGER; + break; + case "BOOLEAN": + tmpFieldsType[i] = Types.BOOLEAN; + break; + case "BIGINT": + tmpFieldsType[i] = Types.BIGINT; + break; + case "SHORT": + tmpFieldsType[i] = Types.SMALLINT; + break; + case "STRING": + case "CHAR": + tmpFieldsType[i] = Types.CHAR; + break; + case "BYTE": + tmpFieldsType[i] = Types.BINARY; + break; + case "FLOAT": + tmpFieldsType[i] = Types.FLOAT; + break; + case "DOUBLE": + tmpFieldsType[i] = Types.DOUBLE; + break; + case "TIMESTAMP": + tmpFieldsType[i] = Types.TIMESTAMP; + break; + case "BIGDECIMAL": + tmpFieldsType[i] = Types.DECIMAL; + break; + case "DATE": + tmpFieldsType[i] = Types.DATE; + break; + case "TIME": + tmpFieldsType[i] = Types.TIME; + break; + default: + throw new RuntimeException("no support field type for sql. the input type:" + fieldType); + } + } + return tmpFieldsType; + } + } 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 4e047492f..c2491e586 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 @@ -29,13 +29,13 @@ 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.sql.ResultSet; +import java.security.PrivilegedAction; import java.sql.SQLException; -import java.sql.Statement; import java.util.List; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; @@ -110,10 +110,13 @@ public JDBCUpsertOutputFormat( */ @Override public void open(int taskNumber, int numTasks) throws IOException { + openJdbc(); + } + + public void openJdbc() throws IOException { try { establishConnection(); initMetric(); - if (StringUtils.equalsIgnoreCase(updateMode, EUpdateMode.APPEND.name()) || keyFields == null || keyFields.length == 0) { String insertSql = dialect.getInsertIntoStatement(schema, tableName, fieldNames, partitionFields); LOG.info("execute insert sql: {}", insertSql); 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 42ed545a6..9328257a5 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 @@ -122,7 +122,9 @@ public void addRecord(Tuple2 record) throws SQLException { // we don't need perform a deep copy, because jdbc field are immutable object. Tuple2 tuple2 = objectReuse ? new Tuple2<>(record.f0, Row.copy(record.f1)) : record; // add records to buffer - keyToRows.put(getPrimaryKey(tuple2.f1), tuple2); + if (tuple2.f0) { + keyToRows.put(getPrimaryKey(tuple2.f1), tuple2); + } } @Override @@ -170,8 +172,16 @@ public void executeUpdate(Connection connection) throws SQLException { connection.commit(); } catch (Exception e) { // deal pg error: current transaction is aborted, commands ignored until end of transaction block - connection.rollback(); - connection.commit(); + try { + connection.rollback(); + connection.commit(); + } catch (SQLException e1) { + throw new RuntimeException(e1); + } + + if(e.getMessage().contains("doesn't exist")){ + throw new RuntimeException(e); + } if (metricOutputFormat.outDirtyRecords.getCount() % DIRTYDATA_PRINT_FREQUENTY == 0 || LOG.isDebugEnabled()) { LOG.error("record insert failed ,this row is {}", entry.getValue()); LOG.error("", e); diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/writer/AppendOnlyWriter.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/writer/AppendOnlyWriter.java index 7c3ff4b09..b1d601c22 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/writer/AppendOnlyWriter.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/writer/AppendOnlyWriter.java @@ -118,11 +118,18 @@ public void executeUpdate(Connection connection) { } catch (SQLException e1) { throw new RuntimeException(e1); } + + if(e.getMessage().contains("doesn't exist")){ + throw new RuntimeException(e); + + } if (metricOutputFormat.outDirtyRecords.getCount() % DIRTYDATA_PRINT_FREQUENTY == 0 || LOG.isDebugEnabled()) { LOG.error("record insert failed ,this row is {}", row.toString()); LOG.error("", e); } metricOutputFormat.outDirtyRecords.inc(); + + } }); rows.clear(); diff --git a/redis5/pom.xml b/redis5/pom.xml index 894f786ef..d8e63bef1 100644 --- a/redis5/pom.xml +++ b/redis5/pom.xml @@ -16,5 +16,18 @@ redis5-side + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + redis.clients + jedis + 2.9.0 + + \ No newline at end of file diff --git a/redis5/redis5-side/pom.xml b/redis5/redis5-side/pom.xml index c6623b9d2..10462574c 100644 --- a/redis5/redis5-side/pom.xml +++ b/redis5/redis5-side/pom.xml @@ -11,20 +11,6 @@ 4.0.0 sql.side.redis redis-side - - - com.dtstack.flink - sql.core - 1.0-SNAPSHOT - provided - - - redis.clients - jedis - 2.8.0 - - - redis-side-core 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 8e6a08791..2e3c0887b 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 @@ -26,6 +26,7 @@ import com.dtstack.flink.sql.side.redis.table.RedisSideReqRow; import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; import com.esotericsoftware.minlog.Log; +import com.google.common.collect.Maps; import org.apache.calcite.sql.JoinType; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.collections.MapUtils; @@ -35,7 +36,6 @@ import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.types.Row; import org.apache.flink.util.Collector; -import com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import redis.clients.jedis.HostAndPort; @@ -48,14 +48,18 @@ import java.io.Closeable; import java.io.IOException; import java.sql.SQLException; - +import java.util.Arrays; import java.util.Calendar; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.atomic.AtomicReference; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + /** * @author yanxi */ @@ -63,6 +67,8 @@ public class RedisAllReqRow extends BaseAllReqRow { private static final long serialVersionUID = 7578879189085344807L; + private static final Pattern HOST_PORT_PATTERN = Pattern.compile("(?(.*)):(?\\d+)*"); + private static final Logger LOG = LoggerFactory.getLogger(RedisAllReqRow.class); private static final int CONN_RETRY_NUM = 3; @@ -73,9 +79,9 @@ public class RedisAllReqRow extends BaseAllReqRow { private RedisSideTableInfo tableInfo; - private AtomicReference>> cacheRef = new AtomicReference<>(); + private final AtomicReference>> cacheRef = new AtomicReference<>(); - private RedisSideReqRow redisSideReqRow; + private final RedisSideReqRow redisSideReqRow; public RedisAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new RedisAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); @@ -115,8 +121,8 @@ public void flatMap(CRow input, Collector out) throws Exception { for (int i = 0; i < sideInfo.getEqualValIndex().size(); i++) { Integer conValIndex = sideInfo.getEqualValIndex().get(i); Object equalObj = input.row().getField(conValIndex); - if(equalObj == null){ - if(sideInfo.getJoinType() == JoinType.LEFT){ + if (equalObj == null) { + if (sideInfo.getJoinType() == JoinType.LEFT) { Row data = fillData(input.row(), null); out.collect(new CRow(data, input.change())); } @@ -125,13 +131,13 @@ public void flatMap(CRow input, Collector out) throws Exception { inputParams.put(sideInfo.getEqualFieldList().get(i), equalObj.toString()); } String key = buildCacheKey(inputParams); - if(StringUtils.isBlank(key)){ + if (StringUtils.isBlank(key)) { return; } Map cacheMap = cacheRef.get().get(key); - if(MapUtils.isEmpty(cacheMap)){ - if(sideInfo.getJoinType() != JoinType.LEFT){ + if (MapUtils.isEmpty(cacheMap)) { + if (sideInfo.getJoinType() != JoinType.LEFT) { return; } Row data = fillData(input.row(), null); @@ -146,8 +152,8 @@ public void flatMap(CRow input, Collector out) throws Exception { private String buildCacheKey(Map refData) { StringBuilder keyBuilder = new StringBuilder(tableInfo.getTableName()); List primaryKeys = tableInfo.getPrimaryKeys(); - for(String primaryKey : primaryKeys){ - if(!refData.containsKey(primaryKey)){ + for (String primaryKey : primaryKeys) { + if (!refData.containsKey(primaryKey)) { return null; } keyBuilder.append("_").append(refData.get(primaryKey)); @@ -160,7 +166,7 @@ private void loadData(Map> tmpCache) throws SQLExcep JedisCommands jedis = null; try { StringBuilder keyPattern = new StringBuilder(tableInfo.getTableName()); - for (String key : tableInfo.getPrimaryKeys()) { + for (int i = 0; i < tableInfo.getPrimaryKeys().size(); i++) { keyPattern.append("_").append("*"); } jedis = getJedisWithRetry(CONN_RETRY_NUM); @@ -195,41 +201,54 @@ private JedisCommands getJedis(RedisSideTableInfo tableInfo) { String url = tableInfo.getUrl(); String password = tableInfo.getPassword(); String database = tableInfo.getDatabase() == null ? "0" : tableInfo.getDatabase(); + String masterName = tableInfo.getMasterName(); int timeout = tableInfo.getTimeout(); - if (timeout == 0){ - timeout = 1000; + if (timeout == 0) { + timeout = 10000; } String[] nodes = url.split(","); - String[] firstIpPort = nodes[0].split(":"); - String firstIp = firstIpPort[0]; - String firstPort = firstIpPort[1]; - Set addresses = new HashSet<>(); - Set ipPorts = new HashSet<>(); - for (String ipPort : nodes) { - ipPorts.add(ipPort); - String[] ipPortPair = ipPort.split(":"); - addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.valueOf(ipPortPair[1].trim()))); - } - if (timeout == 0){ - timeout = 1000; - } JedisCommands jedis = null; GenericObjectPoolConfig poolConfig = setPoolConfig(tableInfo.getMaxTotal(), tableInfo.getMaxIdle(), tableInfo.getMinIdle()); - switch (RedisType.parse(tableInfo.getRedisType())){ - //单机 + switch (RedisType.parse(tableInfo.getRedisType())) { case STANDALONE: - pool = new JedisPool(poolConfig, firstIp, Integer.parseInt(firstPort), timeout, password, Integer.parseInt(database)); - jedis = pool.getResource(); + String firstIp = null; + String firstPort = null; + Matcher standalone = HOST_PORT_PATTERN.matcher(nodes[0]); + if (standalone.find()) { + firstIp = standalone.group("host").trim(); + firstPort = standalone.group("port").trim(); + } + if (Objects.nonNull(firstIp)) { + pool = new JedisPool(poolConfig, firstIp, Integer.parseInt(firstPort), timeout, password, Integer.parseInt(database)); + jedis = pool.getResource(); + } else { + throw new IllegalArgumentException( + String.format("redis url error. current url [%s]", nodes[0])); + } break; - //哨兵 case SENTINEL: - jedisSentinelPool = new JedisSentinelPool(tableInfo.getMasterName(), ipPorts, poolConfig, timeout, password, Integer.parseInt(database)); + Set ipPorts = new HashSet<>(Arrays.asList(nodes)); + jedisSentinelPool = new JedisSentinelPool(masterName, ipPorts, poolConfig, timeout, password, Integer.parseInt(database)); jedis = jedisSentinelPool.getResource(); break; - //集群 case CLUSTER: - jedis = new JedisCluster(addresses, timeout, timeout,1, poolConfig); + Set addresses = new HashSet<>(); + // 对ipv6 支持 + for (String node : nodes) { + Matcher matcher = HOST_PORT_PATTERN.matcher(node); + if (matcher.find()) { + String host = matcher.group("host").trim(); + String portStr = matcher.group("port").trim(); + if (org.apache.commons.lang3.StringUtils.isNotBlank(host) && org.apache.commons.lang3.StringUtils.isNotBlank(portStr)) { + // 转化为int格式的端口 + int port = Integer.parseInt(portStr); + addresses.add(new HostAndPort(host, port)); + } + } + } + jedis = new JedisCluster(addresses, timeout, timeout, 10, password, poolConfig); + break; default: break; } @@ -257,35 +276,32 @@ private JedisCommands getJedisWithRetry(int retryNum) { return null; } - private Set getRedisKeys(RedisType redisType, JedisCommands jedis, String keyPattern){ - if(!redisType.equals(RedisType.CLUSTER)){ + private Set getRedisKeys(RedisType redisType, JedisCommands jedis, String keyPattern) { + if (!redisType.equals(RedisType.CLUSTER)) { return ((Jedis) jedis).keys(keyPattern); } Set keys = new TreeSet<>(); - Map clusterNodes = ((JedisCluster)jedis).getClusterNodes(); - for(String k : clusterNodes.keySet()){ + Map clusterNodes = ((JedisCluster) jedis).getClusterNodes(); + for (String k : clusterNodes.keySet()) { JedisPool jp = clusterNodes.get(k); - Jedis connection = jp.getResource(); - try { + try (Jedis connection = jp.getResource()) { keys.addAll(connection.keys(keyPattern)); - } catch (Exception e){ - LOG.error("Getting keys error: {}", e); - } finally { - connection.close(); + } catch (Exception e) { + LOG.error("Getting keys error.", e); } } return keys; } - 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; diff --git a/redis5/redis5-side/redis-async-side/pom.xml b/redis5/redis5-side/redis-async-side/pom.xml index 39a6280cc..666bace2b 100644 --- a/redis5/redis5-side/redis-async-side/pom.xml +++ b/redis5/redis5-side/redis-async-side/pom.xml @@ -35,7 +35,7 @@ org.apache.maven.plugins maven-shade-plugin - 1.4 + 3.0.0 package @@ -43,6 +43,12 @@ shade + + + io.netty + com.dtstack.flink.sql.side.redis.io.netty + + 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 d4d1505f8..5fc8f62b9 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 @@ -18,17 +18,9 @@ package com.dtstack.flink.sql.side.redis; +import com.dtstack.flink.sql.enums.ECacheContentType; import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.BaseAsyncReqRow; -import io.lettuce.core.KeyValue; -import io.lettuce.core.api.async.RedisStringAsyncCommands; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.functions.async.ResultFuture; -import org.apache.flink.table.runtime.types.CRow; -import org.apache.flink.types.Row; - -import com.dtstack.flink.sql.enums.ECacheContentType; import com.dtstack.flink.sql.side.CacheMissVal; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; @@ -38,19 +30,28 @@ import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; import io.lettuce.core.RedisClient; import io.lettuce.core.RedisFuture; +import io.lettuce.core.RedisURI; import io.lettuce.core.api.StatefulRedisConnection; import io.lettuce.core.api.async.RedisHashAsyncCommands; import io.lettuce.core.api.async.RedisKeyAsyncCommands; import io.lettuce.core.cluster.RedisClusterClient; import io.lettuce.core.cluster.api.StatefulRedisClusterConnection; +import io.lettuce.core.internal.HostAndPort; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang.StringUtils; -import com.google.common.collect.Maps; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.runtime.types.CRow; +import org.apache.flink.types.Row; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.function.Consumer; +import java.util.Objects; +import java.util.regex.Matcher; + /** * @author yanxi */ @@ -70,7 +71,7 @@ public class RedisAsyncReqRow extends BaseAsyncReqRow { private RedisSideTableInfo redisSideTableInfo; - private RedisSideReqRow redisSideReqRow; + private final RedisSideReqRow redisSideReqRow; public RedisAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new RedisAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); @@ -87,35 +88,54 @@ public void open(Configuration parameters) throws Exception { private void buildRedisClient(RedisSideTableInfo tableInfo){ String url = redisSideTableInfo.getUrl(); String password = redisSideTableInfo.getPassword(); - if (password != null){ - password = password + "@"; - } else { - password = ""; - } String database = redisSideTableInfo.getDatabase(); if (database == null){ database = "0"; } switch (RedisType.parse(tableInfo.getRedisType())){ case STANDALONE: - StringBuilder redisUri = new StringBuilder(); - redisUri.append("redis://").append(password).append(url).append("/").append(database); - redisClient = RedisClient.create(redisUri.toString()); + RedisURI redisURI = RedisURI.create("redis://" + url); + redisURI.setPassword(password); + redisURI.setDatabase(Integer.parseInt(database)); + redisClient = RedisClient.create(redisURI); connection = redisClient.connect(); async = connection.async(); break; case SENTINEL: - StringBuilder sentinelUri = new StringBuilder(); - sentinelUri.append("redis-sentinel://").append(password) - .append(url).append("/").append(database).append("#").append(redisSideTableInfo.getMasterName()); - redisClient = RedisClient.create(sentinelUri.toString()); + String[] urlSplit = StringUtils.split(url, ","); + RedisURI.Builder builder = null; + for (String item : urlSplit) { + Matcher mather = RedisSideReqRow.HOST_PORT_PATTERN.matcher(item); + if (mather.find()) { + builder = buildSentinelUri( + mather.group("host"), + mather.group("port"), + builder + ); + } else { + throw new IllegalArgumentException( + String.format("Illegal format with redis url [%s]", item) + ); + } + } + + if (Objects.nonNull(builder)) { + builder + .withPassword(tableInfo.getPassword()) + .withDatabase(Integer.parseInt(tableInfo.getDatabase())) + .withSentinelMasterId(tableInfo.getMasterName()); + } else { + throw new NullPointerException("build redis uri error!"); + } + + RedisURI uri = builder.build(); + redisClient = RedisClient.create(uri); connection = redisClient.connect(); async = connection.async(); break; case CLUSTER: - StringBuilder clusterUri = new StringBuilder(); - clusterUri.append("redis://").append(password).append(url); - clusterClient = RedisClusterClient.create(clusterUri.toString()); + List clusterURIs = buildClusterURIs(url); + clusterClient = RedisClusterClient.create(clusterURIs); clusterConnection = clusterClient.connect(); async = clusterConnection.async(); default: @@ -123,6 +143,37 @@ private void buildRedisClient(RedisSideTableInfo tableInfo){ } } + private List buildClusterURIs(String url) { + String password = redisSideTableInfo.getPassword(); + String database = redisSideTableInfo.getDatabase(); + String[] addresses = StringUtils.split(url, ","); + List redisURIs = new ArrayList<>(addresses.length); + for (String addr : addresses){ + HostAndPort hostAndPort = HostAndPort.parse(addr); + RedisURI redisURI = RedisURI.create(hostAndPort.hostText, hostAndPort.port); + if (StringUtils.isNotEmpty(password)) { + redisURI.setPassword(password); + } + if (StringUtils.isNotEmpty(database)) { + redisURI.setDatabase(Integer.valueOf(database)); + } + redisURIs.add(redisURI); + } + return redisURIs; + } + + private RedisURI.Builder buildSentinelUri( + String host, + String port, + RedisURI.Builder builder) { + if (Objects.nonNull(builder)) { + builder.withSentinel(host, Integer.parseInt(port)); + } else { + builder = RedisURI.Builder.sentinel(host, Integer.parseInt(port)); + } + return builder; + } + @Override public Row fillData(Row input, Object sideInput) { return redisSideReqRow.fillData(input, sideInput); @@ -135,21 +186,18 @@ public void handleAsyncInvoke(Map inputParams, CRow input, Resul return; } RedisFuture> future = ((RedisHashAsyncCommands) async).hgetall(key); - future.thenAccept(new Consumer>() { - @Override - public void accept(Map values) { - if (MapUtils.isNotEmpty(values)) { - try { - Row row = fillData(input.row(), values); - dealCacheData(key,CacheObj.buildCacheObj(ECacheContentType.SingleLine, row)); - resultFuture.complete(Collections.singleton(new CRow(row, input.change()))); - } catch (Exception e) { - dealFillDataError(input, resultFuture, e); - } - } else { - dealMissKey(input, resultFuture); - dealCacheData(key,CacheMissVal.getMissKeyObj()); + future.thenAccept(values -> { + if (MapUtils.isNotEmpty(values)) { + try { + Row row = fillData(input.row(), values); + dealCacheData(key,CacheObj.buildCacheObj(ECacheContentType.SingleLine, values)); + resultFuture.complete(Collections.singleton(new CRow(row, input.change()))); + } catch (Exception e) { + dealFillDataError(input, resultFuture, e); } + } else { + dealMissKey(input, resultFuture); + dealCacheData(key,CacheMissVal.getMissKeyObj()); } }); } diff --git a/redis5/redis5-side/redis-side-core/pom.xml b/redis5/redis5-side/redis-side-core/pom.xml index 7fb45a291..88e0a00dd 100644 --- a/redis5/redis5-side/redis-side-core/pom.xml +++ b/redis5/redis5-side/redis-side-core/pom.xml @@ -12,15 +12,6 @@ sql.side.redis.core - - - com.dtstack.flink - sql.core - 1.0-SNAPSHOT - provided - - - jar - + jar \ No newline at end of file diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/enums/RedisType.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/enums/RedisType.java index 048c32b08..614447a5d 100644 --- a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/enums/RedisType.java +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/enums/RedisType.java @@ -28,6 +28,7 @@ public static RedisType parse(int redisType){ return type; } } - throw new RuntimeException("unsupport redis type["+ redisType + "]"); + throw new IllegalArgumentException( + "unsupported redis type["+ redisType + "]"); } } 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 634cc66fa..f22f2bddf 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 @@ -29,6 +29,7 @@ import java.sql.Timestamp; import java.util.Map; import java.util.TimeZone; +import java.util.regex.Pattern; /** * redis fill row data @@ -41,9 +42,11 @@ public class RedisSideReqRow implements ISideReqRow, Serializable { private static final long serialVersionUID = 3751171828444748982L; + public static final Pattern HOST_PORT_PATTERN = Pattern.compile("(?(.*)):(?\\d+)*"); + private static final TimeZone LOCAL_TZ = TimeZone.getDefault(); - private BaseSideInfo sideInfo; + private final BaseSideInfo sideInfo; public RedisSideReqRow(BaseSideInfo sideInfo){ this.sideInfo = sideInfo; @@ -102,7 +105,7 @@ public void setRowField(Row row, Integer index, BaseSideInfo sideInfo, String va row.setField(index, Float.valueOf(value)); break; case "java.math.BigDecimal": - row.setField(index, BigDecimal.valueOf(Long.valueOf(value))); + row.setField(index, BigDecimal.valueOf(Long.parseLong(value))); break; case "java.sql.Timestamp": row.setField(index, Timestamp.valueOf(value)); diff --git a/redis5/redis5-sink/pom.xml b/redis5/redis5-sink/pom.xml index be6a445ed..4fe35fb09 100644 --- a/redis5/redis5-sink/pom.xml +++ b/redis5/redis5-sink/pom.xml @@ -16,20 +16,6 @@ redis-sink http://maven.apache.org - - - com.dtstack.flink - sql.core - 1.0-SNAPSHOT - provided - - - redis.clients - jedis - 2.9.0 - - - 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 ab97cf60e..d86f90685 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 @@ -38,13 +38,17 @@ import java.io.Closeable; import java.io.IOException; import java.util.*; +import java.util.regex.Matcher; +import java.util.regex.Pattern; /** * @author yanxi */ -public class RedisOutputFormat extends AbstractDtRichOutputFormat { +public class RedisOutputFormat extends AbstractDtRichOutputFormat> { private static final Logger LOG = LoggerFactory.getLogger(RedisOutputFormat.class); + private static final Pattern HOST_PORT_PATTERN = Pattern.compile("(?(.*)):(?\\d+)*"); + private String url; private String database = "0"; @@ -77,8 +81,6 @@ public class RedisOutputFormat extends AbstractDtRichOutputFormat { private JedisSentinelPool jedisSentinelPool; - private GenericObjectPoolConfig poolConfig; - private RedisOutputFormat(){ } @Override @@ -107,44 +109,60 @@ private GenericObjectPoolConfig setPoolConfig(String maxTotal, String maxIdle, S } private void establishConnection() { - poolConfig = setPoolConfig(maxTotal, maxIdle, minIdle); + GenericObjectPoolConfig poolConfig = setPoolConfig(maxTotal, maxIdle, minIdle); String[] nodes = StringUtils.split(url, ","); - String[] firstIpPort = StringUtils.split(nodes[0], ":"); - String firstIp = firstIpPort[0]; - String firstPort = firstIpPort[1]; - Set addresses = new HashSet<>(); - Set ipPorts = new HashSet<>(); - for (String ipPort : nodes) { - ipPorts.add(ipPort); - String[] ipPortPair = StringUtils.split(ipPort, ":"); - addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.valueOf(ipPortPair[1].trim()))); - } switch (RedisType.parse(redisType)){ case STANDALONE: - pool = new JedisPool(poolConfig, firstIp, Integer.parseInt(firstPort), timeout, password, Integer.parseInt(database)); - jedis = pool.getResource(); + String firstIp = null; + String firstPort = null; + Matcher standalone = HOST_PORT_PATTERN.matcher(nodes[0]); + if (standalone.find()) { + firstIp = standalone.group("host").trim(); + firstPort = standalone.group("port").trim(); + } + if (Objects.nonNull(firstIp)) { + pool = new JedisPool(poolConfig, firstIp, Integer.parseInt(firstPort), timeout, password, Integer.parseInt(database)); + jedis = pool.getResource(); + } else { + throw new IllegalArgumentException( + String.format("redis url error. current url [%s]", nodes[0])); + } break; case SENTINEL: + Set ipPorts = new HashSet<>(Arrays.asList(nodes)); jedisSentinelPool = new JedisSentinelPool(masterName, ipPorts, poolConfig, timeout, password, Integer.parseInt(database)); jedis = jedisSentinelPool.getResource(); break; case CLUSTER: + Set addresses = new HashSet<>(); + // 对ipv6 支持 + for (String node : nodes) { + Matcher matcher = HOST_PORT_PATTERN.matcher(node); + if (matcher.find()) { + String host = matcher.group("host").trim(); + String portStr = matcher.group("port").trim(); + if (StringUtils.isNotBlank(host) && StringUtils.isNotBlank(portStr)) { + // 转化为int格式的端口 + int port = Integer.parseInt(portStr); + addresses.add(new HostAndPort(host, port)); + } + } + } jedis = new JedisCluster(addresses, timeout, timeout, 10, password, poolConfig); break; default: - throw new RuntimeException("unsupport redis type[ " + redisType + "]"); + throw new IllegalArgumentException("unsupported redis type[ " + redisType + "]"); } } @Override - public void writeRecord(Tuple2 record) throws IOException { - Tuple2 tupleTrans = record; - Boolean retract = tupleTrans.getField(0); + public void writeRecord(Tuple2 record) throws IOException { + Boolean retract = record.getField(0); if (!retract) { return; } - Row row = tupleTrans.getField(1); + Row row = record.getField(1); if (row.getArity() != fieldNames.length) { return; } @@ -153,9 +171,7 @@ public void writeRecord(Tuple2 record) throws IOException { refData.put(fieldNames[i], row.getField(i)); } String redisKey = buildCacheKey(refData); - refData.entrySet().forEach(e ->{ - jedis.hset(redisKey, e.getKey(), String.valueOf(e.getValue())); - }); + refData.forEach((key, value) -> jedis.hset(redisKey, key, String.valueOf(value))); if (outRecords.getCount() % ROW_PRINT_FREQUENCY == 0){ LOG.info(record.toString()); 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 cc49a3ba8..f99e69725 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 @@ -33,6 +33,8 @@ import org.apache.flink.types.Row; import java.util.List; +import java.util.Objects; + /** * @author yanxi */ @@ -64,6 +66,10 @@ public class RedisSink implements RetractStreamTableSink, IStreamSinkGener< protected String masterName; + protected Integer parallelism = 1; + + protected String registerTableName; + public RedisSink(){ } @@ -82,6 +88,9 @@ public RedisSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { this.minIdle = redisTableInfo.getMinIdle(); this.masterName = redisTableInfo.getMasterName(); this.timeout = redisTableInfo.getTimeout(); + this.parallelism = Objects.isNull(redisTableInfo.getParallelism()) ? + parallelism : redisTableInfo.getParallelism(); + this.registerTableName = redisTableInfo.getName(); return this; } @@ -108,7 +117,7 @@ public void emitDataStream(DataStream> dataStream) { .setMasterName(this.masterName); RedisOutputFormat redisOutputFormat = builder.finish(); RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(redisOutputFormat); - dataStream.addSink(richSinkFunction); + dataStream.addSink(richSinkFunction).setParallelism(parallelism).name(registerTableName); } @Override diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java index 8961f7da9..44d127884 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java @@ -51,7 +51,7 @@ public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map primaryKeysList = Lists.newArrayList(); + List primaryKeysList; primaryKeysList = Arrays.asList(StringUtils.split(primaryKeysStr, ",")); redisTableInfo.setPrimaryKeys(primaryKeysList); }